Support remote execution in repository_rule

Motivation:

This change adds support for running a command on a remote machine from a
repository_rule and getting its stdout, stderr and exit code back. The
initial use case of this feature is to allow a repository_rule to inspect
a remote machine, discover the installed tools, libraries and environment
in order to generate Bazel platforms and toolchains for the remote
environment.

The upload of files as part of the command and download of its outputs is
not supported. The use cases looked at so far don't have a need for it.
For example, Bazel's own cc_configure rule only uses one file as part of
ctx.execute(). The contents of the file are "int main() {}". There the
lack of file inputs can simply be worked around with
"echo 'int main() {}' > empty.cc && gcc empty.cc". I am not aware of a
use case that would require downloading output files.

How to Use:

This feature is not on by default. It can be enabled with
--experimental_repository_remote_exec.

repository_rule gains an additional attribute remotable. If true any
repository_ctx.execute() call may be executed on a remote worker. That
is, if Bazel is configured build/test with remote execution so is
repository_ctx.execute():

def _cc_configure_impl(ctx):
  result = ctx.execute(["/bin/bash", "-c", "export"])
  print("stdout: " + result.stdout)
  print("stderr: " + result.stderr)
  print("exit code: " + result.return_code)

cc_configure = repository_rule(
  implementation = _cc_configure_impl,
  remotable = True,
)

Note: The 'remotable' attribute is set by rule author as any such rule needs
to be carefully crafted to work with both local and remote execution.

The remote platform of repository_ctx.execute() is specified via the newly
introduced exec_properties attribute on the repository_rule target:

load(..., "cc_configure")

cc_configure(
  name = "remote_cc_configure",
  exec_properties = {
    "OSFamily" : "Linux",
    "ISA" : "x86-64"
  }
)

The intended usage is for the repository_rule to generate the Bazel
platform and toolchains for the remote platform that the build/test
actions will be run on:

bazel build \
  --experimental_repository_remote_exec \
  --platforms=@remote_cc_configure//:platform \
  --remote_executor=remote.example.com \
  ...

If --remote_executor is not specified then repository_ctx.execute()
is executed on the local/host machine (just like before this change).

RELNOTES: None
PiperOrigin-RevId: 281476515
diff --git a/src/main/java/com/google/devtools/build/lib/analysis/platform/PlatformUtils.java b/src/main/java/com/google/devtools/build/lib/analysis/platform/PlatformUtils.java
index 637b607..78dd897 100644
--- a/src/main/java/com/google/devtools/build/lib/analysis/platform/PlatformUtils.java
+++ b/src/main/java/com/google/devtools/build/lib/analysis/platform/PlatformUtils.java
@@ -33,6 +33,30 @@
 /** Utilities for accessing platform properties. */
 public final class PlatformUtils {
 
+  private static void sortPlatformProperties(Platform.Builder builder) {
+    List<Platform.Property> properties =
+        Ordering.from(Comparator.comparing(Platform.Property::getName))
+            .sortedCopy(builder.getPropertiesList());
+    builder.clearProperties();
+    builder.addAllProperties(properties);
+  }
+
+  @Nullable
+  public static Platform buildPlatformProto(Map<String, String> executionProperties) {
+    if (executionProperties.isEmpty()) {
+      return null;
+    }
+    Platform.Builder builder = Platform.newBuilder();
+    for (Map.Entry<String, String> keyValue : executionProperties.entrySet()) {
+      Property property =
+          Property.newBuilder().setName(keyValue.getKey()).setValue(keyValue.getValue()).build();
+      builder.addProperties(property);
+    }
+
+    sortPlatformProperties(builder);
+    return builder.build();
+  }
+
   @Nullable
   public static Platform getPlatformProto(Spawn spawn, @Nullable RemoteOptions remoteOptions)
       throws UserExecException {
@@ -73,12 +97,7 @@
       }
     }
 
-    // Sort the properties.
-    List<Platform.Property> properties =
-        Ordering.from(Comparator.comparing(Platform.Property::getName))
-            .sortedCopy(platformBuilder.getPropertiesList());
-    platformBuilder.clearProperties();
-    platformBuilder.addAllProperties(properties);
+    sortPlatformProperties(platformBuilder);
     return platformBuilder.build();
   }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/bazel/Bazel.java b/src/main/java/com/google/devtools/build/lib/bazel/Bazel.java
