remote: introduce --experimental_remote_download_outputs=minimal

This implements the first milestone of #6862. When specifying
--experimental_remote_download_outputs=minimal Bazel will avoid
downloading most action outputs when used with remote caching
and remote execution. That is, it will only download an action's
stdout/stderr as well as .d and .jdeps output files when building
C++ and Java. All other output files will only be downloaded on
demand, that is if a local action declares a remote output as an
input.

Enabling this mode currently requires specifying three additional
flags to the set of the remote caching/execution flags:
$ bazel build ... \
  --experimental_inmemory_jdeps_files \
  --experimental_inmemory_dotd_files \
  --experimental_remote_download_outputs=minimal

This mode does not yet interact well with the Build Event
Service (--bes_backend=...). Specifically, local file uploads
and remote file name conversions are disabled when
--experimental_remote_download_outputs=minimal is specified.

Lastly, this mode also does not at all interact with Bazel's
on disk action cache. That is, all (local) cached state will
be lost after restarting the Bazel server. Also changing the
value of --experimental_remote_download_outputs between builds
will invalidate all (cached) actions.

Closes #7905.

PiperOrigin-RevId: 241681028
diff --git a/src/main/java/com/google/devtools/build/lib/remote/AbstractRemoteActionCache.java b/src/main/java/com/google/devtools/build/lib/remote/AbstractRemoteActionCache.java
index 3008dfe..da0920b 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/AbstractRemoteActionCache.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/AbstractRemoteActionCache.java
@@ -393,7 +393,7 @@
     if (!metadata.symlinks().isEmpty()) {
       throw new IOException(
           "Symlinks in action outputs are not yet supported by "
-              + "--experimental_remote_fetch_outputs");
+              + "--experimental_remote_download_outputs=minimal");
     }
 
     ActionInput inMemoryOutput = null;
@@ -443,7 +443,7 @@
       if (!directory.symlinks().isEmpty()) {
         throw new IOException(
             "Symlinks in action outputs are not yet supported by "
-                + "--experimental_remote_fetch_outputs");
+                + "--experimental_remote_download_outputs=minimal");
       }
       ImmutableMap.Builder<PathFragment, RemoteFileArtifactValue> childMetadata =
           ImmutableMap.builder();
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteActionContextProvider.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteActionContextProvider.java
index f8e2359..e73a44c 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteActionContextProvider.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteActionContextProvider.java
@@ -160,6 +160,12 @@
     }
   }
 
