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"