index ba6ae1c..745e8b1 100644
--- a/src/main/java/com/google/devtools/build/lib/bazel/Bazel.java
+++ b/src/main/java/com/google/devtools/build/lib/bazel/Bazel.java
@@ -43,6 +43,7 @@
           com.google.devtools.build.lib.runtime.mobileinstall.MobileInstallModule.class,
           com.google.devtools.build.lib.bazel.BazelWorkspaceStatusModule.class,
           com.google.devtools.build.lib.bazel.BazelDiffAwarenessModule.class,
+          com.google.devtools.build.lib.remote.RemoteModule.class,
           com.google.devtools.build.lib.bazel.BazelRepositoryModule.class,
           com.google.devtools.build.lib.bazel.repository.skylark.SkylarkRepositoryDebugModule.class,
           com.google.devtools.build.lib.bazel.debug.WorkspaceRuleModule.class,
@@ -55,7 +56,6 @@
           com.google.devtools.build.lib.outputfilter.OutputFilteringModule.class,
           com.google.devtools.build.lib.ssd.SsdModule.class,
           com.google.devtools.build.lib.worker.WorkerModule.class,
-          com.google.devtools.build.lib.remote.RemoteModule.class,
           com.google.devtools.build.lib.runtime.CacheFileDigestsModule.class,
           com.google.devtools.build.lib.standalone.StandaloneModule.class,
           com.google.devtools.build.lib.sandbox.SandboxModule.class,
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteModule.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteModule.java
index c91a3f7..9f4a11b 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteModule.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteModule.java
@@ -53,6 +53,8 @@
 import com.google.devtools.build.lib.runtime.BuildEventArtifactUploaderFactory;
 import com.google.devtools.build.lib.runtime.Command;
 import com.google.devtools.build.lib.runtime.CommandEnvironment;
+import com.google.devtools.build.lib.runtime.RepositoryRemoteExecutor;
+import com.google.devtools.build.lib.runtime.RepositoryRemoteExecutorFactory;
 import com.google.devtools.build.lib.runtime.ServerBuilder;
 import com.google.devtools.build.lib.skyframe.AspectValue;
 import com.google.devtools.build.lib.util.AbruptExitException;
@@ -90,10 +92,14 @@
   private final BuildEventArtifactUploaderFactoryDelegate
       buildEventArtifactUploaderFactoryDelegate = new BuildEventArtifactUploaderFactoryDelegate();
 
+  private final RepositoryRemoteExecutorFactoryDelegate repositoryRemoteExecutorFactoryDelegate =
+      new RepositoryRemoteExecutorFactoryDelegate();
+
   @Override
   public void serverInit(OptionsParsingResult startupOptions, ServerBuilder builder) {
     builder.addBuildEventArtifactUploaderFactory(
         buildEventArtifactUploaderFactoryDelegate, "remote");
+    builder.setRepositoryRemoteExecutorFactory(repositoryRemoteExecutorFactoryDelegate);
   }
 
   /** Returns whether remote execution should be available. */
@@ -273,6 +279,17 @@
         actionContextProvider =
             RemoteActionContextProvider.createForRemoteExecution(
                 env, remoteCache, remoteExecutor, retryScheduler, digestUtil, logDir);
+        Context repoContext =
+            TracingMetadataUtils.contextWithMetadata(
+                buildRequestId, invocationId, "repository_rule");
+        repositoryRemoteExecutorFactoryDelegate.init(
+            new RemoteRepositoryRemoteExecutorFactory(
+                remoteCache,
+                remoteExecutor,
+                digestUtil,
+                repoContext,
+                remoteOptions.remoteInstanceName,
+                remoteOptions.remoteAcceptCached));
       } else {
         if (enableDiskCache) {
           cacheClient =
@@ -420,6 +437,7 @@
     }
 
     buildEventArtifactUploaderFactoryDelegate.reset();
+    repositoryRemoteExecutorFactoryDelegate.reset();
     actionContextProvider = null;
     actionInputFetcher = null;
     remoteOutputsMode = null;
@@ -500,7 +518,7 @@
 
   @Override
   public Iterable<Class<? extends OptionsBase>> getCommandOptions(Command command) {
-    return "build".equals(command.name())
+    return ImmutableList.of("build", "test", "fetch").contains(command.name())
         ? ImmutableList.of(RemoteOptions.class, AuthAndTLSOptions.class)
         : ImmutableList.of();
   }
@@ -528,4 +546,28 @@
       return uploaderFactory0.create(env);
     }
   }
