Add support for multiplex worker sandboxing. This sandboxing requires worker support.
Design doc: https://docs.google.com/document/d/1ncLW0hz6uDhNvci1dpzfEoifwTiNTqiBEm1vi-bIIRM/edit
The multiplex worker process will run in a subdir "<mnemonic>-multiplex-workdir/<workspacename>" under the global execroot. Inside that directory, each request gets its own subdir "sandbox/<workerid>/<workspacename>".
RELNOTES: Adds `--experimental_worker_multiplex_sandboxing` flag that controls whether to sandbox multiplex workers that support it.
PiperOrigin-RevId: 419591728
diff --git a/site/docs/creating-workers.md b/site/docs/creating-workers.md
index 2653929..fdcbf6a 100644
--- a/site/docs/creating-workers.md
+++ b/site/docs/creating-workers.md
@@ -59,6 +59,9 @@
Bazel sets the `verbosity` field to 10, but smaller or larger values can be
used manually for different amounts of output.
+The optional `sandbox_dir` field is used only by workers that support
+[multiplex sandboxing](multiplex-worker.md).
+
### Work responses
A `WorkResponse` contains a request id, a zero or nonzero exit
diff --git a/site/docs/multiplex-worker.md b/site/docs/multiplex-worker.md
index 34bfb15..ba8acb6 100644
--- a/site/docs/multiplex-worker.md
+++ b/site/docs/multiplex-worker.md
@@ -66,14 +66,33 @@
pressure and improve performance. However, multiplex workers are not currently
compatible with [dynamic execution](/dynamic-execution.html).
-### Warning about rare bug
+## Multiplex sandboxing
-Due to a rare bug, multiplex workers are currently not enabled by default.
-Occasionally, Bazel hangs indefinitely at the execution phase. If you see this
-behavior, stop the Bazel server and rerun. This delay is probably caused by
+Multiplex workers can be sandboxed by adding explicit support for it in the
+worker implementations. While singleplex worker sandboxing can be done by
+running each worker process in its own sandbox, multiplex workers share the
+process working directory between multiple parallel requests. To allow
+sandboxing of multiplex workers, the worker must support reading from and
+writing to a subdirectory specified in each request, instead of directly in
+its working directory.
- * Multiplex workers waiting for responses from the worker process that never
- comes.
- * Incorrectly configured ruleset worker implementation where a thread dies or
- a race condition occurs. To counteract this, ensure the worker process
- returns responses in all circumstances.
+To support multiplex sandboxing, the worker must use the `sandbox_dir` field
+from the `WorkRequest` and use that as a prefix for all file reads and writes.
+While the `arguments` and `inputs` fields remain unchanged from an unsandboxed
+request, the actual inputs are relative to the `sandbox_dir`. The worker must
+translate file paths found in `arguments` and `inputs` to read from this
+modified path, and must also write all outputs relative to the `sandbox_dir`.
+This includes paths such as '.', as well as paths found in files specified
+in the arguments (e.g. ["argfile"](https://docs.oracle.com/javase/7/docs/technotes/tools/windows/javac.html#commandlineargfile) arguments).
+
+Once a worker supports multiplex sandboxing, the ruleset can declare this
+support by adding `supports-multiplex-sandboxing` to the
+`execution_requirements` of an action. Bazel will then use multiplex sandboxing
+if the `--experimental_worker_multiplex_sandboxing` flag is passed, or if
+the worker is used with dynamic execution.
+
+The worker files of a sandboxed multiplex worker are still relative to the
+working directory of the worker process. Thus, if a file is
+used both for running the worker and as an input, it must be specified both as
+an input in the flagfile argument as well as in `tools`, `executable`, or
+`runfiles`.
diff --git a/site/docs/persistent-workers.md b/site/docs/persistent-workers.md
index 06059af..84d7532 100644
--- a/site/docs/persistent-workers.md
+++ b/site/docs/persistent-workers.md
@@ -250,6 +250,11 @@
workers offer less strict sandboxing than a pure sandbox, because the tool may
keep other internal state that has been affected by previous requests.
+Multiplex workers can only be sandboxed if the worker implementation support it,
+and this sandboxing must be separately enabled with the
+`--experimental_worker_multiplex_sandboxing` flag. See more details in
+[the design doc](https://docs.google.com/document/d/1ncLW0hz6uDhNvci1dpzfEoifwTiNTqiBEm1vi-bIIRM/edit))
+
## Further reading <a name="further-reading"></a>
For more information on persistent workers, see:
diff --git a/src/main/java/com/google/devtools/build/lib/actions/ExecutionRequirements.java b/src/main/java/com/google/devtools/build/lib/actions/ExecutionRequirements.java
index a21f141..ffcc82d 100644
--- a/src/main/java/com/google/devtools/build/lib/actions/ExecutionRequirements.java
+++ b/src/main/java/com/google/devtools/build/lib/actions/ExecutionRequirements.java
@@ -162,6 +162,8 @@
public static final String SUPPORTS_WORKER_CANCELLATION = "supports-worker-cancellation";
+ public static final String SUPPORTS_MULTIPLEX_SANDBOXING = "supports-multiplex-sandboxing";
+
/** Denotes what the type of worker protocol the worker uses. */
public enum WorkerProtocolFormat {
JSON,
diff --git a/src/main/java/com/google/devtools/build/lib/actions/Spawns.java b/src/main/java/com/google/devtools/build/lib/actions/Spawns.java
index 82db8bd..1ea5abc 100644
--- a/src/main/java/com/google/devtools/build/lib/actions/Spawns.java
+++ b/src/main/java/com/google/devtools/build/lib/actions/Spawns.java
@@ -102,6 +102,15 @@
}
/**
+ * Returns whether the {@link Spawn} supports sandboxing for multiplex workers through the {@code
+ * WorkRequest.sandbox_dir} field.
+ */
+ public static boolean supportsMultiplexSandboxing(Spawn spawn) {
+ return "1"
+ .equals(spawn.getExecutionInfo().get(ExecutionRequirements.SUPPORTS_MULTIPLEX_SANDBOXING));
+ }
+
+ /**
* Returns which worker protocol format a Spawn claims a persistent worker uses. Defaults to proto
* if the protocol format is not specified.
*/
diff --git a/src/main/java/com/google/devtools/build/lib/sandbox/AbstractContainerizingSandboxedSpawn.java b/src/main/java/com/google/devtools/build/lib/sandbox/AbstractContainerizingSandboxedSpawn.java
index 9b09b64..352db02 100644
--- a/src/main/java/com/google/devtools/build/lib/sandbox/AbstractContainerizingSandboxedSpawn.java
+++ b/src/main/java/com/google/devtools/build/lib/sandbox/AbstractContainerizingSandboxedSpawn.java
@@ -15,7 +15,6 @@
package com.google.devtools.build.lib.sandbox;
-import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@@ -27,7 +26,6 @@
import com.google.devtools.build.lib.vfs.Path;
import com.google.devtools.build.lib.vfs.PathFragment;
import java.io.IOException;
-import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.Set;
import java.util.stream.Collectors;
@@ -115,7 +113,7 @@
filterInputsAndDirsToCreate(inputsToCreate, dirsToCreate);
// Finally create what needs creating.
- createDirectories(dirsToCreate);
+ SandboxHelpers.createDirectories(dirsToCreate, sandboxExecRoot, /* strict=*/ true);
createInputs(inputsToCreate, inputs);
inputs.materializeVirtualInputs(sandboxExecRoot);
}
@@ -125,45 +123,6 @@
throws IOException {}
/**
- * Creates all directories needed for the sandbox.
- *
- * <p>No input can be a child of another input, because otherwise we might try to create a symlink
- * below another symlink we created earlier - which means we'd actually end up writing somewhere
- * in the workspace.
- *
- * <p>If all inputs were regular files, this situation could naturally not happen - but
- * unfortunately, we might get the occasional action that has directories in its inputs.
- *
- * <p>Creating all parent directories first ensures that we can safely create symlinks to
- * directories, too, because we'll get an IOException with EEXIST if inputs happen to be nested
- * once we start creating the symlinks for all inputs.
- */
- void createDirectories(Iterable<PathFragment> dirsToCreate) throws IOException {
- Set<Path> knownDirectories = new HashSet<>();
- // Add sandboxExecRoot and it's parent -- all paths must fall under the parent of
- // sandboxExecRoot and we know that sandboxExecRoot exists. This stops the recursion in
- // createDirectoryAndParentsInSandboxRoot.
- knownDirectories.add(sandboxExecRoot);
- knownDirectories.add(sandboxExecRoot.getParentDirectory());
-
- for (PathFragment path : dirsToCreate) {
- Preconditions.checkArgument(!path.isAbsolute(), path);
- if (path.containsUplevelReferences() && path.isMultiSegment()) {
- // Allow a single up-level reference to allow inputs from the siblings of the main
- // repository in the sandbox execution root, but forbid multiple up-level references.
- // PathFragment is normalized, so up-level references are guaranteed to be at the beginning.
- Preconditions.checkArgument(
- !PathFragment.containsUplevelReferences(path.getSegment(1)),
- "%s escapes the sandbox exec root.",
- path);
- }
-
- SandboxHelpers.createDirectoryAndParentsInSandboxRoot(
- sandboxExecRoot.getRelative(path), knownDirectories, sandboxExecRoot);
- }
- }
-
- /**
* Creates all inputs needed for this spawn's sandbox.
*
* @param inputsToCreate The inputs that actually need to be created. Some inputs may already
diff --git a/src/main/java/com/google/devtools/build/lib/sandbox/SandboxHelpers.java b/src/main/java/com/google/devtools/build/lib/sandbox/SandboxHelpers.java
index 4d088b0..ea9f071 100644
--- a/src/main/java/com/google/devtools/build/lib/sandbox/SandboxHelpers.java
+++ b/src/main/java/com/google/devtools/build/lib/sandbox/SandboxHelpers.java
@@ -19,9 +19,11 @@
import static com.google.devtools.build.lib.vfs.Dirent.Type.SYMLINK;
import com.google.auto.value.AutoValue;
+import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
import com.google.common.flogger.GoogleLogger;
import com.google.devtools.build.lib.actions.ActionInput;
import com.google.devtools.build.lib.actions.Artifact;
@@ -309,6 +311,51 @@
knownDirectories.add(path);
}
+ /**
+ * Creates all directories needed for the sandbox.
+ *
+ * <p>No input can be a child of another input, because otherwise we might try to create a symlink
+ * below another symlink we created earlier - which means we'd actually end up writing somewhere
+ * in the workspace.
+ *
+ * <p>If all inputs were regular files, this situation could naturally not happen - but
+ * unfortunately, we might get the occasional action that has directories in its inputs.
+ *
+ * <p>Creating all parent directories first ensures that we can safely create symlinks to
+ * directories, too, because we'll get an IOException with EEXIST if inputs happen to be nested
+ * once we start creating the symlinks for all inputs.
+ *
+ * @param strict If true, absolute directories or directories with multiple up-level references
+ * are disallowed, for stricter sandboxing.
+ */
+ public static void createDirectories(
+ Iterable<PathFragment> dirsToCreate, Path dir, boolean strict) throws IOException {
+ Set<Path> knownDirectories = new HashSet<>();
+ // Add sandboxExecRoot and it's parent -- all paths must fall under the parent of
+ // sandboxExecRoot and we know that sandboxExecRoot exists. This stops the recursion in
+ // createDirectoryAndParentsInSandboxRoot.
+ knownDirectories.add(dir);
+ knownDirectories.add(dir.getParentDirectory());
+
+ for (PathFragment path : dirsToCreate) {
+ if (strict) {
+ Preconditions.checkArgument(!path.isAbsolute(), path);
+ if (path.containsUplevelReferences() && path.isMultiSegment()) {
+ // Allow a single up-level reference to allow inputs from the siblings of the main
+ // repository in the sandbox execution root, but forbid multiple up-level references.
+ // PathFragment is normalized, so up-level references are guaranteed to be at the
+ // beginning.
+ Preconditions.checkArgument(
+ !PathFragment.containsUplevelReferences(path.getSegment(1)),
+ "%s escapes the sandbox exec root.",
+ path);
+ }
+ }
+
+ createDirectoryAndParentsInSandboxRoot(dir.getRelative(path), knownDirectories, dir);
+ }
+ }
+
/** Wrapper class for the inputs of a sandbox. */
public static final class SandboxInputs {
@@ -397,6 +444,17 @@
}
}
+ /**
+ * Returns a new SandboxInputs instance with only the inputs/symlinks listed in {@code allowed}
+ * included.
+ */
+ public SandboxInputs limitedCopy(Set<PathFragment> allowed) {
+ return new SandboxInputs(
+ Maps.filterKeys(files, allowed::contains),
+ ImmutableSet.of(),
+ Maps.filterKeys(symlinks, allowed::contains));
+ }
+
@Override
public String toString() {
return "Files: " + files + "\nVirtualInputs: " + virtualInputs + "\nSymlinks: " + symlinks;
diff --git a/src/main/java/com/google/devtools/build/lib/worker/SandboxedWorkerProxy.java b/src/main/java/com/google/devtools/build/lib/worker/SandboxedWorkerProxy.java
new file mode 100644
index 0000000..fbad965
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/worker/SandboxedWorkerProxy.java
@@ -0,0 +1,116 @@
+// Copyright 2021 The Bazel Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.devtools.build.lib.worker;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.flogger.GoogleLogger;
+import com.google.devtools.build.lib.sandbox.SandboxHelpers;
+import com.google.devtools.build.lib.sandbox.SandboxHelpers.SandboxInputs;
+import com.google.devtools.build.lib.sandbox.SandboxHelpers.SandboxOutputs;
+import com.google.devtools.build.lib.vfs.Path;
+import com.google.devtools.build.lib.vfs.PathFragment;
+import com.google.devtools.build.lib.worker.WorkerProtocol.WorkRequest;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+/**
+ * A multiplex worker proxy with sandboxing. The multiplexer process runs in {@code workDir}, while
+ * each proxy has a fixed subdir where it sets up its files. The subdir is then passed to the worker
+ * in {@link WorkRequest#sandbox_dir}. The worker implementation is responsible for reading from and
+ * writing to that subdir only.
+ */
+public class SandboxedWorkerProxy extends WorkerProxy {
+ private static final GoogleLogger logger = GoogleLogger.forEnclosingClass();
+ /** The sandbox directory for the current request, inside {@code workDir}. */
+ private final Path sandboxDir;
+
+ private final PathFragment sandboxName;
+
+ SandboxedWorkerProxy(
+ WorkerKey workerKey,
+ int workerId,
+ Path logFile,
+ WorkerMultiplexer workerMultiplexer,
+ Path workDir) {
+ super(workerKey, workerId, logFile, workerMultiplexer, workDir);
+ sandboxName =
+ PathFragment.create(
+ Joiner.on(PathFragment.SEPARATOR_CHAR)
+ .join(
+ "__sandbox",
+ Integer.toString(workerId),
+ workerKey.getExecRoot().getBaseName()));
+ sandboxDir = this.workDir.getRelative(sandboxName);
+ }
+
+ @Override
+ public boolean isSandboxed() {
+ return true;
+ }
+
+ @Override
+ public void prepareExecution(
+ SandboxInputs inputFiles, SandboxOutputs outputs, Set<PathFragment> workerFiles)
+ throws IOException {
+ workerMultiplexer.createSandboxedProcess(workDir, workerFiles, inputFiles);
+
+ sandboxDir.createDirectoryAndParents();
+ LinkedHashSet<PathFragment> dirsToCreate = new LinkedHashSet<>();
+ Set<PathFragment> inputsToCreate = new HashSet<>();
+
+ SandboxHelpers.populateInputsAndDirsToCreate(
+ ImmutableSet.of(),
+ inputsToCreate,
+ dirsToCreate,
+ Iterables.concat(inputFiles.getFiles().keySet(), inputFiles.getSymlinks().keySet()),
+ outputs.files(),
+ outputs.dirs());
+ SandboxHelpers.cleanExisting(
+ sandboxDir.getParentDirectory(), inputFiles, inputsToCreate, dirsToCreate, sandboxDir);
+ // Finally, create anything that is still missing. This is non-strict only for historical
+ // reasons, we haven't seen what would break if we make it strict.
+ SandboxHelpers.createDirectories(dirsToCreate, sandboxDir, /* strict=*/ false);
+ WorkerExecRoot.createInputs(inputsToCreate, inputFiles, sandboxDir);
+ }
+
+ /** Send the WorkRequest to multiplexer. */
+ @Override
+ protected void putRequest(WorkRequest request) throws IOException {
+ // Modifying the request on the way out is not great. The alternatives are having the
+ // spawn runner ask the worker for the dir or making the spawn runner understand the sandbox,
+ // dir structure, neither of which are nice either.
+ workerMultiplexer.putRequest(
+ request.toBuilder().setSandboxDir(sandboxName.getPathString()).build());
+ }
+
+ @Override
+ public void finishExecution(Path execRoot, SandboxOutputs outputs) throws IOException {
+ SandboxHelpers.moveOutputs(outputs, sandboxDir, execRoot);
+ }
+
+ @Override
+ synchronized void destroy() {
+ super.destroy();
+ try {
+ workDir.deleteTree();
+ } catch (IOException e) {
+ logger.atWarning().withCause(e).log("Caught IOException while deleting workdir.");
+ }
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/lib/worker/Worker.java b/src/main/java/com/google/devtools/build/lib/worker/Worker.java
index 781cd8f..86ab2e6 100644
--- a/src/main/java/com/google/devtools/build/lib/worker/Worker.java
+++ b/src/main/java/com/google/devtools/build/lib/worker/Worker.java
@@ -103,7 +103,12 @@
*/
abstract WorkResponse getResponse(int requestId) throws IOException, InterruptedException;
- /** Does whatever cleanup may be required after execution is done. */
+ /**
+ * Does whatever cleanup may be required after execution is done.
+ *
+ * @param execRoot The global execRoot, where outputs must go.
+ * @param outputs The expected outputs.
+ */
public abstract void finishExecution(Path execRoot, SandboxOutputs outputs) throws IOException;
/**
diff --git a/src/main/java/com/google/devtools/build/lib/worker/WorkerExecRoot.java b/src/main/java/com/google/devtools/build/lib/worker/WorkerExecRoot.java
index e0b8e25..b2e23e3 100644
--- a/src/main/java/com/google/devtools/build/lib/worker/WorkerExecRoot.java
+++ b/src/main/java/com/google/devtools/build/lib/worker/WorkerExecRoot.java
@@ -22,7 +22,6 @@
import com.google.devtools.build.lib.vfs.Path;
import com.google.devtools.build.lib.vfs.PathFragment;
import java.io.IOException;
-import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.Set;
@@ -59,30 +58,19 @@
SandboxHelpers.cleanExisting(
workDir.getParentDirectory(), inputs, inputsToCreate, dirsToCreate, workDir);
- // Finally, create anything that is still missing.
- createDirectories(dirsToCreate);
- createInputs(inputsToCreate, inputs);
+ // Finally, create anything that is still missing. This is non-strict only for historical
+ // reasons,
+ // we haven't seen what would break if we make it strict.
+ SandboxHelpers.createDirectories(dirsToCreate, workDir, /* strict=*/ false);
+ createInputs(inputsToCreate, inputs, workDir);
inputs.materializeVirtualInputs(workDir);
}
- private void createDirectories(Iterable<PathFragment> dirsToCreate) throws IOException {
- Set<Path> knownDirectories = new HashSet<>();
- // Add sandboxExecRoot and it's parent -- all paths must fall under the parent of
- // sandboxExecRoot and we know that sandboxExecRoot exists. This stops the recursion in
- // createDirectoryAndParentsInSandboxRoot.
- knownDirectories.add(workDir);
- knownDirectories.add(workDir.getParentDirectory());
- for (PathFragment fragment : dirsToCreate) {
- SandboxHelpers.createDirectoryAndParentsInSandboxRoot(
- workDir.getRelative(fragment), knownDirectories, workDir);
- }
- }
-
- private void createInputs(Iterable<PathFragment> inputsToCreate, SandboxInputs inputs)
+ static void createInputs(Iterable<PathFragment> inputsToCreate, SandboxInputs inputs, Path dir)
throws IOException {
for (PathFragment fragment : inputsToCreate) {
- Path key = workDir.getRelative(fragment);
+ Path key = dir.getRelative(fragment);
if (inputs.getFiles().containsKey(fragment)) {
Path fileDest = inputs.getFiles().get(fragment);
if (fileDest != null) {
diff --git a/src/main/java/com/google/devtools/build/lib/worker/WorkerFactory.java b/src/main/java/com/google/devtools/build/lib/worker/WorkerFactory.java
index 36ce023..bb73223 100644
--- a/src/main/java/com/google/devtools/build/lib/worker/WorkerFactory.java
+++ b/src/main/java/com/google/devtools/build/lib/worker/WorkerFactory.java
@@ -66,11 +66,19 @@
Worker worker;
if (key.isSandboxed()) {
- Path workDir = getSandboxedWorkerPath(key, workerId);
- worker = new SandboxedWorker(key, workerId, workDir, logFile);
+ if (key.isMultiplex()) {
+ WorkerMultiplexer workerMultiplexer = WorkerMultiplexerManager.getInstance(key, logFile);
+ Path workDir = getSandboxedWorkerPath(key);
+ worker = new SandboxedWorkerProxy(key, workerId, logFile, workerMultiplexer, workDir);
+ } else {
+ Path workDir = getSandboxedWorkerPath(key, workerId);
+ worker = new SandboxedWorker(key, workerId, workDir, logFile);
+ }
} else if (key.isMultiplex()) {
WorkerMultiplexer workerMultiplexer = WorkerMultiplexerManager.getInstance(key, logFile);
- worker = new WorkerProxy(key, workerId, workerMultiplexer.getLogFile(), workerMultiplexer);
+ worker =
+ new WorkerProxy(
+ key, workerId, workerMultiplexer.getLogFile(), workerMultiplexer, key.getExecRoot());
} else {
worker = new SingleplexWorker(key, workerId, key.getExecRoot(), logFile);
}
diff --git a/src/main/java/com/google/devtools/build/lib/worker/WorkerMultiplexer.java b/src/main/java/com/google/devtools/build/lib/worker/WorkerMultiplexer.java
index f894762..24b22bd 100644
--- a/src/main/java/com/google/devtools/build/lib/worker/WorkerMultiplexer.java
+++ b/src/main/java/com/google/devtools/build/lib/worker/WorkerMultiplexer.java
@@ -16,20 +16,28 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
import com.google.devtools.build.lib.events.Event;
import com.google.devtools.build.lib.events.EventHandler;
+import com.google.devtools.build.lib.sandbox.SandboxHelpers;
+import com.google.devtools.build.lib.sandbox.SandboxHelpers.SandboxInputs;
+import com.google.devtools.build.lib.sandbox.SandboxHelpers.SandboxOutputs;
import com.google.devtools.build.lib.shell.Subprocess;
import com.google.devtools.build.lib.shell.SubprocessBuilder;
import com.google.devtools.build.lib.shell.SubprocessFactory;
import com.google.devtools.build.lib.vfs.Path;
+import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.worker.WorkerProtocol.WorkRequest;
import com.google.devtools.build.lib.worker.WorkerProtocol.WorkResponse;
import java.io.File;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
import java.util.List;
import java.util.Optional;
+import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -51,7 +59,7 @@
* send them to the worker. This prevents dynamic execution interrupts from corrupting the {@code
* stdin} of the worker process.
*/
- private final BlockingQueue<WorkRequest> pendingRequests = new LinkedBlockingQueue<>();
+ @VisibleForTesting final BlockingQueue<WorkRequest> pendingRequests = new LinkedBlockingQueue<>();
/**
* A map of {@code WorkResponse}s received from the worker process. They are stored in this map
* keyed by the request id until the corresponding {@code WorkerProxy} picks them up.
@@ -70,7 +78,7 @@
* once, when creating a new process. If the process dies or its stdio streams get corrupted, the
* {@code WorkerMultiplexer} gets discarded as well and a new one gets created as needed.
*/
- private Subprocess process;
+ @VisibleForTesting Subprocess process;
/** The implementation of the worker protocol (JSON or Proto). */
private WorkerProtocolImpl workerProtocol;
/** InputStream from the worker process. */
@@ -127,6 +135,35 @@
/**
* Creates a worker process corresponding to this {@code WorkerMultiplexer}, if it doesn't already
+ * exist. Also starts up the subthreads handling reading and writing requests and responses, and
+ * sets up the sandbox root dir with the required worker files.
+ */
+ public synchronized void createSandboxedProcess(
+ Path workDir, Set<PathFragment> workerFiles, SandboxInputs inputFiles) throws IOException {
+ // TODO: Make blaze clean remove the workdir.
+ if (this.process == null) {
+ // This should be a once-only operation.
+ workDir.createDirectoryAndParents();
+ workDir.deleteTreesBelow();
+ LinkedHashSet<PathFragment> dirsToCreate = new LinkedHashSet<>();
+ Set<PathFragment> inputsToCreate = new HashSet<>();
+ SandboxHelpers.populateInputsAndDirsToCreate(
+ ImmutableSet.of(),
+ inputsToCreate,
+ dirsToCreate,
+ workerFiles,
+ SandboxOutputs.getEmptyInstance().files(),
+ SandboxOutputs.getEmptyInstance().dirs());
+ SandboxHelpers.cleanExisting(
+ workDir.getParentDirectory(), inputFiles, inputsToCreate, dirsToCreate, workDir);
+ SandboxHelpers.createDirectories(dirsToCreate, workDir, /* strict=*/ false);
+ WorkerExecRoot.createInputs(inputsToCreate, inputFiles.limitedCopy(workerFiles), workDir);
+ createProcess(workDir);
+ }
+ }
+
+ /**
+ * Creates a worker process corresponding to this {@code WorkerMultiplexer}, if it doesn't already
* exist. Also starts up the subthreads handling reading and writing requests and responses.
*/
public synchronized void createProcess(Path workDir) throws IOException {
@@ -412,4 +449,10 @@
}
return process.getProcessId();
}
+
+ // TODO: Check if this can be removed
+ @VisibleForTesting
+ Subprocess getProcess() {
+ return process;
+ }
}
diff --git a/src/main/java/com/google/devtools/build/lib/worker/WorkerMultiplexerManager.java b/src/main/java/com/google/devtools/build/lib/worker/WorkerMultiplexerManager.java
index 3a58a96..d34e84c 100644
--- a/src/main/java/com/google/devtools/build/lib/worker/WorkerMultiplexerManager.java
+++ b/src/main/java/com/google/devtools/build/lib/worker/WorkerMultiplexerManager.java
@@ -47,7 +47,8 @@
*/
public static synchronized WorkerMultiplexer getInstance(WorkerKey key, Path logFile) {
InstanceInfo instanceInfo =
- multiplexerInstance.computeIfAbsent(key, k -> new InstanceInfo(logFile, k));
+ multiplexerInstance.computeIfAbsent(
+ key, k -> new InstanceInfo(new WorkerMultiplexer(logFile, k)));
instanceInfo.increaseRefCount();
return instanceInfo.getWorkerMultiplexer();
}
@@ -75,7 +76,8 @@
InstanceInfo instanceInfo = multiplexerInstance.get(key);
if (instanceInfo == null) {
throw createUserExecException(
- String.format("Attempting to remove non-existent multiplexer instance for %s.", key),
+ String.format(
+ "Attempting to remove non-existent %s multiplexer instance.", key.getMnemonic()),
Code.MULTIPLEXER_INSTANCE_REMOVAL_FAILURE);
}
instanceInfo.decreaseRefCount();
@@ -123,8 +125,8 @@
private final WorkerMultiplexer workerMultiplexer;
private Integer refCount;
- public InstanceInfo(Path logFile, WorkerKey workerKey) {
- this.workerMultiplexer = new WorkerMultiplexer(logFile, workerKey);
+ public InstanceInfo(WorkerMultiplexer workerMultiplexer) {
+ this.workerMultiplexer = workerMultiplexer;
this.refCount = 0;
}
@@ -147,10 +149,16 @@
/** Resets the instances. For testing only. */
@VisibleForTesting
- static void reset() {
+ static void resetForTesting() {
for (InstanceInfo i : multiplexerInstance.values()) {
i.workerMultiplexer.destroyMultiplexer();
}
multiplexerInstance.clear();
}
+
+ /** Injects a given WorkerMultiplexer into the instance map with refcount 0. For testing only. */
+ @VisibleForTesting
+ static synchronized void injectForTesting(WorkerKey key, WorkerMultiplexer multiplexer) {
+ multiplexerInstance.put(key, new InstanceInfo(multiplexer));
+ }
}
diff --git a/src/main/java/com/google/devtools/build/lib/worker/WorkerOptions.java b/src/main/java/com/google/devtools/build/lib/worker/WorkerOptions.java
index 6a2b9b9..2fede9b 100644
--- a/src/main/java/com/google/devtools/build/lib/worker/WorkerOptions.java
+++ b/src/main/java/com/google/devtools/build/lib/worker/WorkerOptions.java
@@ -186,4 +186,15 @@
effectTags = {OptionEffectTag.EXECUTION},
help = "If enabled, Bazel may send cancellation requests to workers that support them.")
public boolean workerCancellation;
+
+ @Option(
+ name = "experimental_worker_multiplex_sandboxing",
+ defaultValue = "false",
+ documentationCategory = OptionDocumentationCategory.UNCATEGORIZED,
+ effectTags = {OptionEffectTag.EXECUTION},
+ help =
+ "If enabled, multiplex workers will be sandboxed, using a separate sandbox directory"
+ + " per work request. Only workers that have the 'supports-multiplex-sandboxing' "
+ + "execution requirement will be sandboxed.")
+ public boolean multiplexSandboxing;
}
diff --git a/src/main/java/com/google/devtools/build/lib/worker/WorkerParser.java b/src/main/java/com/google/devtools/build/lib/worker/WorkerParser.java
index a45cdde..e0889d9 100644
--- a/src/main/java/com/google/devtools/build/lib/worker/WorkerParser.java
+++ b/src/main/java/com/google/devtools/build/lib/worker/WorkerParser.java
@@ -50,7 +50,9 @@
/** Pattern for @flagfile.txt and --flagfile=flagfile.txt */
private static final Pattern FLAG_FILE_PATTERN = Pattern.compile("(?:@|--?flagfile=)(.+)");
+ /** The global execRoot. */
private final Path execRoot;
+
private final WorkerOptions workerOptions;
private final LocalEnvProvider localEnvProvider;
private final BinTools binTools;
@@ -125,6 +127,17 @@
WorkerOptions options,
boolean dynamic,
WorkerProtocolFormat protocolFormat) {
+ boolean multiplex = options.workerMultiplex && Spawns.supportsMultiplexWorkers(spawn);
+ if (dynamic && !(Spawns.supportsMultiplexSandboxing(spawn) && options.multiplexSandboxing)) {
+ multiplex = false;
+ }
+ boolean sandboxed;
+ if (multiplex) {
+ sandboxed =
+ Spawns.supportsMultiplexSandboxing(spawn) && (options.multiplexSandboxing || dynamic);
+ } else {
+ sandboxed = options.workerSandboxing || dynamic;
+ }
return new WorkerKey(
workerArgs,
env,
@@ -132,11 +145,8 @@
Spawns.getWorkerKeyMnemonic(spawn),
workerFilesCombinedHash,
workerFiles,
- /* sandboxed= */ options.workerSandboxing || dynamic,
- /* multiplex= */ options.workerMultiplex
- && Spawns.supportsMultiplexWorkers(spawn)
- && !dynamic
- && !options.workerSandboxing,
+ sandboxed,
+ multiplex,
Spawns.supportsWorkerCancellation(spawn),
protocolFormat);
}
diff --git a/src/main/java/com/google/devtools/build/lib/worker/WorkerProxy.java b/src/main/java/com/google/devtools/build/lib/worker/WorkerProxy.java
index 726f645..92fb4cb 100644
--- a/src/main/java/com/google/devtools/build/lib/worker/WorkerProxy.java
+++ b/src/main/java/com/google/devtools/build/lib/worker/WorkerProxy.java
@@ -28,16 +28,20 @@
import java.util.Set;
/** A proxy that talks to the multiplexer */
-final class WorkerProxy extends Worker {
+class WorkerProxy extends Worker {
private static final GoogleLogger logger = GoogleLogger.forEnclosingClass();
- private final WorkerMultiplexer workerMultiplexer;
- /** The execution root of the worker. */
- private final Path workDir;
+ protected final WorkerMultiplexer workerMultiplexer;
+ /** The execution root of the worker. This is the CWD of the worker process. */
+ protected final Path workDir;
WorkerProxy(
- WorkerKey workerKey, int workerId, Path logFile, WorkerMultiplexer workerMultiplexer) {
+ WorkerKey workerKey,
+ int workerId,
+ Path logFile,
+ WorkerMultiplexer workerMultiplexer,
+ Path workDir) {
super(workerKey, workerId, logFile);
- this.workDir = workerKey.getExecRoot();
+ this.workDir = workDir;
this.workerMultiplexer = workerMultiplexer;
}
@@ -52,7 +56,6 @@
workerMultiplexer.setReporter(reporter);
}
- @Override
public void prepareExecution(
SandboxInputs inputFiles, SandboxOutputs outputs, Set<PathFragment> workerFiles)
throws IOException {
@@ -70,7 +73,7 @@
/** Send the WorkRequest to multiplexer. */
@Override
- void putRequest(WorkRequest request) throws IOException {
+ protected void putRequest(WorkRequest request) throws IOException {
workerMultiplexer.putRequest(request);
}
@@ -81,7 +84,7 @@
}
@Override
- public void finishExecution(Path execRoot, SandboxOutputs outputs) {}
+ public void finishExecution(Path execRoot, SandboxOutputs outputs) throws IOException {}
@Override
boolean diedUnexpectedly() {
diff --git a/src/main/protobuf/worker_protocol.proto b/src/main/protobuf/worker_protocol.proto
index 6d3d1aa..90a15c8 100644
--- a/src/main/protobuf/worker_protocol.proto
+++ b/src/main/protobuf/worker_protocol.proto
@@ -54,10 +54,21 @@
// The arguments and inputs fields must be empty and should be ignored.
bool cancel = 4;
- // Values greater than 0 indicate that the worker should output extra debug
+ // Values greater than 0 indicate that the worker may output extra debug
// information to stderr (which will go into the worker log). Setting the
// --worker_verbose flag for Bazel makes this flag default to 10.
int32 verbosity = 5;
+
+ // The relative directory inside the workers working directory where the
+ // inputs and outputs are placed, for sandboxing purposes. For singleplex
+ // workers, this is unset, as they can use their working directory as sandbox.
+ // For multiplex workers, this will be set when the
+ // --experimental_worker_multiplex_sandbox flag is set _and_ the execution
+ // requirements for the worker includes 'supports-multiplex-sandbox'.
+ // The paths in `inputs` will not contain this prefix, but the actual files
+ // will be placed/must be written relative to this directory. The worker
+ // implementation is responsible for resolving the file paths.
+ string sandbox_dir = 6;
}
// The worker sends this message to Blaze when it finished its work on the
diff --git a/src/test/java/com/google/devtools/build/lib/worker/ExampleWorker.java b/src/test/java/com/google/devtools/build/lib/worker/ExampleWorker.java
index b172e16..0385f04 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/ExampleWorker.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/ExampleWorker.java
@@ -30,6 +30,7 @@
import java.io.BufferedReader;
import java.io.BufferedWriter;
import java.io.ByteArrayOutputStream;
+import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStreamWriter;
@@ -194,6 +195,7 @@
try {
if (currentRequest.getVerbosity() > 0) {
originalStdErr.println("VERBOSE: Pretending to do work.");
+ originalStdErr.println("VERBOSE: Running in " + new File(".").getAbsolutePath());
}
parseOptionsAndLog(args);
} catch (Exception e) {
diff --git a/src/test/java/com/google/devtools/build/lib/worker/ExampleWorkerMultiplexer.java b/src/test/java/com/google/devtools/build/lib/worker/ExampleWorkerMultiplexer.java
index f49652f..54d7c15 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/ExampleWorkerMultiplexer.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/ExampleWorkerMultiplexer.java
@@ -25,9 +25,11 @@
import com.google.devtools.build.lib.worker.WorkerProtocol.WorkResponse;
import com.google.devtools.common.options.OptionsParser;
import java.io.ByteArrayOutputStream;
+import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
import java.nio.file.Files;
+import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.LinkedHashMap;
@@ -43,8 +45,13 @@
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import java.util.stream.Collectors;
-/** An example implementation of a worker process that is used for integration tests. */
+/**
+ * An example implementation of a multiplex worker process that is used for integration tests. By
+ * default, it concatenates writes the options residue and outputs it on stdout. {@link
+ * ExampleWorkerMultiplexerOptions} specifies ways the behaviour can be modofied.
+ */
public class ExampleWorkerMultiplexer {
static final Pattern FLAG_FILE_PATTERN = Pattern.compile("(?:@|--?flagfile=)(.+)");
@@ -54,6 +61,7 @@
// A UUID that uniquely identifies this running worker process.
static final UUID WORKER_UUID = UUID.randomUUID();
+ public static final String FILE_INPUT_PREFIX = "FILE:";
// A counter that increases with each work unit processed.
static int workUnitCounter = 1;
@@ -82,7 +90,7 @@
runPersistentWorker(workerOptions);
} else {
// This is a single invocation of the example that exits after it processed the request.
- processRequest(parserHelper(ImmutableList.copyOf(args)));
+ processRequest(parserHelper(ImmutableList.copyOf(args)), WorkRequest.getDefaultInstance());
}
}
@@ -127,7 +135,8 @@
}
results.add(
executorService.submit(
- createTask(originalStdOut, originalStdErr, requestId, parser, poisoned)));
+ createTask(
+ originalStdOut, originalStdErr, requestId, parser, poisoned, request)));
} catch (Exception e) {
e.printStackTrace();
exitCode = 1;
@@ -180,7 +189,8 @@
PrintStream originalStdErr,
int requestId,
OptionsParser parser,
- boolean poisoned) {
+ boolean poisoned,
+ WorkRequest request) {
return () -> {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
int exitCode = 0;
@@ -201,7 +211,12 @@
System.out.write(b);
} else {
try {
- processRequest(parser);
+ if (request.getVerbosity() > 0) {
+ originalStdErr.println("VERBOSE: Pretending to do work.");
+ originalStdErr.println("VERBOSE: Running in " + new File(".").getAbsolutePath());
+ originalStdErr.println("VERBOSE: Args " + request.getArgumentsList());
+ }
+ processRequest(parser, request);
} catch (Exception e) {
e.printStackTrace();
exitCode = 1;
@@ -231,7 +246,7 @@
};
}
- private static void processRequest(OptionsParser parser) throws Exception {
+ private static void processRequest(OptionsParser parser, WorkRequest request) throws Exception {
ExampleWorkMultiplexerOptions options = parser.getOptions(ExampleWorkMultiplexerOptions.class);
List<String> outputs = new ArrayList<>();
@@ -250,11 +265,30 @@
outputs.add("COUNTER " + counterOutput);
}
- String residueStr = Joiner.on(' ').join(parser.getResidue());
+ List<String> residue = parser.getResidue();
+ List<String> paths =
+ residue.stream().filter(s -> s.startsWith(FILE_INPUT_PREFIX)).collect(Collectors.toList());
+ residue =
+ residue.stream().filter(p -> !paths.contains(p)).collect(ImmutableList.toImmutableList());
+
+ String residueStr = Joiner.on(' ').join(residue);
if (options.uppercase) {
residueStr = residueStr.toUpperCase();
}
outputs.add(residueStr);
+ String prefix = options.ignoreSandbox ? "" : request.getSandboxDir();
+ while (prefix.endsWith("/")) {
+ prefix = prefix.substring(0, prefix.length() - 1);
+ }
+ for (String p : paths) {
+ Path path = Paths.get(prefix, p.substring(FILE_INPUT_PREFIX.length()));
+ List<String> lines = Files.readAllLines(path);
+ String content = Joiner.on("\n").join(lines);
+ if (options.uppercase) {
+ content = content.toUpperCase();
+ }
+ outputs.add(content);
+ }
if (options.printInputs) {
for (Map.Entry<String, String> input : inputs.entrySet()) {
@@ -272,7 +306,8 @@
if (options.outputFile.isEmpty()) {
System.out.println(outputStr);
} else {
- try (PrintStream outputFile = new PrintStream(options.outputFile)) {
+ String actualFile = prefix.isEmpty() ? options.outputFile : prefix + "/" + options.outputFile;
+ try (PrintStream outputFile = new PrintStream(actualFile)) {
outputFile.println(outputStr);
}
}
diff --git a/src/test/java/com/google/devtools/build/lib/worker/ExampleWorkerMultiplexerOptions.java b/src/test/java/com/google/devtools/build/lib/worker/ExampleWorkerMultiplexerOptions.java
index 98da65a..777ddd4 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/ExampleWorkerMultiplexerOptions.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/ExampleWorkerMultiplexerOptions.java
@@ -78,6 +78,14 @@
defaultValue = "false",
help = "Randomly delay the worker response (between 100 to 300 ms).")
public boolean delay;
+
+ @Option(
+ name = "ignore_sandbox",
+ documentationCategory = OptionDocumentationCategory.UNCATEGORIZED,
+ effectTags = {OptionEffectTag.NO_OP},
+ defaultValue = "false",
+ help = "Ignore the sandbox settings in work requests.")
+ public boolean ignoreSandbox;
}
@Option(
diff --git a/src/test/java/com/google/devtools/build/lib/worker/SandboxedWorkerProxyTest.java b/src/test/java/com/google/devtools/build/lib/worker/SandboxedWorkerProxyTest.java
new file mode 100644
index 0000000..b0270c7
--- /dev/null
+++ b/src/test/java/com/google/devtools/build/lib/worker/SandboxedWorkerProxyTest.java
@@ -0,0 +1,193 @@
+// Copyright 2021 The Bazel Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.devtools.build.lib.worker;
+
+import static com.google.common.truth.Truth.assertThat;
+import static com.google.devtools.build.lib.actions.ExecutionRequirements.SUPPORTS_MULTIPLEX_SANDBOXING;
+import static com.google.devtools.build.lib.actions.ExecutionRequirements.WorkerProtocolFormat.PROTO;
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.devtools.build.lib.actions.Spawn;
+import com.google.devtools.build.lib.vfs.DigestHashFunction;
+import com.google.devtools.build.lib.vfs.FileSystem;
+import com.google.devtools.build.lib.vfs.FileSystemUtils;
+import com.google.devtools.build.lib.vfs.Path;
+import com.google.devtools.build.lib.vfs.inmemoryfs.InMemoryFileSystem;
+import com.google.devtools.build.lib.worker.TestUtils.FakeSubprocess;
+import com.google.devtools.build.lib.worker.WorkerProtocol.WorkRequest;
+import java.io.IOException;
+import java.io.PipedInputStream;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for WorkerProxy */
+@RunWith(JUnit4.class)
+public class SandboxedWorkerProxyTest {
+ final FileSystem fs = new InMemoryFileSystem(DigestHashFunction.SHA256);
+ private Path globalExecRoot;
+ private Path workerBaseDir;
+ private Path globalOutputBase;
+
+ @Before
+ public void setUp() throws IOException {
+ Path testRoot = fs.getPath(com.google.devtools.build.lib.testutil.TestUtils.tmpDir());
+
+ globalOutputBase = testRoot.getChild("outputbase");
+ globalExecRoot = globalOutputBase.getChild("execroot");
+ globalExecRoot.createDirectoryAndParents();
+
+ workerBaseDir = testRoot.getRelative("bazel-workers");
+ workerBaseDir.createDirectoryAndParents();
+ }
+
+ @Test
+ public void prepareExecution_createsFilesInSandbox() throws IOException {
+ SandboxedWorkerProxy proxy = createSandboxedWorkerProxy();
+ Path workDir = workerBaseDir.getChild("Mnem-multiplex-worker-workdir").getChild("execroot");
+ Path sandboxDir =
+ workDir
+ .getChild("__sandbox")
+ .getChild(Integer.toString(proxy.getWorkerId()))
+ .getChild("execroot");
+ SandboxHelper sandboxHelper =
+ new SandboxHelper(globalExecRoot, workDir)
+ .addAndCreateInputFile("anInputFile", "anInputFile", "Just stuff")
+ // Worker files are expected to also be inputs.
+ .addInputFile("worker.sh", "worker.sh")
+ .addOutput("very/output.txt")
+ .addAndCreateWorkerFile("worker.sh", "#!/bin/bash");
+
+ PipedInputStream serverInputStream = new PipedInputStream();
+ proxy.workerMultiplexer.setProcessFactory(params -> new FakeSubprocess(serverInputStream));
+
+ proxy.prepareExecution(
+ sandboxHelper.getSandboxInputs(),
+ sandboxHelper.getSandboxOutputs(),
+ sandboxHelper.getWorkerFiles());
+
+ assertThat(workDir.isDirectory()).isTrue();
+ assertThat(workDir.getChild("worker.sh").exists()).isTrue();
+ assertThat(workDir.getChild("worker.sh").isSymbolicLink()).isTrue();
+ assertThat(sandboxDir.isDirectory()).isTrue();
+ assertThat(sandboxDir.getChild("anInputFile").exists()).isTrue();
+ assertThat(sandboxDir.getChild("anInputFile").isSymbolicLink()).isTrue();
+ assertThat(sandboxDir.getChild("very").exists()).isTrue();
+ assertThat(sandboxDir.getChild("very").isDirectory()).isTrue();
+ }
+
+ @Test
+ public void putRequest_setsSandboxDir() throws IOException, InterruptedException {
+ SandboxedWorkerProxy worker = createFakedSandboxedWorkerProxy();
+ Path workDir = workerBaseDir.getChild("Mnem-multiplex-worker-workdir").getChild("execroot");
+ SandboxHelper sandboxHelper =
+ new SandboxHelper(globalExecRoot, workDir)
+ .addAndCreateInputFile("anInputFile", "anInputFile", "Just stuff")
+ .addOutput("very/output.txt")
+ .addAndCreateWorkerFile("worker.sh", "#!/bin/bash");
+ worker.prepareExecution(
+ sandboxHelper.getSandboxInputs(),
+ sandboxHelper.getSandboxOutputs(),
+ sandboxHelper.getWorkerFiles());
+ worker.putRequest(WorkRequest.newBuilder().setRequestId(2).build());
+ assertThat(worker.workerMultiplexer.pendingRequests).isNotEmpty();
+ WorkRequest actualRequest = worker.workerMultiplexer.pendingRequests.take();
+ assertThat(actualRequest.getRequestId()).isEqualTo(2);
+ assertThat(actualRequest.getSandboxDir())
+ .isEqualTo("__sandbox/" + worker.getWorkerId() + "/execroot");
+ }
+
+ @Test
+ public void finishExecution_copiesOutputs() throws IOException, InterruptedException {
+ SandboxedWorkerProxy worker = createFakedSandboxedWorkerProxy();
+ Path workDir = workerBaseDir.getChild("Mnem-multiplex-worker-workdir").getChild("execroot");
+ SandboxHelper sandboxHelper =
+ new SandboxHelper(globalExecRoot, workDir)
+ .addAndCreateInputFile("anInputFile", "anInputFile", "Just stuff")
+ .addOutput("very/output.txt")
+ .addOutput("rootFile")
+ .addAndCreateWorkerFile("worker.sh", "#!/bin/bash");
+ worker.prepareExecution(
+ sandboxHelper.getSandboxInputs(),
+ sandboxHelper.getSandboxOutputs(),
+ sandboxHelper.getWorkerFiles());
+ worker.putRequest(WorkRequest.newBuilder().setRequestId(2).build());
+ WorkRequest actualRequest = worker.workerMultiplexer.pendingRequests.take();
+ String requestSandboxSubdir = actualRequest.getSandboxDir();
+
+ // Pretend to do work.
+ sandboxHelper.createExecRootFile(
+ Joiner.on("/").join(requestSandboxSubdir, "very/output.txt"), "some output");
+ sandboxHelper.createExecRootFile("very/output.txt", "some wrongly placed output");
+ sandboxHelper.createExecRootFile(
+ Joiner.on("/").join(requestSandboxSubdir, "rootFile"), "some output in root");
+ sandboxHelper.createExecRootFile(
+ Joiner.on("/").join(requestSandboxSubdir, "randomFile"), "some randomOutput");
+
+ worker.finishExecution(globalExecRoot, sandboxHelper.getSandboxOutputs());
+
+ assertThat(globalExecRoot.getChild("randomFile").exists()).isFalse();
+ assertThat(FileSystemUtils.readContent(globalExecRoot.getChild("rootFile"), UTF_8))
+ .isEqualTo("some output in root");
+ assertThat(
+ FileSystemUtils.readContent(
+ globalExecRoot.getChild("very").getChild("output.txt"), UTF_8))
+ .isEqualTo("some output");
+ }
+
+ private SandboxedWorkerProxy createSandboxedWorkerProxy() throws IOException {
+ ImmutableMap.Builder<String, String> req = TestUtils.execRequirementsBuilder("Mnem");
+ req.put(SUPPORTS_MULTIPLEX_SANDBOXING, "1");
+ Spawn spawn = TestUtils.createSpawn(req.buildOrThrow());
+
+ WorkerOptions options = new WorkerOptions();
+ options.workerMultiplex = true;
+ options.multiplexSandboxing = true;
+
+ WorkerKey key =
+ TestUtils.createWorkerKeyFromOptions(
+ PROTO, globalOutputBase, options, true, spawn, "worker.sh");
+ WorkerFactory factory = new WorkerFactory(workerBaseDir);
+ return (SandboxedWorkerProxy) factory.create(key);
+ }
+
+ private SandboxedWorkerProxy createFakedSandboxedWorkerProxy() throws IOException {
+ ImmutableMap.Builder<String, String> req = TestUtils.execRequirementsBuilder("Mnem");
+ req.put(SUPPORTS_MULTIPLEX_SANDBOXING, "1");
+ Spawn spawn = TestUtils.createSpawn(req.buildOrThrow());
+
+ WorkerOptions options = new WorkerOptions();
+ options.workerMultiplex = true;
+ options.multiplexSandboxing = true;
+
+ WorkerKey key =
+ TestUtils.createWorkerKeyFromOptions(
+ PROTO, globalOutputBase, options, true, spawn, "worker.sh");
+ WorkerMultiplexerManager.injectForTesting(
+ key,
+ new WorkerMultiplexer(globalExecRoot.getChild("testWorker.log"), key) {
+ @Override
+ public synchronized void createProcess(Path workDir) throws IOException {
+ PipedInputStream serverInputStream = new PipedInputStream();
+ super.process = new FakeSubprocess(serverInputStream);
+ }
+ });
+ WorkerFactory factory = new WorkerFactory(workerBaseDir);
+ return (SandboxedWorkerProxy) factory.create(key);
+ }
+}
diff --git a/src/test/java/com/google/devtools/build/lib/worker/TestUtils.java b/src/test/java/com/google/devtools/build/lib/worker/TestUtils.java
index 5f3b845..b4398b4 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/TestUtils.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/TestUtils.java
@@ -67,19 +67,67 @@
workerOptions.workerMultiplex = multiplex;
workerOptions.workerSandboxing = sandboxed;
- Spawn spawn =
- createSpawn(
- ImmutableMap.of(
- ExecutionRequirements.WORKER_KEY_MNEMONIC, mnemonic,
- ExecutionRequirements.REQUIRES_WORKER_PROTOCOL, "proto",
- ExecutionRequirements.SUPPORTS_WORKERS, "1",
- ExecutionRequirements.SUPPORTS_MULTIPLEX_WORKERS, "1"));
+ return createWorkerKeyFromOptions(
+ protocolFormat,
+ fs.getPath("/outputbase"),
+ workerOptions,
+ dynamic,
+ createSpawn(execRequirementsBuilder(mnemonic).buildOrThrow()),
+ args);
+ }
+
+ /**
+ * Creates a worker key based on a set of options. The {@code extraRequirements} are added to the
+ * {@link Spawn} execution info with the value "1". The "supports-workers" and
+ * "supports-multiplex-workers" execution requirements are always set.
+ *
+ * @param outputBase Global (for the test) outputBase.
+ */
+ static WorkerKey createWorkerKeyWithRequirements(
+ Path outputBase,
+ WorkerOptions workerOptions,
+ String mnemonic,
+ boolean dynamic,
+ String... extraRequirements) {
+ ImmutableMap.Builder<String, String> builder = execRequirementsBuilder(mnemonic);
+ for (String req : extraRequirements) {
+ builder.put(req, "1");
+ }
+ Spawn spawn = createSpawn(builder.buildOrThrow());
+
+ return WorkerParser.createWorkerKey(
+ spawn,
+ /* workerArgs= */ ImmutableList.of(),
+ /* env= */ ImmutableMap.of("env1", "foo", "env2", "bar"),
+ /* execRoot= */ outputBase.getChild("execroot"),
+ /* workerFilesCombinedHash= */ HashCode.fromInt(0),
+ /* workerFiles= */ ImmutableSortedMap.of(),
+ workerOptions,
+ dynamic,
+ WorkerProtocolFormat.PROTO);
+ }
+
+ static ImmutableMap.Builder<String, String> execRequirementsBuilder(String mnemonic) {
+ return ImmutableMap.<String, String>builder()
+ .put(ExecutionRequirements.WORKER_KEY_MNEMONIC, mnemonic)
+ .put(ExecutionRequirements.REQUIRES_WORKER_PROTOCOL, "proto")
+ .put(ExecutionRequirements.SUPPORTS_WORKERS, "1")
+ .put(ExecutionRequirements.SUPPORTS_MULTIPLEX_WORKERS, "1");
+ }
+
+ static WorkerKey createWorkerKeyFromOptions(
+ WorkerProtocolFormat protocolFormat,
+ Path outputBase,
+ WorkerOptions workerOptions,
+ boolean dynamic,
+ Spawn spawn,
+ String... args) {
return WorkerParser.createWorkerKey(
spawn,
/* workerArgs= */ ImmutableList.copyOf(args),
/* env= */ ImmutableMap.of("env1", "foo", "env2", "bar"),
- /* execRoot= */ fs.getPath("/outputbase/execroot/workspace"),
+ /* execRoot= */ outputBase.getChild("execroot"),
/* workerFilesCombinedHash= */ HashCode.fromInt(0),
/* workerFiles= */ ImmutableSortedMap.of(),
workerOptions,
diff --git a/src/test/java/com/google/devtools/build/lib/worker/WorkerFactoryTest.java b/src/test/java/com/google/devtools/build/lib/worker/WorkerFactoryTest.java
index c6fd54e..1759495 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/WorkerFactoryTest.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/WorkerFactoryTest.java
@@ -39,7 +39,7 @@
@After
public void tearDown() {
- WorkerMultiplexerManager.reset();
+ WorkerMultiplexerManager.resetForTesting();
}
/**
diff --git a/src/test/java/com/google/devtools/build/lib/worker/WorkerModuleTest.java b/src/test/java/com/google/devtools/build/lib/worker/WorkerModuleTest.java
index 67d65c7..c0ed65e 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/WorkerModuleTest.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/WorkerModuleTest.java
@@ -38,6 +38,7 @@
import com.google.devtools.build.lib.vfs.Path;
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.inmemoryfs.InMemoryFileSystem;
+import com.google.devtools.common.options.OptionsParsingException;
import com.google.devtools.common.options.OptionsParsingResult;
import java.io.IOException;
import org.junit.Before;
@@ -88,7 +89,7 @@
}
@Test
- public void buildStarting_noRestartOnSandboxChanges() throws IOException, AbruptExitException {
+ public void buildStarting_noRestartOnSandboxChange() throws IOException, AbruptExitException {
WorkerModule module = new WorkerModule();
WorkerOptions options = WorkerOptions.DEFAULTS;
when(request.getOptions(WorkerOptions.class)).thenReturn(options);
@@ -113,7 +114,7 @@
@Test
public void buildStarting_workersDestroyedOnRestart()
- throws IOException, AbruptExitException, InterruptedException {
+ throws IOException, AbruptExitException, InterruptedException, OptionsParsingException {
WorkerModule module = new WorkerModule();
WorkerOptions options = WorkerOptions.DEFAULTS;
options.workerVerbose = true;
diff --git a/src/test/java/com/google/devtools/build/lib/worker/WorkerMultiplexerManagerTest.java b/src/test/java/com/google/devtools/build/lib/worker/WorkerMultiplexerManagerTest.java
index 88d2dc1..ceaff39 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/WorkerMultiplexerManagerTest.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/WorkerMultiplexerManagerTest.java
@@ -43,12 +43,12 @@
@Before
public void setUp() {
fileSystem = new InMemoryFileSystem(BlazeClock.instance(), DigestHashFunction.SHA256);
- WorkerMultiplexerManager.reset();
+ WorkerMultiplexerManager.resetForTesting();
}
@After
public void tearDown() {
- WorkerMultiplexerManager.reset();
+ WorkerMultiplexerManager.resetForTesting();
}
@Test
diff --git a/src/test/java/com/google/devtools/build/lib/worker/WorkerMultiplexerTest.java b/src/test/java/com/google/devtools/build/lib/worker/WorkerMultiplexerTest.java
index dacd11d..89fb7f2 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/WorkerMultiplexerTest.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/WorkerMultiplexerTest.java
@@ -55,7 +55,7 @@
@After
public void tearDown() {
- WorkerMultiplexerManager.reset();
+ WorkerMultiplexerManager.resetForTesting();
}
@Test
@@ -68,7 +68,8 @@
multiplexer.setProcessFactory(params -> new FakeSubprocess(serverInputStream));
WorkRequest request1 = WorkRequest.newBuilder().setRequestId(1).build();
- WorkerProxy worker = new WorkerProxy(workerKey, 2, logPath, multiplexer);
+ WorkerProxy worker =
+ new WorkerProxy(workerKey, 2, logPath, multiplexer, workerKey.getExecRoot());
worker.prepareExecution(null, null, null);
worker.putRequest(request1);
WorkResponse response1 = WorkResponse.newBuilder().setRequestId(1).build();
@@ -91,12 +92,14 @@
OutputStream workerOutputStream = new PipedOutputStream(serverInputStream);
multiplexer.setProcessFactory(params -> new FakeSubprocess(serverInputStream));
- WorkerProxy worker1 = new WorkerProxy(workerKey, 1, logPath, multiplexer);
+ WorkerProxy worker1 =
+ new WorkerProxy(workerKey, 1, logPath, multiplexer, workerKey.getExecRoot());
worker1.prepareExecution(null, null, null);
WorkRequest request1 = WorkRequest.newBuilder().setRequestId(3).build();
worker1.putRequest(request1);
- WorkerProxy worker2 = new WorkerProxy(workerKey, 2, logPath, multiplexer);
+ WorkerProxy worker2 =
+ new WorkerProxy(workerKey, 2, logPath, multiplexer, workerKey.getExecRoot());
worker2.prepareExecution(null, null, null);
WorkRequest request2 = WorkRequest.newBuilder().setRequestId(42).build();
worker2.putRequest(request2);
@@ -123,16 +126,18 @@
WorkerKey workerKey = TestUtils.createWorkerKey(fileSystem, "test3", true, "fakeBinary");
WorkerMultiplexer multiplexer = WorkerMultiplexerManager.getInstance(workerKey, logPath);
- PipedInputStream serverInputStrean = new PipedInputStream();
- OutputStream workerOutputStream = new PipedOutputStream(serverInputStrean);
- multiplexer.setProcessFactory(params -> new FakeSubprocess(serverInputStrean));
+ PipedInputStream serverInputStream = new PipedInputStream();
+ OutputStream workerOutputStream = new PipedOutputStream(serverInputStream);
+ multiplexer.setProcessFactory(params -> new FakeSubprocess(serverInputStream));
- WorkerProxy worker1 = new WorkerProxy(workerKey, 1, logPath, multiplexer);
+ WorkerProxy worker1 =
+ new WorkerProxy(workerKey, 1, logPath, multiplexer, workerKey.getExecRoot());
worker1.prepareExecution(null, null, null);
WorkRequest request1 = WorkRequest.newBuilder().setRequestId(3).build();
worker1.putRequest(request1);
- WorkerProxy worker2 = new WorkerProxy(workerKey, 2, logPath, multiplexer);
+ WorkerProxy worker2 =
+ new WorkerProxy(workerKey, 2, logPath, multiplexer, workerKey.getExecRoot());
worker2.prepareExecution(null, null, null);
WorkRequest request2 = WorkRequest.newBuilder().setRequestId(42).build();
worker2.putRequest(request2);
@@ -184,12 +189,14 @@
OutputStream workerOutputStream = new PipedOutputStream(serverInputStream);
multiplexer.setProcessFactory(params -> new FakeSubprocess(serverInputStream));
- WorkerProxy worker1 = new WorkerProxy(workerKey, 1, logPath, multiplexer);
+ WorkerProxy worker1 =
+ new WorkerProxy(workerKey, 1, logPath, multiplexer, workerKey.getExecRoot());
worker1.prepareExecution(null, null, null);
WorkRequest request1 = WorkRequest.newBuilder().setRequestId(3).build();
worker1.putRequest(request1);
- WorkerProxy worker2 = new WorkerProxy(workerKey, 2, logPath, multiplexer);
+ WorkerProxy worker2 =
+ new WorkerProxy(workerKey, 2, logPath, multiplexer, workerKey.getExecRoot());
worker2.prepareExecution(null, null, null);
WorkRequest request2 = WorkRequest.newBuilder().setRequestId(42).build();
worker2.putRequest(request2);
diff --git a/src/test/java/com/google/devtools/build/lib/worker/WorkerParserTest.java b/src/test/java/com/google/devtools/build/lib/worker/WorkerParserTest.java
index c4acf99..dfa8ccb 100644
--- a/src/test/java/com/google/devtools/build/lib/worker/WorkerParserTest.java
+++ b/src/test/java/com/google/devtools/build/lib/worker/WorkerParserTest.java
@@ -15,12 +15,11 @@
package com.google.devtools.build.lib.worker;
import static com.google.common.truth.Truth.assertThat;
+import static com.google.devtools.build.lib.actions.ExecutionRequirements.SUPPORTS_MULTIPLEX_SANDBOXING;
-import com.google.devtools.build.lib.actions.ExecException;
import com.google.devtools.build.lib.vfs.DigestHashFunction;
import com.google.devtools.build.lib.vfs.FileSystem;
import com.google.devtools.build.lib.vfs.inmemoryfs.InMemoryFileSystem;
-import java.io.IOException;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
@@ -32,7 +31,7 @@
final FileSystem fs = new InMemoryFileSystem(DigestHashFunction.SHA256);
@Test
- public void workerKeyComputationCheck() throws ExecException, IOException, InterruptedException {
+ public void workerKeyComputationCheck() {
WorkerKey keyNomultiNoSandboxedNoDynamic = TestUtils.createWorkerKey(fs, false, false, false);
assertThat(keyNomultiNoSandboxedNoDynamic.isMultiplex()).isFalse();
assertThat(keyNomultiNoSandboxedNoDynamic.isSandboxed()).isFalse();
@@ -49,9 +48,9 @@
assertThat(keyNomultiSandboxedNoDynamic.getWorkerTypeName()).isEqualTo("worker");
WorkerKey keyMultiSandboxedNoDynamic = TestUtils.createWorkerKey(fs, true, true, false);
- assertThat(keyMultiSandboxedNoDynamic.isMultiplex()).isFalse();
- assertThat(keyMultiSandboxedNoDynamic.isSandboxed()).isTrue();
- assertThat(keyMultiSandboxedNoDynamic.getWorkerTypeName()).isEqualTo("worker");
+ assertThat(keyMultiSandboxedNoDynamic.isMultiplex()).isTrue();
+ assertThat(keyMultiSandboxedNoDynamic.isSandboxed()).isFalse();
+ assertThat(keyMultiSandboxedNoDynamic.getWorkerTypeName()).isEqualTo("multiplex-worker");
WorkerKey keyNomultiNoSandboxedDynamic = TestUtils.createWorkerKey(fs, false, false, true);
assertThat(keyNomultiNoSandboxedDynamic.isMultiplex()).isFalse();
@@ -73,4 +72,52 @@
assertThat(keyMultiSandboxedDynamic.isSandboxed()).isTrue();
assertThat(keyMultiSandboxedDynamic.getWorkerTypeName()).isEqualTo("worker");
}
+
+ @Test
+ public void createWorkerKey_understandsMultiplexSandboxing() {
+ WorkerOptions options = new WorkerOptions();
+ options.multiplexSandboxing = false;
+ options.workerMultiplex = true;
+
+ WorkerKey keyNoMultiplexSandboxing =
+ TestUtils.createWorkerKeyWithRequirements(fs.getPath("/outputbase"), options, "Nom", false);
+ assertThat(keyNoMultiplexSandboxing.isMultiplex()).isTrue();
+ assertThat(keyNoMultiplexSandboxing.isSandboxed()).isFalse();
+ assertThat(keyNoMultiplexSandboxing.getWorkerTypeName()).isEqualTo("multiplex-worker");
+
+ WorkerKey keyForcedSandboxedDynamic =
+ TestUtils.createWorkerKeyWithRequirements(fs.getPath("/outputbase"), options, "Nom", true);
+ assertThat(keyForcedSandboxedDynamic.isMultiplex()).isFalse();
+ assertThat(keyForcedSandboxedDynamic.isSandboxed()).isTrue();
+ assertThat(keyForcedSandboxedDynamic.getWorkerTypeName()).isEqualTo("worker");
+
+ WorkerKey keyForcedeMultiplexSandboxing =
+ TestUtils.createWorkerKeyWithRequirements(
+ fs.getPath("/outputbase"), options, "Nom", true, SUPPORTS_MULTIPLEX_SANDBOXING);
+ assertThat(keyForcedeMultiplexSandboxing.isMultiplex()).isFalse();
+ assertThat(keyForcedeMultiplexSandboxing.isSandboxed()).isTrue();
+ assertThat(keyForcedeMultiplexSandboxing.getWorkerTypeName()).isEqualTo("worker");
+
+ options.multiplexSandboxing = true;
+
+ WorkerKey keyBaseMultiplexNoSandbox =
+ TestUtils.createWorkerKeyWithRequirements(fs.getPath("/outputbase"), options, "Nom", false);
+ assertThat(keyBaseMultiplexNoSandbox.isMultiplex()).isTrue();
+ assertThat(keyBaseMultiplexNoSandbox.isSandboxed()).isFalse();
+ assertThat(keyBaseMultiplexNoSandbox.getWorkerTypeName()).isEqualTo("multiplex-worker");
+
+ WorkerKey keyBaseMultiplexSandboxing =
+ TestUtils.createWorkerKeyWithRequirements(
+ fs.getPath("/outputbase"), options, "Nom", false, SUPPORTS_MULTIPLEX_SANDBOXING);
+ assertThat(keyBaseMultiplexSandboxing.isMultiplex()).isTrue();
+ assertThat(keyBaseMultiplexSandboxing.isSandboxed()).isTrue();
+ assertThat(keyBaseMultiplexSandboxing.getWorkerTypeName()).isEqualTo("multiplex-worker");
+
+ WorkerKey keyDynamicMultiplexSandboxing =
+ TestUtils.createWorkerKeyWithRequirements(
+ fs.getPath("/outputbase"), options, "Nom", true, SUPPORTS_MULTIPLEX_SANDBOXING);
+ assertThat(keyDynamicMultiplexSandboxing.isMultiplex()).isTrue();
+ assertThat(keyDynamicMultiplexSandboxing.isSandboxed()).isTrue();
+ assertThat(keyDynamicMultiplexSandboxing.getWorkerTypeName()).isEqualTo("multiplex-worker");
+ }
}
diff --git a/src/test/shell/integration/bazel_worker_multiplexer_test.sh b/src/test/shell/integration/bazel_worker_multiplexer_test.sh
index bd00eae..b41efdb 100755
--- a/src/test/shell/integration/bazel_worker_multiplexer_test.sh
+++ b/src/test/shell/integration/bazel_worker_multiplexer_test.sh
@@ -90,7 +90,7 @@
executable=worker,
progress_message="Working on %s" % ctx.label.name,
mnemonic="Work",
- execution_requirements={"supports-multiplex-workers": "1"},
+ execution_requirements={"supports-multiplex-workers": "1", "supports-multiplex-sandboxing": "1"},
arguments=ctx.attr.worker_args + argfile_arguments,
)
@@ -555,4 +555,71 @@
bazel build :hello_world_1 :hello_world_2 :hello_world_3 &> $TEST_log \
|| fail "build failed"
}
+
+# This is just to test that file handling in multiplex example worker works.
+function test_multiplexer_files() {
+ prepare_example_worker
+
+ mkdir -p dir1/dir2
+ echo "base file" > file.txt
+ echo "subdir file" > dir1/dir2/file.txt
+
+ cat >>BUILD <<EOF
+work(
+ name = "hello_world",
+ worker = ":worker",
+ srcs = [":file.txt", ":dir1/dir2/file.txt"],
+ args = ["hello world", "FILE:${WORKSPACE_SUBDIR}/file.txt", "FILE:${WORKSPACE_SUBDIR}/dir1/dir2/file.txt"],
+)
+EOF
+
+ bazel build :hello_world &> $TEST_log \
+ || (sleep 1 && fail "build failed")
+ assert_equals "hello world base file subdir file" "$( echo $(cat $BINS/hello_world.out))"
+}
+
+function test_sandboxed_multiplexer_files() {
+ prepare_example_worker
+ echo "base file" > file.txt
+
+ mkdir -p ${PWD}/dir1/dir2
+ echo "base file" > file.txt
+ echo "subdir file" > dir1/dir2/file.txt
+ cat >>BUILD <<EOF
+work(
+ name = "hello_world",
+ worker = ":worker",
+ srcs = [":file.txt", ":dir1/dir2/file.txt"],
+ args = ["hello world", "FILE:${WORKSPACE_SUBDIR}/file.txt", "FILE:${WORKSPACE_SUBDIR}/dir1/dir2/file.txt"],
+)
+
+EOF
+
+ bazel build :hello_world --experimental_worker_multiplex_sandboxing &> $TEST_log \
+ || fail "build failed"
+ assert_equals "hello world base file subdir file" "$( echo $(cat $BINS/hello_world.out))"
+}
+
+function test_sandboxed_multiplexer_files_fails_if_ignoring_sandbox() {
+ prepare_example_worker
+ echo "base file" > file.txt
+
+ mkdir -p ${PWD}/dir1/dir2
+ echo "base file" > file.txt
+ echo "subdir file" > dir1/dir2/file.txt
+ cat >>BUILD <<EOF
+work(
+ name = "hello_world_ignore_sandbox",
+ worker = ":worker",
+ srcs = [":file.txt", ":dir1/dir2/file.txt"],
+ args = ["--ignore_sandbox", "hello world", "FILE:${WORKSPACE_SUBDIR}/file.txt", "FILE:${WORKSPACE_SUBDIR}/dir1/dir2/file.txt"],
+)
+
+EOF
+
+ bazel build :hello_world_ignore_sandbox --experimental_worker_multiplex_sandboxing &> $TEST_log \
+ && fail "expected build to fail"
+ expect_log "java.nio.file.NoSuchFileException"
+}
+
run_suite "Worker multiplexer integration tests"