+  /** Returns the remote cache object if any. */
+  @Nullable
+  AbstractRemoteActionCache getRemoteCache() {
+    return cache;
+  }
+
   @Override
   public void executionPhaseEnding() {
     if (cache != null) {
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteActionInputFetcher.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteActionInputFetcher.java
index 8e6470d..4eb4e2a 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteActionInputFetcher.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteActionInputFetcher.java
@@ -39,7 +39,7 @@
  * Stages output files that are stored remotely to the local filesystem.
  *
  * <p>This is necessary for remote caching/execution when {@code
- * --experimental_remote_fetch_outputs=minimal} is specified.
+ * --experimental_remote_download_outputs=minimal} is specified.
  */
 class RemoteActionInputFetcher implements ActionInputPrefetcher {
 
@@ -126,7 +126,7 @@
                   new IOException(
                       String.format(
                           "Failed to fetch file with hash '%s' because it does not exist remotely."
-                              + " --experimental_remote_fetch_outputs=minimal does not work if"
+                              + " --experimental_remote_download_outputs=minimal does not work if"
                               + " your remote cache evicts files during builds.",
                           ((CacheNotFoundException) e).getMissingDigest().getHash()));
             }
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 16ffb84..4f3732c 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
@@ -31,6 +31,7 @@
 import com.google.devtools.build.lib.exec.ExecutorBuilder;
 import com.google.devtools.build.lib.remote.logging.LoggingInterceptor;
 import com.google.devtools.build.lib.remote.options.RemoteOptions;
+import com.google.devtools.build.lib.remote.options.RemoteOutputsMode;
 import com.google.devtools.build.lib.remote.util.DigestUtil;
 import com.google.devtools.build.lib.remote.util.TracingMetadataUtils;
 import com.google.devtools.build.lib.runtime.BlazeModule;
@@ -59,15 +60,21 @@
 import java.util.List;
 import java.util.concurrent.Executors;
 import java.util.function.Predicate;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 /** RemoteModule provides distributed cache and remote execution for Bazel. */
 public final class RemoteModule extends BlazeModule {
 
+  private static final Logger logger = Logger.getLogger(RemoteModule.class.getName());
+
   private AsynchronousFileOutputStream rpcLogFile;
 
   private final ListeningScheduledExecutorService retryScheduler =
       MoreExecutors.listeningDecorator(Executors.newScheduledThreadPool(1));
+
   private RemoteActionContextProvider actionContextProvider;
+  private RemoteActionInputFetcher actionInputFetcher;
 
   private final BuildEventArtifactUploaderFactoryDelegate
       buildEventArtifactUploaderFactoryDelegate = new BuildEventArtifactUploaderFactoryDelegate();
@@ -117,11 +124,15 @@
 
   @Override
   public void beforeCommand(CommandEnvironment env) throws AbruptExitException {
+    Preconditions.checkState(actionContextProvider == null, "actionContextProvider must be null");
+    Preconditions.checkState(actionInputFetcher == null, "actionInputFetcher must be null");
+
     RemoteOptions remoteOptions = env.getOptions().getOptions(RemoteOptions.class);
     if (remoteOptions == null) {
       // Quit if no supported command is being used. See getCommandOptions for details.
       return;
     }
+
     AuthAndTLSOptions authAndTlsOptions = env.getOptions().getOptions(AuthAndTLSOptions.class);
     DigestHashFunction hashFn = env.getRuntime().getFileSystem().getDigestFunction();
     DigestUtil digestUtil = new DigestUtil(hashFn);
@@ -230,14 +241,24 @@
                 digestUtil,
                 uploader.retain());
         uploader.release();
-        Context requestContext =
-            TracingMetadataUtils.contextWithMetadata(buildRequestId, invocationId, "bes-upload");
-        buildEventArtifactUploaderFactoryDelegate.init(
-            new ByteStreamBuildEventArtifactUploaderFactory(
-                uploader,
-                cacheChannel.authority(),
-                requestContext,
-                remoteOptions.remoteInstanceName));
+        if (remoteOptions.remoteOutputsMode.downloadAllOutputs()) {
+          Context requestContext =
+              TracingMetadataUtils.contextWithMetadata(buildRequestId, invocationId, "bes-upload");
+          buildEventArtifactUploaderFactoryDelegate.init(
+              new ByteStreamBuildEventArtifactUploaderFactory(
+                  uploader,
+                  cacheChannel.authority(),
+                  requestContext,
+                  remoteOptions.remoteInstanceName));
+        } else {
+          // TODO(buchgr): Fix BES local file upload to work with
+          // --experimental_remote_download_outputs
+          env.getReporter()
+              .handle(
+                  Event.warn(
+                      "BES artifact upload is disabled when using "
+                          + "--experimental_remote_download_outputs=minimal"));
+        }
       }
 
       if (enableBlobStoreCache) {
@@ -321,24 +342,84 @@
   }
 
   @Override
-  public void afterCommand() {
+  public void afterCommand() throws AbruptExitException {
+    IOException failure = null;
+
+    try {
+      closeRpcLogFile();
+    } catch (IOException e) {
+      logger.log(Level.WARNING, "Partially wrote rpc log file", e);
+      failure = e;
+    }
+
+    try {
+      deleteDownloadedInputs();
+    } catch (IOException e) {
+      failure = e;
+    }
+
     buildEventArtifactUploaderFactoryDelegate.reset();
     actionContextProvider = null;
-    if (rpcLogFile != null) {
+    actionInputFetcher = null;
+
+    if (failure != null) {
+      throw new AbruptExitException(ExitCode.LOCAL_ENVIRONMENTAL_ERROR, failure);
+    }
+  }
+
+  /**
+   * Delete any input files that have been fetched from the remote cache during the build. This is
+   * so that Bazel's view of the output base is identical with the output base after a build i.e.
+   * files that Bazel thinks exist only remotely actually do.
+   */
+  private void deleteDownloadedInputs() throws IOException {
+    if (actionInputFetcher == null) {
+      return;
+    }
+    IOException deletionFailure = null;
+    for (Path file : actionInputFetcher.downloadedFiles()) {
       try {
-        rpcLogFile.close();
+        file.delete();
       } catch (IOException e) {
-        throw new RuntimeException(e);
-      } finally {
-        rpcLogFile = null;
+        logger.log(
+            Level.SEVERE,
+            String.format("Failed to delete remote output '%s' from the " + "output base.", file),
+            e);
+        deletionFailure = e;
       }
     }
+    if (deletionFailure != null) {
+      throw deletionFailure;
+    }
+  }
+
+  private void closeRpcLogFile() throws IOException {
+    if (rpcLogFile != null) {
+      AsynchronousFileOutputStream oldLogFile = rpcLogFile;
+      rpcLogFile = null;
+      oldLogFile.close();
+    }
   }
 
   @Override
   public void executorInit(CommandEnvironment env, BuildRequest request, ExecutorBuilder builder) {
-    if (actionContextProvider != null) {
-      builder.addActionContextProvider(actionContextProvider);
+    Preconditions.checkState(actionInputFetcher == null, "actionInputFetcher must be null");
+    if (actionContextProvider == null) {
+      return;
+    }
+    builder.addActionContextProvider(actionContextProvider);
+    RemoteOptions remoteOptions =
+        Preconditions.checkNotNull(
+            env.getOptions().getOptions(RemoteOptions.class), "RemoteOptions");
+    RemoteOutputsMode remoteOutputsMode = remoteOptions.remoteOutputsMode;
+    if (!remoteOutputsMode.downloadAllOutputs()) {
+      Context ctx =
+          TracingMetadataUtils.contextWithMetadata(
+              env.getBuildRequestId(), env.getCommandId().toString(), "fetch-remote-inputs");
+      actionInputFetcher =
+          new RemoteActionInputFetcher(
+              actionContextProvider.getRemoteCache(), env.getExecRoot(), ctx);
+      builder.setActionInputPrefetcher(actionInputFetcher);
     }
   }
 
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 2da78bf..a482f1b 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
@@ -14,6 +14,8 @@
 package com.google.devtools.build.lib.remote;
 
 import static com.google.common.base.Strings.isNullOrEmpty;
+import static com.google.devtools.build.lib.remote.util.Utils.createSpawnResult;
+import static com.google.devtools.build.lib.remote.util.Utils.getInMemoryOutputPath;
 
 import build.bazel.remote.execution.v2.Action;
 import build.bazel.remote.execution.v2.ActionResult;
@@ -39,9 +41,11 @@
 import com.google.devtools.build.lib.profiler.SilentCloseable;
 import com.google.devtools.build.lib.remote.merkletree.MerkleTree;
 import com.google.devtools.build.lib.remote.options.RemoteOptions;
+import com.google.devtools.build.lib.remote.options.RemoteOutputsMode;
 import com.google.devtools.build.lib.remote.util.DigestUtil;
 import com.google.devtools.build.lib.remote.util.DigestUtil.ActionKey;
 import com.google.devtools.build.lib.remote.util.TracingMetadataUtils;
+import com.google.devtools.build.lib.remote.util.Utils.InMemoryOutput;
 import com.google.devtools.build.lib.vfs.Path;
 import com.google.devtools.build.lib.vfs.PathFragment;
 import io.grpc.Context;
@@ -114,12 +118,12 @@
     Command command =
         RemoteSpawnRunner.buildCommand(
             spawn.getOutputFiles(), spawn.getArguments(), spawn.getEnvironment(), platform);
+    RemoteOutputsMode remoteOutputsMode = options.remoteOutputsMode;
     Action action =
         RemoteSpawnRunner.buildAction(
             digestUtil.compute(command), merkleTreeRoot, context.getTimeout(), true);
     // Look up action cache, and reuse the action output if it is found.
     ActionKey actionKey = digestUtil.computeActionKey(action);
-
     Context withMetadata =
         TracingMetadataUtils.contextWithMetadata(buildRequestId, commandId, actionKey);
 
@@ -135,16 +139,30 @@
         if (result != null && result.getExitCode() == 0) {
           // In case if failed action returned (exit code != 0) we treat it as a cache miss
           // Otherwise, we know that result exists.
-          try (SilentCloseable c = Profiler.instance().profile("RemoteCache.download")) {
-            remoteCache.download(result, execRoot, context.getFileOutErr());
+          PathFragment inMemoryOutputPath = getInMemoryOutputPath(spawn);
+          InMemoryOutput inMemoryOutput = null;
+          switch (remoteOutputsMode) {
+            case MINIMAL:
+              try (SilentCloseable c = Profiler.instance().profile("RemoteCache.downloadMinimal")) {
+                inMemoryOutput =
+                    remoteCache.downloadMinimal(
+                        result,
+                        spawn.getOutputFiles(),
+                        inMemoryOutputPath,
+                        context.getFileOutErr(),
+                        execRoot,
+                        context.getMetadataInjector());
+              }
+              break;
+            case ALL:
+              try (SilentCloseable c = Profiler.instance().profile("RemoteCache.download")) {
+                remoteCache.download(result, execRoot, context.getFileOutErr());
+              }
+              break;
           }
           SpawnResult spawnResult =
-              new SpawnResult.Builder()
-                  .setStatus(Status.SUCCESS)
-                  .setExitCode(result.getExitCode())
-                  .setCacheHit(true)
-                  .setRunnerName("remote cache hit")
-                  .build();
+              createSpawnResult(
+                  result.getExitCode(), /* cacheHit= */ true, "remote", inMemoryOutput);
           return SpawnCache.success(spawnResult);
         }
       } catch (CacheNotFoundException e) {
@@ -160,6 +178,9 @@
         withMetadata.detach(previous);
       }
     }
+
+    context.prefetchInputs();
+
     if (options.remoteUploadLocalResults) {
       return new CacheHandle() {
         @Override
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 8e56601..e03cd4c 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
@@ -14,7 +14,9 @@
 
 package com.google.devtools.build.lib.remote;
 
+import static com.google.devtools.build.lib.remote.util.Utils.createSpawnResult;
 import static com.google.devtools.build.lib.remote.util.Utils.getFromFuture;
+import static com.google.devtools.build.lib.remote.util.Utils.getInMemoryOutputPath;
 
 import build.bazel.remote.execution.v2.Action;
 import build.bazel.remote.execution.v2.ActionResult;
@@ -53,9 +55,11 @@
 import com.google.devtools.build.lib.profiler.SilentCloseable;
 import com.google.devtools.build.lib.remote.merkletree.MerkleTree;
 import com.google.devtools.build.lib.remote.options.RemoteOptions;
+import com.google.devtools.build.lib.remote.options.RemoteOutputsMode;
 import com.google.devtools.build.lib.remote.util.DigestUtil;
 import com.google.devtools.build.lib.remote.util.DigestUtil.ActionKey;
 import com.google.devtools.build.lib.remote.util.TracingMetadataUtils;
+import com.google.devtools.build.lib.remote.util.Utils.InMemoryOutput;
 import com.google.devtools.build.lib.util.ExitCode;
 import com.google.devtools.build.lib.util.io.FileOutErr;
 import com.google.devtools.build.lib.vfs.Path;
@@ -83,8 +87,8 @@
 
 /** A client for the remote execution service. */
 @ThreadSafe
-class RemoteSpawnRunner implements SpawnRunner {
-  private static final int POSIX_TIMEOUT_EXIT_CODE = /*SIGNAL_BASE=*/128 + /*SIGALRM=*/14;
+public class RemoteSpawnRunner implements SpawnRunner {
+  private static final int POSIX_TIMEOUT_EXIT_CODE = /*SIGNAL_BASE=*/ 128 + /*SIGALRM=*/ 14;
 
   private final Path execRoot;
   private final RemoteOptions remoteOptions;
@@ -147,7 +151,7 @@
     boolean spawnCachable = Spawns.mayBeCached(spawn);
 
     context.report(ProgressStatus.EXECUTING, getName());
-
+    RemoteOutputsMode remoteOutputsMode = remoteOptions.remoteOutputsMode;
     SortedMap<PathFragment, ActionInput> inputMap = context.getInputMapping(true);
     final MerkleTree merkleTree =
         MerkleTree.build(inputMap, context.getMetadataProvider(), execRoot, digestUtil);
@@ -190,9 +194,9 @@
             // Set acceptCachedResult to false in order to force the action re-execution
             acceptCachedResult = false;
           } else {
-            try (SilentCloseable c = Profiler.instance().profile("Remote.downloadRemoteResults")) {
-              remoteCache.download(cachedResult, execRoot, context.getFileOutErr());
-              return createSpawnResult(cachedResult.getExitCode(), /* cacheHit= */ true);
+            try {
+              return downloadAndFinalizeSpawnResult(
+                  cachedResult, /* cacheHit= */ true, spawn, context, remoteOutputsMode);
             } catch (CacheNotFoundException e) {
               // No cache hit, so we fall through to local or remote execution.
               // We set acceptCachedResult to false in order to force the action re-execution.
@@ -257,27 +261,63 @@
                 maybeDownloadServerLogs(reply, actionKey);
               }
 
-              try (SilentCloseable c =
-                  Profiler.instance().profile("Remote.downloadRemoteResults")) {
-                remoteCache.download(actionResult, execRoot, outErr);
+              try {
+                return downloadAndFinalizeSpawnResult(
+                    actionResult, reply.getCachedResult(), spawn, context, remoteOutputsMode);
               } catch (CacheNotFoundException e) {
                 // No cache hit, so if we retry this execution, we must no longer accept
                 // cached results, it must be reexecuted
                 requestBuilder.setSkipCacheLookup(true);
                 throw e;
               }
-              return createSpawnResult(actionResult.getExitCode(), reply.getCachedResult());
             });
       } catch (IOException e) {
         return execLocallyAndUploadOrFail(
             spawn, context, inputMap, actionKey, action, command, uploadLocalResults, e);
       }
-
     } finally {
       withMetadata.detach(previous);
     }
   }
 
+  private SpawnResult downloadAndFinalizeSpawnResult(
+      ActionResult actionResult,
+      boolean cacheHit,
+      Spawn spawn,
+      SpawnExecutionContext context,
+      RemoteOutputsMode remoteOutputsMode)
+      throws ExecException, IOException, InterruptedException {
+    SpawnResult.Status actionStatus =
+        actionResult.getExitCode() == 0 ? Status.SUCCESS : Status.NON_ZERO_EXIT;
+    // In case the action failed, download all outputs. It might be helpful for debugging
+    // and there is no point in injecting output metadata of a failed action.
+    RemoteOutputsMode effectiveOutputsStrategy =
+        actionStatus == Status.SUCCESS ? remoteOutputsMode : RemoteOutputsMode.ALL;
+    PathFragment inMemoryOutputPath = getInMemoryOutputPath(spawn);
+    InMemoryOutput inMemoryOutput = null;
+    switch (effectiveOutputsStrategy) {
+      case MINIMAL:
+        try (SilentCloseable c = Profiler.instance().profile("Remote.downloadMinimal")) {
+          inMemoryOutput =
+              remoteCache.downloadMinimal(
+                  actionResult,
+                  spawn.getOutputFiles(),
+                  inMemoryOutputPath,
+                  context.getFileOutErr(),
+                  execRoot,
+                  context.getMetadataInjector());
+        }
+        break;
+
+      case ALL:
+        try (SilentCloseable c = Profiler.instance().profile("Remote.downloadRemoteResults")) {
+          remoteCache.download(actionResult, execRoot, context.getFileOutErr());
+        }
+        break;
+    }
+    return createSpawnResult(actionResult.getExitCode(), cacheHit, getName(), inMemoryOutput);
+  }
+
   @Override
   public boolean canExec(Spawn spawn) {
     return Spawns.mayBeExecutedRemotely(spawn);
@@ -328,15 +368,6 @@
     }
   }
 
-  private SpawnResult createSpawnResult(int exitCode, boolean cacheHit) {
-    return new SpawnResult.Builder()
-        .setStatus(exitCode == 0 ? Status.SUCCESS : Status.NON_ZERO_EXIT)
-        .setExitCode(exitCode)
-        .setRunnerName(cacheHit ? getName() + " cache hit" : getName())
-        .setCacheHit(cacheHit)
-        .build();
-  }
-
   private SpawnResult execLocally(Spawn spawn, SpawnExecutionContext context)
       throws ExecException, InterruptedException, IOException {
     return fallbackRunner.get().exec(spawn, context);
@@ -403,11 +434,7 @@
         /* forciblyRunRemotely= */ false);
   }
 
-  static Action buildAction(
-      Digest command,
-      Digest inputRoot,
-      Duration timeout,
-      boolean cacheable) {
+  static Action buildAction(Digest command, Digest inputRoot, Duration timeout, boolean cacheable) {
 
     Action.Builder action = Action.newBuilder();
     action.setCommandDigest(command);
@@ -499,7 +526,7 @@
   }
 
   private Map<Path, Long> getInputCtimes(SortedMap<PathFragment, ActionInput> inputMap) {
-    HashMap<Path, Long>  ctimes = new HashMap<>();
+    HashMap<Path, Long> ctimes = new HashMap<>();
     for (Map.Entry<PathFragment, ActionInput> e : inputMap.entrySet()) {
       ActionInput input = e.getValue();
       if (input instanceof VirtualActionInput) {
@@ -577,8 +604,7 @@
    */
   static Collection<Path> resolveActionInputs(
       Path execRoot, Collection<? extends ActionInput> actionInputs) {
-    return actionInputs
-        .stream()
+    return actionInputs.stream()
         .map((inp) -> execRoot.getRelative(inp.getExecPath()))
         .collect(ImmutableList.toImmutableList());
   }
diff --git a/src/main/java/com/google/devtools/build/lib/remote/options/BUILD b/src/main/java/com/google/devtools/build/lib/remote/options/BUILD
index c876d5e..7f9df49 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/options/BUILD
+++ b/src/main/java/com/google/devtools/build/lib/remote/options/BUILD
@@ -8,10 +8,11 @@
 
 java_library(
     name = "options",
-    srcs = ["RemoteOptions.java"],
+    srcs = glob(["*.java"]),
     deps = [
         "//src/main/java/com/google/devtools/build/lib:util",
         "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment",
         "//src/main/java/com/google/devtools/common/options",
+        "//third_party:guava",
     ],
 )
diff --git a/src/main/java/com/google/devtools/build/lib/remote/options/RemoteOptions.java b/src/main/java/com/google/devtools/build/lib/remote/options/RemoteOptions.java
index 21bce91..57079a0 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/options/RemoteOptions.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/options/RemoteOptions.java
@@ -14,8 +14,10 @@
 
 package com.google.devtools.build.lib.remote.options;
 
+import com.google.common.base.Strings;
 import com.google.devtools.build.lib.util.OptionsUtils;
 import com.google.devtools.build.lib.vfs.PathFragment;
+import com.google.devtools.common.options.EnumConverter;
 import com.google.devtools.common.options.Option;
 import com.google.devtools.common.options.OptionDocumentationCategory;
 import com.google.devtools.common.options.OptionEffectTag;
@@ -248,6 +250,26 @@
   public boolean allowSymlinkUpload;
 
   @Option(
+      name = "experimental_remote_download_outputs",
+      defaultValue = "all",
+      category = "remote",
+      documentationCategory = OptionDocumentationCategory.OUTPUT_PARAMETERS,
+      effectTags = {OptionEffectTag.AFFECTS_OUTPUTS},
+      converter = RemoteOutputsStrategyConverter.class,
+      help =
+          "If set to 'minimal' doesn't download any remote build outputs to the local machine, "
+              + "except the ones required by local actions. This option can significantly reduce"
+              + " build times if network bandwidth is a bottleneck.")
+  public RemoteOutputsMode remoteOutputsMode;
+
+  /** Outputs strategy flag parser */
+  public static class RemoteOutputsStrategyConverter extends EnumConverter<RemoteOutputsMode> {
+    public RemoteOutputsStrategyConverter() {
+      super(RemoteOutputsMode.class, "download remote outputs");
+    }
+  }
+
+  @Option(
       name = "remote_result_cache_priority",
       defaultValue = "0",
       documentationCategory = OptionDocumentationCategory.REMOTE,
@@ -295,4 +317,10 @@
 
   /** The maximum size of an outbound message sent via a gRPC channel. */
   public int maxOutboundMessageSize = 1024 * 1024;
+
+  public boolean isRemoteEnabled() {
+    return !Strings.isNullOrEmpty(remoteCache)
+        || !Strings.isNullOrEmpty(remoteExecutor)
+        || !Strings.isNullOrEmpty(remoteHttpCache);
+  }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/remote/options/RemoteOutputsMode.java b/src/main/java/com/google/devtools/build/lib/remote/options/RemoteOutputsMode.java
new file mode 100644
index 0000000..632e67d
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/remote/options/RemoteOutputsMode.java
@@ -0,0 +1,33 @@
+// 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.options;
+
+/** Describes what kind of remote build outputs to download locally. */
+public enum RemoteOutputsMode {
+
+  /** Download all remote outputs locally. */
+  ALL,
+
+  /**
+   * Generally don't download remote action outputs. The only outputs that are being downloaded are:
+   * stdout, stderr and .d and .jdeps files for C++ and Java compilation actions.
+   */
+  MINIMAL;
+
+  /** Returns {@code true} iff action outputs should always be downloaded. */
+  public boolean downloadAllOutputs() {
+    return this == ALL;
+  }
+}