+
+  private static class RepositoryRemoteExecutorFactoryDelegate
+      implements RepositoryRemoteExecutorFactory {
+
+    private volatile RepositoryRemoteExecutorFactory delegate;
+
+    public void init(RepositoryRemoteExecutorFactory delegate) {
+      Preconditions.checkState(this.delegate == null);
+      this.delegate = delegate;
+    }
+
+    public void reset() {
+      this.delegate = null;
+    }
+
+    @Override
+    public RepositoryRemoteExecutor create() {
+      RepositoryRemoteExecutorFactory delegate = this.delegate;
+      if (delegate == null) {
+        return null;
+      }
+      return delegate.create();
+    }
+  }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteRepositoryRemoteExecutor.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteRepositoryRemoteExecutor.java
new file mode 100644
index 0000000..71fa2f3
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteRepositoryRemoteExecutor.java
@@ -0,0 +1,136 @@
+// Copyright 2019 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.remote;
+
+import build.bazel.remote.execution.v2.Action;
+import build.bazel.remote.execution.v2.ActionResult;
+import build.bazel.remote.execution.v2.Command;
+import build.bazel.remote.execution.v2.Digest;
+import build.bazel.remote.execution.v2.ExecuteRequest;
+import build.bazel.remote.execution.v2.ExecuteResponse;
+import build.bazel.remote.execution.v2.Platform;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.Maps;
+import com.google.devtools.build.lib.analysis.platform.PlatformUtils;
+import com.google.devtools.build.lib.remote.common.RemoteCacheClient.ActionKey;
+import com.google.devtools.build.lib.remote.merkletree.MerkleTree;
+import com.google.devtools.build.lib.remote.util.DigestUtil;
+import com.google.devtools.build.lib.remote.util.Utils;
+import com.google.devtools.build.lib.runtime.RepositoryRemoteExecutor;
+import com.google.protobuf.Message;
+import io.grpc.Context;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Map;
+
+/** The remote package's implementation of {@link RepositoryRemoteExecutor}. */
+public class RemoteRepositoryRemoteExecutor implements RepositoryRemoteExecutor {
+
+  private final RemoteExecutionCache remoteCache;
+  private final GrpcRemoteExecutor remoteExecutor;
+  private final DigestUtil digestUtil;
+  private final Context requestCtx;
+
+  private final String remoteInstanceName;
+  private final boolean acceptCached;
+
+  public RemoteRepositoryRemoteExecutor(
+      RemoteExecutionCache remoteCache,
+      GrpcRemoteExecutor remoteExecutor,
+      DigestUtil digestUtil,
+      Context requestCtx,
+      String remoteInstanceName,
+      boolean acceptCached) {
+    this.remoteCache = remoteCache;
+    this.remoteExecutor = remoteExecutor;
+    this.digestUtil = digestUtil;
+    this.requestCtx = requestCtx;
+    this.remoteInstanceName = remoteInstanceName;
+    this.acceptCached = acceptCached;
+  }
+
+  private ExecutionResult downloadOutErr(ActionResult result)
+      throws IOException, InterruptedException {
+    byte[] stdout = new byte[0];
+    if (!result.getStdoutRaw().isEmpty()) {
+      stdout = result.getStdoutRaw().toByteArray();
+    } else if (result.hasStdoutDigest()) {
+      stdout = Utils.getFromFuture(remoteCache.downloadBlob(result.getStdoutDigest()));
+    }
+
+    byte[] stderr = new byte[0];
+    if (!result.getStderrRaw().isEmpty()) {
+      stderr = result.getStderrRaw().toByteArray();
+    } else if (result.hasStderrDigest()) {
+      stderr = Utils.getFromFuture(remoteCache.downloadBlob(result.getStderrDigest()));
+    }
+
+    return new ExecutionResult(result.getExitCode(), stdout, stderr);
+  }
+
+  @Override
+  public ExecutionResult execute(
+      ImmutableList<String> arguments,
+      ImmutableMap<String, String> executionProperties,
+      ImmutableMap<String, String> environment,
+      String workingDirectory,
+      Duration timeout)
+      throws IOException, InterruptedException {
+    Context prev = requestCtx.attach();
+    try {
+      Platform platform = PlatformUtils.buildPlatformProto(executionProperties);
+      Command command =
+          RemoteSpawnRunner.buildCommand(
+              /* outputs= */ ImmutableList.of(),
+              arguments,
+              environment,
+              platform,
+              workingDirectory);
+      Digest commandHash = digestUtil.compute(command);
+      MerkleTree merkleTree =
+          MerkleTree.build(
+              ImmutableSortedMap.of(),
+              /* metadataProvider= */ null,
+              /* execRoot= */ null,
+              digestUtil);
+      Action action =
+          RemoteSpawnRunner.buildAction(
+              commandHash, merkleTree.getRootDigest(), timeout, acceptCached);
+      Digest actionDigest = digestUtil.compute(action);
+      ActionKey actionKey = new ActionKey(actionDigest);
+      ActionResult actionResult = remoteCache.downloadActionResult(actionKey);
+      if (actionResult == null) {
+        Map<Digest, Message> additionalInputs = Maps.newHashMapWithExpectedSize(2);
+        additionalInputs.put(actionDigest, action);
+        additionalInputs.put(commandHash, command);
+        remoteCache.ensureInputsPresent(merkleTree, additionalInputs);
+
+        ExecuteRequest executeRequest =
+            ExecuteRequest.newBuilder()
+                .setActionDigest(actionDigest)
+                .setInstanceName(remoteInstanceName)
+                .setSkipCacheLookup(!acceptCached)
+                .build();
+
+        ExecuteResponse response = remoteExecutor.executeRemotely(executeRequest);
+        actionResult = response.getResult();
+      }
+      return downloadOutErr(actionResult);
+    } finally {
+      requestCtx.detach(prev);
+    }
+  }
+}
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteRepositoryRemoteExecutorFactory.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteRepositoryRemoteExecutorFactory.java
new file mode 100644
index 0000000..0ba7f73
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteRepositoryRemoteExecutorFactory.java
@@ -0,0 +1,57 @@
+// Copyright 2019 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.remote;
+
+import com.google.devtools.build.lib.remote.util.DigestUtil;
+import com.google.devtools.build.lib.runtime.RepositoryRemoteExecutor;
+import com.google.devtools.build.lib.runtime.RepositoryRemoteExecutorFactory;
+import io.grpc.Context;
+
+/** Factory for {@link RemoteRepositoryRemoteExecutor}. */
+class RemoteRepositoryRemoteExecutorFactory implements RepositoryRemoteExecutorFactory {
+
+  private final RemoteExecutionCache remoteExecutionCache;
+  private final GrpcRemoteExecutor remoteExecutor;
+  private final DigestUtil digestUtil;
+  private final Context requestCtx;
+
+  private final String remoteInstanceName;
+  private final boolean acceptCached;
+
+  RemoteRepositoryRemoteExecutorFactory(
+      RemoteExecutionCache remoteExecutionCache,
+      GrpcRemoteExecutor remoteExecutor,
+      DigestUtil digestUtil,
+      Context requestCtx,
+      String remoteInstanceName,
+      boolean acceptCached) {
+    this.remoteExecutionCache = remoteExecutionCache;
+    this.remoteExecutor = remoteExecutor;
+    this.digestUtil = digestUtil;
+    this.requestCtx = requestCtx;
+    this.remoteInstanceName = remoteInstanceName;
+    this.acceptCached = acceptCached;
+  }
+
+  @Override
+  public RepositoryRemoteExecutor create() {
+    return new RemoteRepositoryRemoteExecutor(
+        remoteExecutionCache,
+        remoteExecutor,
+        digestUtil,
+        requestCtx,
+        remoteInstanceName,
+        acceptCached);
+  }
+}
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnCache.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnCache.java
index d32a79f..5c25075 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnCache.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnCache.java
@@ -135,7 +135,11 @@
 
     Command command =
         RemoteSpawnRunner.buildCommand(
-            spawn.getOutputFiles(), spawn.getArguments(), spawn.getEnvironment(), platform);
+            spawn.getOutputFiles(),
+            spawn.getArguments(),
+            spawn.getEnvironment(),
+            platform,
+            /* workingDirectory= */ null);
     RemoteOutputsMode remoteOutputsMode = options.remoteOutputsMode;
     Action action =
         RemoteSpawnRunner.buildAction(
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnRunner.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnRunner.java
index e2e2f35..a70516b 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnRunner.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnRunner.java
@@ -33,6 +33,7 @@
 import build.bazel.remote.execution.v2.Platform;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -211,7 +212,11 @@
 
     Command command =
         buildCommand(
-            spawn.getOutputFiles(), spawn.getArguments(), spawn.getEnvironment(), platform);
+            spawn.getOutputFiles(),
+            spawn.getArguments(),
+            spawn.getEnvironment(),
+            platform,
+            /* workingDirectory= */ null);
     Digest commandHash = digestUtil.compute(command);
     Action action =
         buildAction(
@@ -496,7 +501,8 @@
       Collection<? extends ActionInput> outputs,
       List<String> arguments,
       ImmutableMap<String, String> env,
-      @Nullable Platform platform) {
+      @Nullable Platform platform,
+      @Nullable String workingDirectory) {
     Command.Builder command = Command.newBuilder();
     ArrayList<String> outputFiles = new ArrayList<>();
     ArrayList<String> outputDirectories = new ArrayList<>();
@@ -522,6 +528,10 @@
     for (String var : variables) {
       command.addEnvironmentVariablesBuilder().setName(var).setValue(env.get(var));
     }
+
+    if (!Strings.isNullOrEmpty(workingDirectory)) {
+      command.setWorkingDirectory(workingDirectory);
+    }
     return command.build();
   }
 
