Make HTTP cache works with cache eviction.
Previously, it's possible for an HTTP cache to delete CAS entries referenced by AC without deleting the AC itself because HTTP cache doesn't understand the relationship between AC and CAS. This could result in permanent build errors because Bazel always trust the AC from remote cache assuming all referenced CAS entries exist.
Now, we record the digest of lost blobs before build rewinding, so that during the next build, we can ignore the stale AC and continue with execution.
Fixes #18696.
RELNOTES: Added support for using a remote cache that evicts blobs and doesn't have AC integrity check (e.g. HTTP cache).
PiperOrigin-RevId: 672536163
Change-Id: Ic1271431d28333f6d86e5963542d15a133075157
diff --git a/src/main/java/com/google/devtools/build/lib/remote/AbstractActionInputPrefetcher.java b/src/main/java/com/google/devtools/build/lib/remote/AbstractActionInputPrefetcher.java
index c55f736..fbaed81 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/AbstractActionInputPrefetcher.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/AbstractActionInputPrefetcher.java
@@ -534,8 +534,9 @@
})
.doOnError(
error -> {
- if (error instanceof CacheNotFoundException) {
- reporter.post(new LostInputsEvent());
+ if (error instanceof CacheNotFoundException cacheNotFoundException) {
+ reporter.post(
+ new LostInputsEvent(cacheNotFoundException.getMissingDigest()));
}
}));
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 f52ff94..19d1d63 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
@@ -16,6 +16,7 @@
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
+import build.bazel.remote.execution.v2.Digest;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
import com.google.devtools.build.lib.exec.ExecutionOptions;
@@ -33,6 +34,7 @@
import com.google.devtools.build.lib.util.TempPathGenerator;
import com.google.devtools.build.lib.vfs.OutputService;
import com.google.devtools.build.lib.vfs.Path;
+import java.util.Set;
import java.util.concurrent.Executor;
import javax.annotation.Nullable;
@@ -51,6 +53,7 @@
@Nullable private RemoteActionInputFetcher actionInputFetcher;
@Nullable private final RemoteOutputChecker remoteOutputChecker;
@Nullable private final OutputService outputService;
+ private final Set<Digest> knownMissingCasDigests;
private RemoteActionContextProvider(
Executor executor,
@@ -61,7 +64,8 @@
DigestUtil digestUtil,
@Nullable Path logDir,
@Nullable RemoteOutputChecker remoteOutputChecker,
- @Nullable OutputService outputService) {
+ @Nullable OutputService outputService,
+ Set<Digest> knownMissingCasDigests) {
this.executor = executor;
this.env = Preconditions.checkNotNull(env, "env");
this.remoteCache = remoteCache;
@@ -71,12 +75,14 @@
this.logDir = logDir;
this.remoteOutputChecker = remoteOutputChecker;
this.outputService = outputService;
+ this.knownMissingCasDigests = knownMissingCasDigests;
}
public static RemoteActionContextProvider createForPlaceholder(
CommandEnvironment env,
ListeningScheduledExecutorService retryScheduler,
- DigestUtil digestUtil) {
+ DigestUtil digestUtil,
+ Set<Digest> knownMissingCasDigests) {
return new RemoteActionContextProvider(
directExecutor(),
env,
@@ -86,7 +92,8 @@
digestUtil,
/* logDir= */ null,
/* remoteOutputChecker= */ null,
- /* outputService= */ null);
+ /* outputService= */ null,
+ knownMissingCasDigests);
}
public static RemoteActionContextProvider createForRemoteCaching(
@@ -96,7 +103,8 @@
ListeningScheduledExecutorService retryScheduler,
DigestUtil digestUtil,
@Nullable RemoteOutputChecker remoteOutputChecker,
- OutputService outputService) {
+ OutputService outputService,
+ Set<Digest> knownMissingCasDigests) {
return new RemoteActionContextProvider(
executor,
env,
@@ -106,7 +114,8 @@
digestUtil,
/* logDir= */ null,
remoteOutputChecker,
- checkNotNull(outputService));
+ checkNotNull(outputService),
+ knownMissingCasDigests);
}
public static RemoteActionContextProvider createForRemoteExecution(
@@ -118,7 +127,8 @@
DigestUtil digestUtil,
Path logDir,
@Nullable RemoteOutputChecker remoteOutputChecker,
- OutputService outputService) {
+ OutputService outputService,
+ Set<Digest> knownMissingCasDigests) {
return new RemoteActionContextProvider(
executor,
env,
@@ -128,7 +138,8 @@
digestUtil,
logDir,
remoteOutputChecker,
- checkNotNull(outputService));
+ checkNotNull(outputService),
+ knownMissingCasDigests);
}
private RemotePathResolver createRemotePathResolver() {
@@ -177,7 +188,8 @@
tempPathGenerator,
captureCorruptedOutputsDir,
remoteOutputChecker,
- outputService);
+ outputService,
+ knownMissingCasDigests);
env.getEventBus().register(remoteExecutionService);
}
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionCache.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionCache.java
index c6fd785..b5f7295 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionCache.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionCache.java
@@ -195,7 +195,7 @@
// If we get here, the remote input was determined to exist in the remote or disk cache at
// some point before action execution, but reported to be missing when querying the remote
// for missing action inputs; possibly because it was evicted in the interim.
- reporter.post(new LostInputsEvent());
+ reporter.post(new LostInputsEvent(digest));
throw new CacheNotFoundException(digest, path.getPathString());
}
} catch (IOException e) {
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionService.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionService.java
index 1a2f968..fbd2231 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionService.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionService.java
@@ -74,6 +74,7 @@
import com.google.devtools.build.lib.actions.SpawnResult;
import com.google.devtools.build.lib.actions.Spawns;
import com.google.devtools.build.lib.analysis.platform.PlatformUtils;
+import com.google.devtools.build.lib.buildtool.buildevent.BuildCompleteEvent;
import com.google.devtools.build.lib.buildtool.buildevent.BuildInterruptedEvent;
import com.google.devtools.build.lib.events.Event;
import com.google.devtools.build.lib.events.Reporter;
@@ -88,6 +89,7 @@
import com.google.devtools.build.lib.remote.RemoteExecutionService.ActionResultMetadata.SymlinkMetadata;
import com.google.devtools.build.lib.remote.Scrubber.SpawnScrubber;
import com.google.devtools.build.lib.remote.common.BulkTransferException;
+import com.google.devtools.build.lib.remote.common.LostInputsEvent;
import com.google.devtools.build.lib.remote.common.OperationObserver;
import com.google.devtools.build.lib.remote.common.OutputDigestMismatchException;
import com.google.devtools.build.lib.remote.common.ProgressStatusListener;
@@ -192,6 +194,7 @@
private final OutputService outputService;
@Nullable private final Scrubber scrubber;
+ private final Set<Digest> knownMissingCasDigests;
public RemoteExecutionService(
Executor executor,
@@ -208,7 +211,8 @@
TempPathGenerator tempPathGenerator,
@Nullable Path captureCorruptedOutputsDir,
@Nullable RemoteOutputChecker remoteOutputChecker,
- OutputService outputService) {
+ OutputService outputService,
+ Set<Digest> knownMissingCasDigests) {
this.reporter = reporter;
this.verboseFailures = verboseFailures;
this.execRoot = execRoot;
@@ -234,6 +238,7 @@
this.scheduler = Schedulers.from(executor, /* interruptibleWorker= */ true);
this.remoteOutputChecker = remoteOutputChecker;
this.outputService = outputService;
+ this.knownMissingCasDigests = knownMissingCasDigests;
}
private Command buildCommand(
@@ -626,6 +631,7 @@
private final ActionResult actionResult;
@Nullable private final ExecuteResponse executeResponse;
@Nullable private final String cacheName;
+ @Nullable private ActionResultMetadata metadata;
/** Creates a new {@link RemoteActionResult} instance from a cached result. */
public static RemoteActionResult createFromCache(CachedActionResult cachedActionResult) {
@@ -666,8 +672,20 @@
return actionResult.getOutputDirectoriesList();
}
- public int getOutputDirectoriesCount() {
- return actionResult.getOutputDirectoriesCount();
+ public ActionResultMetadata getOrParseActionResultMetadata(
+ RemoteCache remoteCache,
+ DigestUtil digestUtil,
+ RemoteActionExecutionContext context,
+ RemotePathResolver remotePathResolver)
+ throws IOException, InterruptedException {
+ if (metadata == null) {
+ try (SilentCloseable c = Profiler.instance().profile("Remote.parseActionResultMetadata")) {
+ metadata =
+ parseActionResultMetadata(
+ remoteCache, digestUtil, context, actionResult, remotePathResolver);
+ }
+ }
+ return metadata;
}
public List<OutputSymlink> getOutputDirectorySymlinks() {
@@ -760,7 +778,53 @@
return null;
}
- return RemoteActionResult.createFromCache(cachedActionResult);
+ var result = RemoteActionResult.createFromCache(cachedActionResult);
+
+ // We only add digests to `knownMissingCasDigests` when LostInputsEvent occurs which will cause
+ // the build to abort and rewind, so there is no data race here. This allows us to avoid the
+ // check until cache eviction happens.
+ if (!knownMissingCasDigests.isEmpty()) {
+ var metadata =
+ result.getOrParseActionResultMetadata(
+ remoteCache,
+ digestUtil,
+ action.getRemoteActionExecutionContext(),
+ action.getRemotePathResolver());
+
+ // If we already know digests referenced by this AC is missing from remote cache, ignore it so
+ // that we can fall back to execution. This could happen when the remote cache is an HTTP
+ // cache, or doesn't implement AC integrity check.
+ //
+ // See https://github.com/bazelbuild/bazel/issues/18696.
+ if (updateKnownMissingCasDigests(knownMissingCasDigests, metadata)) {
+ return null;
+ }
+ }
+
+ return result;
+ }
+
+ /**
+ * Removes digests referenced by {@code metadata} from {@code knownMissingCasDigests} and returns
+ * whether any were removed
+ */
+ private static boolean updateKnownMissingCasDigests(
+ Set<Digest> knownMissingCasDigests, ActionResultMetadata metadata) {
+ // Using `remove` below because we assume the missing blob will be uploaded afterwards.
+ var result = false;
+ for (var file : metadata.files()) {
+ if (knownMissingCasDigests.remove(file.digest())) {
+ result = true;
+ }
+ }
+ for (var entry : metadata.directories()) {
+ for (var file : entry.getValue().files()) {
+ if (knownMissingCasDigests.remove(file.digest())) {
+ result = true;
+ }
+ }
+ }
+ return result;
}
private ListenableFuture<FileMetadata> downloadFile(
@@ -969,7 +1033,7 @@
}
}
- private DirectoryMetadata parseDirectory(
+ private static DirectoryMetadata parseDirectory(
Path parent, Directory dir, Map<Digest, Directory> childDirectoriesMap) {
ImmutableList.Builder<FileMetadata> filesBuilder = ImmutableList.builder();
for (FileNode file : dir.getFilesList()) {
@@ -997,16 +1061,18 @@
return new DirectoryMetadata(filesBuilder.build(), symlinksBuilder.build());
}
- ActionResultMetadata parseActionResultMetadata(
+ static ActionResultMetadata parseActionResultMetadata(
+ RemoteCache remoteCache,
+ DigestUtil digestUtil,
RemoteActionExecutionContext context,
- RemoteActionResult result,
+ ActionResult result,
RemotePathResolver remotePathResolver)
throws IOException, InterruptedException {
checkNotNull(remoteCache, "remoteCache can't be null");
Map<Path, ListenableFuture<Tree>> dirMetadataDownloads =
Maps.newHashMapWithExpectedSize(result.getOutputDirectoriesCount());
- for (OutputDirectory dir : result.getOutputDirectories()) {
+ for (OutputDirectory dir : result.getOutputDirectoriesList()) {
var outputPath = dir.getPath();
dirMetadataDownloads.put(
remotePathResolver.outputPathToLocalPath(reencodeExternalToInternal(outputPath)),
@@ -1033,7 +1099,7 @@
}
ImmutableMap.Builder<Path, FileMetadata> files = ImmutableMap.builder();
- for (OutputFile outputFile : result.getOutputFiles()) {
+ for (OutputFile outputFile : result.getOutputFilesList()) {
Path localPath =
remotePathResolver.outputPathToLocalPath(
reencodeExternalToInternal(outputFile.getPath()));
@@ -1045,9 +1111,9 @@
var symlinkMap = new HashMap<Path, SymlinkMetadata>();
var outputSymlinks =
Iterables.concat(
- result.getOutputFileSymlinks(),
- result.getOutputDirectorySymlinks(),
- result.getOutputSymlinks());
+ result.getOutputFileSymlinksList(),
+ result.getOutputDirectorySymlinksList(),
+ result.getOutputSymlinksList());
for (var symlink : outputSymlinks) {
var localPath =
remotePathResolver.outputPathToLocalPath(reencodeExternalToInternal(symlink.getPath()));
@@ -1107,10 +1173,9 @@
context = context.withReadCachePolicy(context.getReadCachePolicy().addRemoteCache());
}
- ActionResultMetadata metadata;
- try (SilentCloseable c = Profiler.instance().profile("Remote.parseActionResultMetadata")) {
- metadata = parseActionResultMetadata(context, result, action.getRemotePathResolver());
- }
+ ActionResultMetadata metadata =
+ result.getOrParseActionResultMetadata(
+ remoteCache, digestUtil, context, action.getRemotePathResolver());
// The expiration time for remote cache entries.
var expireAtEpochMilli = Instant.now().plus(remoteOptions.remoteCacheTtl).toEpochMilli();
@@ -1180,6 +1245,7 @@
if (realToTmpPath.containsKey(file.path)) {
continue;
}
+
if (shouldDownload(result, file.path.relativeTo(execRoot))) {
Path tmpPath = tempPathGenerator.generateTempPath();
realToTmpPath.put(file.path, tmpPath);
@@ -1284,6 +1350,12 @@
}
}
+ if (result.executeResponse != null && !knownMissingCasDigests.isEmpty()) {
+ // A succeeded execution uploads outputs to CAS. Refresh our knowledge about missing
+ // digests.
+ var unused = updateKnownMissingCasDigests(knownMissingCasDigests, metadata);
+ }
+
// When downloading outputs from just remotely executed action, the action result comes from
// Execution response which means, if disk cache is enabled, action result hasn't been
// uploaded to it. Upload action result to disk cache here so next build could hit it.
@@ -1787,6 +1859,20 @@
buildInterrupted.set(true);
}
+ @Subscribe
+ public void onBuildComplete(BuildCompleteEvent event) {
+ if (event.getResult().getSuccess()) {
+ // If build succeeded, clear knownMissingCasDigests in case there are missing digests from
+ // other targets from previous builds which are not relevant anymore.
+ knownMissingCasDigests.clear();
+ }
+ }
+
+ @Subscribe
+ public void onLostInputs(LostInputsEvent event) {
+ knownMissingCasDigests.add(event.getMissingDigest());
+ }
+
/**
* Shuts the service down. Wait for active network I/O to finish but new requests are rejected.
*/
@@ -1821,7 +1907,6 @@
}
void report(Event evt) {
-
synchronized (this) {
if (reportedErrors.contains(evt.getMessage())) {
return;
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 6dda124..c846c3f 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
@@ -16,6 +16,7 @@
import static java.util.concurrent.TimeUnit.SECONDS;
+import build.bazel.remote.execution.v2.Digest;
import build.bazel.remote.execution.v2.DigestFunction;
import com.github.benmanes.caffeine.cache.Cache;
import com.google.auth.Credentials;
@@ -26,6 +27,7 @@
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -112,6 +114,7 @@
import java.nio.channels.ClosedChannelException;
import java.util.List;
import java.util.Optional;
+import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
@@ -126,6 +129,8 @@
private final ListeningScheduledExecutorService retryScheduler =
MoreExecutors.listeningDecorator(Executors.newScheduledThreadPool(1));
+ private final Set<Digest> knownMissingCasDigests = Sets.newConcurrentHashSet();
+
@Nullable private AsynchronousMessageOutputStream<LogEntry> rpcLogFile;
@Nullable private ExecutorService executorService;
@Nullable private RemoteActionContextProvider actionContextProvider;
@@ -255,7 +260,8 @@
/* retryScheduler= */ null,
digestUtil,
remoteOutputChecker,
- outputService);
+ outputService,
+ knownMissingCasDigests);
}
@Override
@@ -278,6 +284,10 @@
Preconditions.checkState(remoteOutputChecker == null, "remoteOutputChecker must be null");
Preconditions.checkState(outputService == null, "remoteOutputService must be null");
+ if ("clean".equals(env.getCommandName())) {
+ knownMissingCasDigests.clear();
+ }
+
RemoteOptions remoteOptions = env.getOptions().getOptions(RemoteOptions.class);
if (remoteOptions == null) {
// Quit if no supported command is being used. See getCommandOptions for details.
@@ -334,7 +344,8 @@
if (!enableDiskCache && !enableHttpCache && !enableGrpcCache && !enableRemoteExecution) {
// Quit if no remote caching or execution was enabled.
actionContextProvider =
- RemoteActionContextProvider.createForPlaceholder(env, retryScheduler, digestUtil);
+ RemoteActionContextProvider.createForPlaceholder(
+ env, retryScheduler, digestUtil, knownMissingCasDigests);
return;
}
@@ -649,7 +660,8 @@
digestUtil,
logDir,
remoteOutputChecker,
- outputService);
+ outputService,
+ knownMissingCasDigests);
repositoryRemoteExecutorFactoryDelegate.init(
new RemoteRepositoryRemoteExecutorFactory(
remoteCache,
@@ -686,7 +698,8 @@
retryScheduler,
digestUtil,
remoteOutputChecker,
- outputService);
+ outputService,
+ knownMissingCasDigests);
}
buildEventArtifactUploaderFactoryDelegate.init(
diff --git a/src/main/java/com/google/devtools/build/lib/remote/common/BUILD b/src/main/java/com/google/devtools/build/lib/remote/common/BUILD
index 6b8390e..45a01ec 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/common/BUILD
+++ b/src/main/java/com/google/devtools/build/lib/remote/common/BUILD
@@ -35,6 +35,7 @@
srcs = ["LostInputsEvent.java"],
deps = [
"//src/main/java/com/google/devtools/build/lib/events",
+ "@remoteapis//:build_bazel_remote_execution_v2_remote_execution_java_proto",
],
)
diff --git a/src/main/java/com/google/devtools/build/lib/remote/common/LostInputsEvent.java b/src/main/java/com/google/devtools/build/lib/remote/common/LostInputsEvent.java
index b770ff2..ded8760 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/common/LostInputsEvent.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/common/LostInputsEvent.java
@@ -13,10 +13,21 @@
// limitations under the License.
package com.google.devtools.build.lib.remote.common;
+import build.bazel.remote.execution.v2.Digest;
import com.google.devtools.build.lib.events.ExtendedEventHandler.Postable;
/**
* An event sent when an input to an action was previously determined to exist remotely, but has
* since been evicted.
*/
-public class LostInputsEvent implements Postable {}
+public class LostInputsEvent implements Postable {
+ private final Digest missingDigest;
+
+ public LostInputsEvent(Digest missingDigest) {
+ this.missingDigest = missingDigest;
+ }
+
+ public Digest getMissingDigest() {
+ return missingDigest;
+ }
+}
diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemoteExecutionServiceTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemoteExecutionServiceTest.java
index b6f94887..3f60417 100644
--- a/src/test/java/com/google/devtools/build/lib/remote/RemoteExecutionServiceTest.java
+++ b/src/test/java/com/google/devtools/build/lib/remote/RemoteExecutionServiceTest.java
@@ -53,6 +53,7 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
import com.google.common.eventbus.EventBus;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.SettableFuture;
@@ -2524,7 +2525,8 @@
tempPathGenerator,
null,
remoteOutputChecker,
- outputService);
+ outputService,
+ Sets.newConcurrentHashSet());
}
private RunfilesTree createRunfilesTree(String root, Collection<Artifact> artifacts) {
diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnCacheTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnCacheTest.java
index e25cfdf..2ef60d6 100644
--- a/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnCacheTest.java
+++ b/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnCacheTest.java
@@ -41,6 +41,7 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedSet;
+import com.google.common.collect.Sets;
import com.google.common.eventbus.EventBus;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.devtools.build.lib.actions.ActionContext;
@@ -299,7 +300,8 @@
tempPathGenerator,
/* captureCorruptedOutputsDir= */ null,
DUMMY_REMOTE_OUTPUT_CHECKER,
- mock(OutputService.class)));
+ mock(OutputService.class),
+ Sets.newConcurrentHashSet()));
return new RemoteSpawnCache(
execRoot, options, /* verboseFailures= */ true, service, digestUtil);
}
diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnRunnerTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnRunnerTest.java
index b7c3920..7ea21d5 100644
--- a/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnRunnerTest.java
+++ b/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnRunnerTest.java
@@ -47,6 +47,7 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
import com.google.common.eventbus.EventBus;
import com.google.common.io.ByteStreams;
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
@@ -1155,7 +1156,8 @@
tempPathGenerator,
/* captureCorruptedOutputsDir= */ null,
remoteOutputChecker,
- mock(OutputService.class));
+ mock(OutputService.class),
+ Sets.newConcurrentHashSet());
RemoteSpawnRunner runner =
new RemoteSpawnRunner(
execRoot,
@@ -1691,7 +1693,8 @@
tempPathGenerator,
/* captureCorruptedOutputsDir= */ null,
remoteOutputChecker,
- mock(OutputService.class)));
+ mock(OutputService.class),
+ Sets.newConcurrentHashSet()));
return new RemoteSpawnRunner(
execRoot,
diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnRunnerWithGrpcRemoteExecutorTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnRunnerWithGrpcRemoteExecutorTest.java
index 9fb8d0c..902b1df 100644
--- a/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnRunnerWithGrpcRemoteExecutorTest.java
+++ b/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnRunnerWithGrpcRemoteExecutorTest.java
@@ -60,6 +60,7 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
import com.google.common.eventbus.EventBus;
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
@@ -350,7 +351,8 @@
tempPathGenerator,
/* captureCorruptedOutputsDir= */ null,
remoteOutputChecker,
- mock(OutputService.class));
+ mock(OutputService.class),
+ Sets.newConcurrentHashSet());
client =
new RemoteSpawnRunner(
execRoot,
diff --git a/src/test/shell/bazel/remote/remote_execution_http_test.sh b/src/test/shell/bazel/remote/remote_execution_http_test.sh
index 5b50b3c..a47b478 100755
--- a/src/test/shell/bazel/remote/remote_execution_http_test.sh
+++ b/src/test/shell/bazel/remote/remote_execution_http_test.sh
@@ -536,4 +536,47 @@
expect_log "WARNING: Credentials are transmitted in plaintext"
}
+function test_remote_http_cache_with_missing_cas_referenced_by_ac() {
+ mkdir -p a
+ cat > a/BUILD <<EOF
+genrule(
+ name = 'bar',
+ outs = ["bar.txt"],
+ cmd = "echo \"bar\" > \$@",
+)
+genrule(
+ name = 'foo',
+ srcs = [":bar"],
+ outs = ["foo.txt"],
+ cmd = "echo \"foo\" > \$@",
+ tags = ["local"],
+)
+EOF
+
+ # Populate the cache
+ bazel build \
+ --remote_cache=http://localhost:${http_port} \
+ //a:foo \
+ || fail "Failed to build //a:foo with remote cache"
+ remote_cas_files="$(count_remote_cas_files)"
+ # bar.txt, stdout and stderr for action 'bar'
+ [[ "$remote_cas_files" == 3 ]] || fail "Expected 3 remote cas entries, not $remote_cas_files"
+
+ # Delete blobs from CAS
+ delete_remote_cas_files
+
+ bazel clean
+ bazel build \
+ --remote_cache=http://localhost:${http_port} \
+ --experimental_remote_cache_eviction_retries=1 \
+ //a:foo \
+ || fail "Failed to build //a:foo with remote cache"
+
+ # Assert that AC is ignored because blobs are missing from CAS, and new output
+ # is uploaded to CAS.
+ expect_not_log 'remote cache hit'
+ remote_cas_files="$(count_remote_cas_files)"
+ [[ "$remote_cas_files" == 3 ]] || fail "Expected 3 remote cas entries, not $remote_cas_files"
+}
+
run_suite "Remote execution and remote cache tests"
diff --git a/src/test/shell/bazel/remote/remote_utils.sh b/src/test/shell/bazel/remote/remote_utils.sh
index 0facf1b..b6ade94 100644
--- a/src/test/shell/bazel/remote/remote_utils.sh
+++ b/src/test/shell/bazel/remote/remote_utils.sh
@@ -105,3 +105,7 @@
function append_remote_cas_files() {
find "$cas_path/cas" -type f >> $1
}
+
+function delete_remote_cas_files() {
+ rm -rf "$cas_path/cas"
+}
\ No newline at end of file
diff --git a/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/OnDiskHttpCacheServerHandler.java b/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/OnDiskHttpCacheServerHandler.java
index b9a24a0..59c6790 100644
--- a/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/OnDiskHttpCacheServerHandler.java
+++ b/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/OnDiskHttpCacheServerHandler.java
@@ -22,6 +22,7 @@
import io.netty.channel.ChannelHandler.Sharable;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
import java.io.IOException;
import javax.annotation.Nullable;
@@ -51,6 +52,8 @@
var in = path.getInputStream()) {
ByteStreams.copy(in, out);
return out.toByteArray();
+ } catch (FileNotFoundException e) {
+ return null;
}
}