diff --git a/src/test/shell/bazel/remote/remote_execution_test.sh b/src/test/shell/bazel/remote/remote_execution_test.sh
index 42e5051..97c1059 100755
--- a/src/test/shell/bazel/remote/remote_execution_test.sh
+++ b/src/test/shell/bazel/remote/remote_execution_test.sh
@@ -1536,6 +1536,121 @@
   rm -rf $cache
 }
 
+function test_repo_remote_exec() {
+  # Test that repository_ctx.execute can execute a command remotely.
+
+  touch BUILD
+
+  cat > test.bzl <<'EOF'
+def _impl(ctx):
+  res = ctx.execute(["/bin/bash", "-c", "echo -n $BAZEL_REMOTE_PLATFORM"])
+  if res.return_code != 0:
+    fail("Return code 0 expected, but was " + res.exit_code)
+
+  entries = res.stdout.split(",")
+  if len(entries) != 2:
+    fail("Two platform kv pairs expected. Got:" + str(entries))
+  if entries[0] != "ISA=x86-64":
+    fail("'ISA' expected in remote platform'")
+  if entries[1] != "OSFamily=Linux":
+    fail("'OSFamily' expected in remote platform'")
+
+  ctx.file("BUILD")
+
+foo_configure = repository_rule(
+  implementation = _impl,
+  remotable = True,
+)
+EOF
+
+  cat > WORKSPACE <<'EOF'
+load("//:test.bzl", "foo_configure")
+
+foo_configure(
+  name = "default_foo",
+  exec_properties = {
+    "OSFamily" : "Linux",
+    "ISA" : "x86-64",
+  }
+)
+EOF
+
+  bazel fetch \
+    --remote_executor=grpc://localhost:${worker_port} \
+    --experimental_repo_remote_exec \
+    @default_foo//:all
+}
+
+function test_repo_remote_exec_path_argument() {
+  # Test that repository_ctx.execute fails with a descriptive error message
+  # if a path argument is provided. The upload of files as part of command
+  # execution is not yet supported.
+
+  touch BUILD
+
+  echo "hello world" > input.txt
+
+  cat > test.bzl <<'EOF'
+def _impl(ctx):
+  ctx.execute(["cat", ctx.path("input.txt")])
+  ctx.file("BUILD")
+
+foo_configure = repository_rule(
+  implementation = _impl,
+  remotable = True,
+)
+EOF
+
+  cat > WORKSPACE <<'EOF'
+load("//:test.bzl", "foo_configure")
+
+foo_configure(
+  name = "default_foo",
+)
+EOF
+
+  bazel fetch \
+    --remote_executor=grpc://localhost:${worker_port} \
+    --experimental_repo_remote_exec \
+    @default_foo//:all  >& $TEST_log && fail "Should fail" || true
+
+  expect_log "/input.txt"
+  expect_log "Paths are not supported for repository rules marked as remotable."
+}
+
+function test_repo_remote_exec_timeout() {
+  # Test that a remote job is killed if it exceeds the timeout.
+
+  touch BUILD
+
+  cat > test.bzl <<'EOF'
+def _impl(ctx):
+  ctx.execute(["/bin/bash","-c",
+    "for i in {1..3}; do echo \"Sleeping $i...\" && sleep 1; done"], timeout=1)
+  ctx.file("BUILD")
+
+foo_configure = repository_rule(
+  implementation = _impl,
+  remotable = True,
+)
+EOF
+
+  cat > WORKSPACE <<'EOF'
+load("//:test.bzl", "foo_configure")
+
+foo_configure(
+  name = "default_foo",
+)
+EOF
+
+  bazel fetch \
+    --remote_executor=grpc://localhost:${worker_port} \
+    --experimental_repo_remote_exec \
+    @default_foo//:all >& $TEST_log && fail "Should fail" || true
+
+  expect_log "exceeded deadline"
+}
+
 # TODO(alpha): Add a test that fails remote execution when remote worker
 # supports sandbox.
 
diff --git a/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/ExecutionServer.java b/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/ExecutionServer.java
index 538a25a..5ad5d33 100644
--- a/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/ExecutionServer.java
+++ b/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/ExecutionServer.java
@@ -29,6 +29,7 @@
 import build.bazel.remote.execution.v2.ExecuteResponse;
 import build.bazel.remote.execution.v2.ExecutionGrpc.ExecutionImplBase;
 import build.bazel.remote.execution.v2.Platform;
+import build.bazel.remote.execution.v2.Platform.Property;
 import build.bazel.remote.execution.v2.RequestMetadata;
 import build.bazel.remote.execution.v2.WaitExecutionRequest;
 import com.google.common.base.Throwables;
@@ -76,6 +77,7 @@
 import java.util.concurrent.TimeUnit;
 import java.util.logging.Level;
 import java.util.logging.Logger;
+import javax.annotation.Nullable;
 
 /** A basic implementation of an {@link ExecutionImplBase} service. */
 final class ExecutionServer extends ExecutionImplBase {
@@ -360,15 +362,15 @@
   }
 
   // Returns true if the OS being run on is Windows (or some close approximation thereof).
-  private boolean isWindows() {
+  private static boolean isWindows() {
     return System.getProperty("os.name").startsWith("Windows");
   }
 
-  private boolean wasTimeout(long timeoutMillis, long wallTimeMillis) {
+  private static boolean wasTimeout(long timeoutMillis, long wallTimeMillis) {
     return timeoutMillis > 0 && wallTimeMillis > timeoutMillis;
   }
 
-  private Map<String, String> getEnvironmentVariables(Command command) {
+  private static Map<String, String> getEnvironmentVariables(Command command) {
     HashMap<String, String> result = new HashMap<>();
     for (EnvironmentVariable v : command.getEnvironmentVariablesList()) {
       result.put(v.getName(), v.getValue());
@@ -382,7 +384,7 @@
   // This is used to set "-u UID" flag for commands running inside Docker containers. There are
   // only a small handful of cases where uid is vital (e.g., if strict permissions are set on the
   // output files), so most use cases would work without setting uid.
-  private long getUid() {
+  private static long getUid() {
     com.google.devtools.build.lib.shell.Command cmd =
         new com.google.devtools.build.lib.shell.Command(
             new String[] {"id", "-u"},
@@ -403,7 +405,7 @@
 
   // Checks Action for docker container definition. If no docker container specified, returns
   // null. Otherwise returns docker container name from the parameters.
-  private String dockerContainer(Command cmd) throws StatusException {
+  private static String dockerContainer(Command cmd) throws StatusException {
     String result = null;
     for (Platform.Property property : cmd.getPlatform().getPropertiesList()) {
       if (property.getName().equals(CONTAINER_IMAGE_ENTRY_NAME)) {
@@ -429,6 +431,20 @@
     return result;
   }
 
+  private static String platformAsString(@Nullable Platform platform) {
+    if (platform == null) {
+      return "";
+    }
+
+    String separator = "";
+    StringBuilder value = new StringBuilder();
+    for (Property property : platform.getPropertiesList()) {
+      value.append(separator).append(property.getName()).append("=").append(property.getValue());
+      separator = ",";
+    }
+    return value.toString();
+  }
+
   // Converts the Command proto into the shell Command object.
   // If no docker container is specified, creates a Command straight from the
   // arguments. Otherwise, returns a Command that would run the specified command inside the
@@ -436,6 +452,8 @@
   private com.google.devtools.build.lib.shell.Command getCommand(Command cmd, String pathString)
       throws StatusException {
     Map<String, String> environmentVariables = getEnvironmentVariables(cmd);
+    // This allows Bazel's integration tests to test for the remote platform.
+    environmentVariables.put("BAZEL_REMOTE_PLATFORM", platformAsString(cmd.getPlatform()));
     String container = dockerContainer(cmd);
     if (container != null) {
       // Run command inside a docker container.