From 963640a8d75747c3891911940566cc115eacc340 Mon Sep 17 00:00:00 2001 From: Chi Wang Date: Mon, 20 Feb 2023 03:20:39 -0800 Subject: [PATCH] Cleanup stale state when remote cache evicted Currently, when building without the bytes, if Bazel failed to download blobs from CAS when fetching them as inputs to local actions, Bazel fails the build with message like `... --remote_download_outputs=minimal does not work if your remote cache evicts files during builds.` and this message keep showing up until a manually `bazel clean`. This PR fixes that by cleaning up stale state in skyframe and action cache upon remote cache eviction so that a following build can continue without `bazel shutdown` or `bazel clean`. Fixes #17366. Part of #16660. Closes #17462. PiperOrigin-RevId: 510952745 Change-Id: I4fc59a21195565c68375a19ead76738d2208c4ac --- .../build/lib/actions/ActionCacheChecker.java | 12 +-- .../build/lib/actions/ActionCacheUtils.java | 40 +++++++++ .../remote/AbstractActionInputPrefetcher.java | 87 +++++++++++++------ .../google/devtools/build/lib/remote/BUILD | 3 + .../lib/remote/RemoteActionFileSystem.java | 9 +- .../build/lib/remote/RemoteCache.java | 20 ++++- .../build/lib/remote/RemoteModule.java | 36 ++++++-- .../build/lib/remote/RemoteOutputService.java | 62 +++++++++++++ .../remote/ToplevelArtifactsDownloader.java | 7 +- .../build/lib/runtime/BlazeWorkspace.java | 6 ++ .../remote/ActionInputPrefetcherTestBase.java | 12 +-- .../BuildWithoutTheBytesIntegrationTest.java | 49 ++++++++++- .../remote/RemoteActionFileSystemTest.java | 5 +- .../remote/RemoteActionInputFetcherTest.java | 15 ++++ .../devtools/build/lib/remote/util/BUILD | 1 + .../lib/remote/util/IntegrationTestUtils.java | 52 +++++++---- .../remote/worker/OnDiskBlobStoreCache.java | 14 +++ 17 files changed, 353 insertions(+), 77 deletions(-) create mode 100644 src/main/java/com/google/devtools/build/lib/actions/ActionCacheUtils.java diff --git a/src/main/java/com/google/devtools/build/lib/actions/ActionCacheChecker.java b/src/main/java/com/google/devtools/build/lib/actions/ActionCacheChecker.java index 5d76e5220f73c2..a053d76695ff22 100644 --- a/src/main/java/com/google/devtools/build/lib/actions/ActionCacheChecker.java +++ b/src/main/java/com/google/devtools/build/lib/actions/ActionCacheChecker.java @@ -145,19 +145,11 @@ private ActionCache.Entry getCacheEntry(Action action) { if (!cacheConfig.enabled()) { return null; // ignore existing cache when disabled. } - for (Artifact output : action.getOutputs()) { - ActionCache.Entry entry = actionCache.get(output.getExecPathString()); - if (entry != null) { - return entry; - } - } - return null; + return ActionCacheUtils.getCacheEntry(actionCache, action); } private void removeCacheEntry(Action action) { - for (Artifact output : action.getOutputs()) { - actionCache.remove(output.getExecPathString()); - } + ActionCacheUtils.removeCacheEntry(actionCache, action); } @Nullable diff --git a/src/main/java/com/google/devtools/build/lib/actions/ActionCacheUtils.java b/src/main/java/com/google/devtools/build/lib/actions/ActionCacheUtils.java new file mode 100644 index 00000000000000..49065220a98869 --- /dev/null +++ b/src/main/java/com/google/devtools/build/lib/actions/ActionCacheUtils.java @@ -0,0 +1,40 @@ +// Copyright 2023 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.actions; + +import com.google.devtools.build.lib.actions.cache.ActionCache; +import javax.annotation.Nullable; + +/** Utility functions for {@link ActionCache}. */ +public class ActionCacheUtils { + private ActionCacheUtils() {} + + /** Checks whether one of existing output paths is already used as a key. */ + @Nullable + public static ActionCache.Entry getCacheEntry(ActionCache actionCache, Action action) { + for (Artifact output : action.getOutputs()) { + ActionCache.Entry entry = actionCache.get(output.getExecPathString()); + if (entry != null) { + return entry; + } + } + return null; + } + + public static void removeCacheEntry(ActionCache actionCache, Action action) { + for (Artifact output : action.getOutputs()) { + actionCache.remove(output.getExecPathString()); + } + } +} 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 1fe52c3ed92d41..f1abd105dfa73a 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 @@ -42,6 +42,7 @@ import com.google.devtools.build.lib.events.Event; import com.google.devtools.build.lib.events.ExtendedEventHandler.Postable; import com.google.devtools.build.lib.events.Reporter; +import com.google.devtools.build.lib.remote.common.CacheNotFoundException; import com.google.devtools.build.lib.remote.util.AsyncTaskCache; import com.google.devtools.build.lib.remote.util.RxUtils.TransferResult; import com.google.devtools.build.lib.remote.util.TempPathGenerator; @@ -61,6 +62,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; +import javax.annotation.Nullable; /** * Abstract implementation of {@link ActionInputPrefetcher} which implements the orchestration of @@ -78,6 +80,8 @@ public abstract class AbstractActionInputPrefetcher implements ActionInputPrefet protected final Path execRoot; protected final ImmutableList patternsToDownload; + private final Set missingActionInputs = Sets.newConcurrentHashSet(); + private static class Context { private final Set nonWritableDirs = Sets.newConcurrentHashSet(); @@ -399,7 +403,8 @@ private Completable prefetchInputFileOrSymlink( PathFragment prefetchExecPath = metadata.getMaterializationExecPath().orElse(execPath); Completable prefetch = - downloadFileNoCheckRx(context, execRoot.getRelative(prefetchExecPath), metadata, priority); + downloadFileNoCheckRx( + context, execRoot.getRelative(prefetchExecPath), input, metadata, priority); // If prefetching to a different path, plant a symlink into it. if (!prefetchExecPath.equals(execPath)) { @@ -418,15 +423,23 @@ private Completable prefetchInputFileOrSymlink( * download finished. */ private Completable downloadFileRx( - Context context, Path path, FileArtifactValue metadata, Priority priority) { + Context context, + Path path, + @Nullable ActionInput actionInput, + FileArtifactValue metadata, + Priority priority) { if (!canDownloadFile(path, metadata)) { return Completable.complete(); } - return downloadFileNoCheckRx(context, path, metadata, priority); + return downloadFileNoCheckRx(context, path, actionInput, metadata, priority); } private Completable downloadFileNoCheckRx( - Context context, Path path, FileArtifactValue metadata, Priority priority) { + Context context, + Path path, + @Nullable ActionInput actionInput, + FileArtifactValue metadata, + Priority priority) { if (path.isSymbolicLink()) { try { path = path.getRelative(path.readSymbolicLink()); @@ -440,26 +453,32 @@ private Completable downloadFileNoCheckRx( AtomicBoolean completed = new AtomicBoolean(false); Completable download = Completable.using( - tempPathGenerator::generateTempPath, - tempPath -> - toCompletable( - () -> - doDownloadFile( - tempPath, finalPath.relativeTo(execRoot), metadata, priority), - directExecutor()) - .doOnComplete( - () -> { - finalizeDownload(context, tempPath, finalPath); - completed.set(true); - }), - tempPath -> { - if (!completed.get()) { - deletePartialDownload(tempPath); - } - }, - // Set eager=false here because we want cleanup the download *after* upstream is - // disposed. - /* eager= */ false); + tempPathGenerator::generateTempPath, + tempPath -> + toCompletable( + () -> + doDownloadFile( + tempPath, finalPath.relativeTo(execRoot), metadata, priority), + directExecutor()) + .doOnComplete( + () -> { + finalizeDownload(context, tempPath, finalPath); + completed.set(true); + }), + tempPath -> { + if (!completed.get()) { + deletePartialDownload(tempPath); + } + }, + // Set eager=false here because we want cleanup the download *after* upstream is + // disposed. + /* eager= */ false) + .doOnError( + error -> { + if (error instanceof CacheNotFoundException && actionInput != null) { + missingActionInputs.add(actionInput); + } + }); return downloadCache.executeIfNot( finalPath, @@ -479,19 +498,27 @@ private Completable downloadFileNoCheckRx( *

The file will be written into a temporary file and moved to the final destination after the * download finished. */ - public void downloadFile(Path path, FileArtifactValue metadata) + public void downloadFile(Path path, @Nullable ActionInput actionInput, FileArtifactValue metadata) throws IOException, InterruptedException { - getFromFuture(downloadFileAsync(path.asFragment(), metadata, Priority.CRITICAL)); + getFromFuture(downloadFileAsync(path.asFragment(), actionInput, metadata, Priority.CRITICAL)); } protected ListenableFuture downloadFileAsync( - PathFragment path, FileArtifactValue metadata, Priority priority) { + PathFragment path, + @Nullable ActionInput actionInput, + FileArtifactValue metadata, + Priority priority) { Context context = new Context(); return toListenableFuture( Completable.using( () -> context, ctx -> - downloadFileRx(context, execRoot.getFileSystem().getPath(path), metadata, priority), + downloadFileRx( + context, + execRoot.getFileSystem().getPath(path), + actionInput, + metadata, + priority), Context::finalizeContext)); } @@ -648,4 +675,8 @@ private boolean outputMatchesPattern(Artifact output) { public void flushOutputTree() throws InterruptedException { downloadCache.awaitInProgressTasks(); } + + public ImmutableSet getMissingActionInputs() { + return ImmutableSet.copyOf(missingActionInputs); + } } diff --git a/src/main/java/com/google/devtools/build/lib/remote/BUILD b/src/main/java/com/google/devtools/build/lib/remote/BUILD index 1de1266bbb8e6a..b47ec80d36db3c 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/BUILD +++ b/src/main/java/com/google/devtools/build/lib/remote/BUILD @@ -53,6 +53,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib:runtime/command_line_path_factory", "//src/main/java/com/google/devtools/build/lib/actions", "//src/main/java/com/google/devtools/build/lib/actions:action_input_helper", + "//src/main/java/com/google/devtools/build/lib/actions:action_lookup_data", "//src/main/java/com/google/devtools/build/lib/actions:artifacts", "//src/main/java/com/google/devtools/build/lib/actions:execution_requirements", "//src/main/java/com/google/devtools/build/lib/actions:file_metadata", @@ -185,11 +186,13 @@ java_library( "//src/main/java/com/google/devtools/build/lib/actions:artifacts", "//src/main/java/com/google/devtools/build/lib/actions:file_metadata", "//src/main/java/com/google/devtools/build/lib/events", + "//src/main/java/com/google/devtools/build/lib/remote/common", "//src/main/java/com/google/devtools/build/lib/remote/util", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment", "//third_party:flogger", "//third_party:guava", + "//third_party:jsr305", "//third_party:rxjava3", ], ) diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteActionFileSystem.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteActionFileSystem.java index 311db88ee0f6fd..dfd21d7d79e602 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/RemoteActionFileSystem.java +++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteActionFileSystem.java @@ -24,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.devtools.build.lib.actions.ActionInput; import com.google.devtools.build.lib.actions.ActionInputMap; import com.google.devtools.build.lib.actions.Artifact; import com.google.devtools.build.lib.actions.Artifact.SpecialArtifact; @@ -533,6 +534,12 @@ public RemoteFileArtifactValue getRemoteMetadata() { }; } + @Nullable + protected ActionInput getActionInput(PathFragment path) { + PathFragment execPath = path.relativeTo(execRoot); + return inputArtifactData.getInput(execPath.getPathString()); + } + @Nullable protected RemoteFileArtifactValue getRemoteMetadata(PathFragment path) { if (!isOutput(path)) { @@ -572,7 +579,7 @@ private void downloadFileIfRemote(PathFragment path) throws IOException { FileArtifactValue m = getRemoteMetadata(path); if (m != null) { try { - inputFetcher.downloadFile(delegateFs.getPath(path), m); + inputFetcher.downloadFile(delegateFs.getPath(path), getActionInput(path), m); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException( diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteCache.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteCache.java index 43f9994623069a..f081add8ad9caa 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/RemoteCache.java +++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteCache.java @@ -151,15 +151,21 @@ public ListenableFuture uploadActionResult( */ public ListenableFuture uploadFile( RemoteActionExecutionContext context, Digest digest, Path file) { + return uploadFile(context, digest, file, /* force= */ false); + } + + protected ListenableFuture uploadFile( + RemoteActionExecutionContext context, Digest digest, Path file, boolean force) { if (digest.getSizeBytes() == 0) { return COMPLETED_SUCCESS; } Completable upload = - casUploadCache.executeIfNot( + casUploadCache.execute( digest, RxFutures.toCompletable( - () -> cacheProtocol.uploadFile(context, digest, file), directExecutor())); + () -> cacheProtocol.uploadFile(context, digest, file), directExecutor()), + force); return RxFutures.toListenableFuture(upload); } @@ -176,15 +182,21 @@ public ListenableFuture uploadFile( */ public ListenableFuture uploadBlob( RemoteActionExecutionContext context, Digest digest, ByteString data) { + return uploadBlob(context, digest, data, /* force= */ false); + } + + protected ListenableFuture uploadBlob( + RemoteActionExecutionContext context, Digest digest, ByteString data, boolean force) { if (digest.getSizeBytes() == 0) { return COMPLETED_SUCCESS; } Completable upload = - casUploadCache.executeIfNot( + casUploadCache.execute( digest, RxFutures.toCompletable( - () -> cacheProtocol.uploadBlob(context, digest, data), directExecutor())); + () -> cacheProtocol.uploadBlob(context, digest, data), directExecutor()), + force); return RxFutures.toListenableFuture(upload); } 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 5e425d0e63d2b7..9562643c3d8ae0 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 @@ -34,7 +34,9 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.devtools.build.lib.actions.ActionAnalysisMetadata; +import com.google.devtools.build.lib.actions.ActionInput; import com.google.devtools.build.lib.actions.Artifact; +import com.google.devtools.build.lib.actions.FileArtifactValue; import com.google.devtools.build.lib.analysis.AnalysisResult; import com.google.devtools.build.lib.analysis.BlazeDirectories; import com.google.devtools.build.lib.analysis.ConfiguredTarget; @@ -59,6 +61,7 @@ import com.google.devtools.build.lib.exec.ModuleActionContextRegistry; import com.google.devtools.build.lib.exec.SpawnStrategyRegistry; import com.google.devtools.build.lib.remote.RemoteServerCapabilities.ServerCapabilitiesRequirement; +import com.google.devtools.build.lib.remote.ToplevelArtifactsDownloader.PathToMetadataConverter; import com.google.devtools.build.lib.remote.common.RemoteCacheClient; import com.google.devtools.build.lib.remote.common.RemoteExecutionClient; import com.google.devtools.build.lib.remote.downloader.GrpcRemoteDownloader; @@ -987,7 +990,6 @@ public void executorInit(CommandEnvironment env, BuildRequest request, ExecutorB remoteOptions.useNewExitCodeForLostInputs); env.getEventBus().register(actionInputFetcher); builder.setActionInputPrefetcher(actionInputFetcher); - remoteOutputService.setActionInputFetcher(actionInputFetcher); actionContextProvider.setActionInputFetcher(actionInputFetcher); toplevelArtifactsDownloader = @@ -996,15 +998,35 @@ public void executorInit(CommandEnvironment env, BuildRequest request, ExecutorB remoteOutputsMode.downloadToplevelOutputsOnly(), env.getSkyframeExecutor().getEvaluator(), actionInputFetcher, - (path) -> { - FileSystem fileSystem = path.getFileSystem(); - if (fileSystem instanceof RemoteActionFileSystem) { - return ((RemoteActionFileSystem) path.getFileSystem()) - .getRemoteMetadata(path.asFragment()); + new PathToMetadataConverter() { + @Nullable + @Override + public FileArtifactValue getMetadata(Path path) { + FileSystem fileSystem = path.getFileSystem(); + if (fileSystem instanceof RemoteActionFileSystem) { + return ((RemoteActionFileSystem) path.getFileSystem()) + .getRemoteMetadata(path.asFragment()); + } + return null; + } + + @Nullable + @Override + public ActionInput getActionInput(Path path) { + FileSystem fileSystem = path.getFileSystem(); + if (fileSystem instanceof RemoteActionFileSystem) { + return ((RemoteActionFileSystem) path.getFileSystem()) + .getActionInput(path.asFragment()); + } + return null; } - return null; }); env.getEventBus().register(toplevelArtifactsDownloader); + + remoteOutputService.setActionInputFetcher(actionInputFetcher); + remoteOutputService.setMemoizingEvaluator(env.getSkyframeExecutor().getEvaluator()); + remoteOutputService.setActionCache(env.getBlazeWorkspace().getPersistentActionCache()); + env.getEventBus().register(remoteOutputService); } } diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteOutputService.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteOutputService.java index a293b40408282f..f6c5f138a89dd0 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/RemoteOutputService.java +++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteOutputService.java @@ -18,13 +18,20 @@ import com.google.common.collect.ImmutableCollection; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.eventbus.Subscribe; import com.google.devtools.build.lib.actions.Action; +import com.google.devtools.build.lib.actions.ActionCacheUtils; +import com.google.devtools.build.lib.actions.ActionInput; import com.google.devtools.build.lib.actions.ActionInputMap; +import com.google.devtools.build.lib.actions.ActionLookupData; +import com.google.devtools.build.lib.actions.ActionLookupValue; import com.google.devtools.build.lib.actions.Artifact; import com.google.devtools.build.lib.actions.ArtifactPathResolver; import com.google.devtools.build.lib.actions.FilesetOutputSymlink; +import com.google.devtools.build.lib.actions.cache.ActionCache; import com.google.devtools.build.lib.actions.cache.MetadataHandler; import com.google.devtools.build.lib.actions.cache.MetadataInjector; +import com.google.devtools.build.lib.buildtool.buildevent.ExecutionPhaseCompleteEvent; import com.google.devtools.build.lib.events.EventHandler; import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.vfs.BatchStat; @@ -33,8 +40,10 @@ import com.google.devtools.build.lib.vfs.OutputService; import com.google.devtools.build.lib.vfs.PathFragment; import com.google.devtools.build.lib.vfs.Root; +import com.google.devtools.build.skyframe.MemoizingEvaluator; import com.google.devtools.build.skyframe.SkyFunction.Environment; import java.io.IOException; +import java.util.HashMap; import java.util.Map; import java.util.UUID; import javax.annotation.Nullable; @@ -43,11 +52,21 @@ public class RemoteOutputService implements OutputService { @Nullable private RemoteActionInputFetcher actionInputFetcher; + @Nullable MemoizingEvaluator memoizingEvaluator; + @Nullable ActionCache actionCache; void setActionInputFetcher(RemoteActionInputFetcher actionInputFetcher) { this.actionInputFetcher = Preconditions.checkNotNull(actionInputFetcher, "actionInputFetcher"); } + void setMemoizingEvaluator(MemoizingEvaluator memoizingEvaluator) { + this.memoizingEvaluator = memoizingEvaluator; + } + + void setActionCache(ActionCache actionCache) { + this.actionCache = actionCache; + } + @Override public ActionFileSystemType actionFileSystemType() { return actionInputFetcher != null @@ -107,6 +126,49 @@ public void finalizeBuild(boolean buildSuccessful) { // Intentionally left empty. } + @Subscribe + public void onExecutionPhaseCompleteEvent(ExecutionPhaseCompleteEvent event) { + processMissingInputs(); + } + + private void processMissingInputs() { + if (memoizingEvaluator == null || actionInputFetcher == null) { + return; + } + + var actions = new HashMap(); + + try { + for (ActionInput actionInput : actionInputFetcher.getMissingActionInputs()) { + if (actionInput instanceof Artifact.DerivedArtifact) { + Artifact.DerivedArtifact output = (Artifact.DerivedArtifact) actionInput; + ActionLookupData actionLookupData = output.getGeneratingActionKey(); + var actionLookupValue = + memoizingEvaluator.getExistingValue(actionLookupData.getActionLookupKey()); + if (actionLookupValue instanceof ActionLookupValue) { + Action action = + ((ActionLookupValue) actionLookupValue) + .getAction(actionLookupData.getActionIndex()); + actions.put(actionLookupData, action); + } + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + if (!actions.isEmpty()) { + var actionKeys = actions.keySet(); + memoizingEvaluator.delete(key -> key instanceof ActionLookupData && actionKeys.contains(key)); + + if (actionCache != null) { + for (var action : actions.values()) { + ActionCacheUtils.removeCacheEntry(actionCache, action); + } + } + } + } + @Override public void flushActionFileSystem(FileSystem actionFileSystem) throws IOException { ((RemoteActionFileSystem) actionFileSystem).flush(); diff --git a/src/main/java/com/google/devtools/build/lib/remote/ToplevelArtifactsDownloader.java b/src/main/java/com/google/devtools/build/lib/remote/ToplevelArtifactsDownloader.java index 91574d8f9d5ebd..c266070a881374 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/ToplevelArtifactsDownloader.java +++ b/src/main/java/com/google/devtools/build/lib/remote/ToplevelArtifactsDownloader.java @@ -107,6 +107,9 @@ public ToplevelArtifactsDownloader( public interface PathToMetadataConverter { @Nullable FileArtifactValue getMetadata(Path path); + + @Nullable + ActionInput getActionInput(Path path); } private void downloadTestOutput(Path path) { @@ -118,9 +121,11 @@ private void downloadTestOutput(Path path) { // action didn't get the chance to execute. In this case the metadata is null which is fine // because test outputs are already downloaded (otherwise it cannot hit the action cache). FileArtifactValue metadata = pathToMetadataConverter.getMetadata(path); + ActionInput actionInput = pathToMetadataConverter.getActionInput(path); if (metadata != null) { ListenableFuture future = - actionInputPrefetcher.downloadFileAsync(path.asFragment(), metadata, Priority.LOW); + actionInputPrefetcher.downloadFileAsync( + path.asFragment(), actionInput, metadata, Priority.LOW); addCallback( future, new FutureCallback() { diff --git a/src/main/java/com/google/devtools/build/lib/runtime/BlazeWorkspace.java b/src/main/java/com/google/devtools/build/lib/runtime/BlazeWorkspace.java index 95563cff258630..9573749ba3dd73 100644 --- a/src/main/java/com/google/devtools/build/lib/runtime/BlazeWorkspace.java +++ b/src/main/java/com/google/devtools/build/lib/runtime/BlazeWorkspace.java @@ -274,6 +274,12 @@ public ActionCache getOrLoadPersistentActionCache(Reporter reporter) throws IOEx return actionCache; } + /** Returns reference to the lazily instantiated persistent action cache instance */ + @Nullable + public ActionCache getPersistentActionCache() { + return actionCache; + } + /** * Generates a README file in the output base directory. This README file * contains the name of the workspace directory, so that users can figure out diff --git a/src/test/java/com/google/devtools/build/lib/remote/ActionInputPrefetcherTestBase.java b/src/test/java/com/google/devtools/build/lib/remote/ActionInputPrefetcherTestBase.java index ea7b7442c4982d..32c76aee14cd83 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/ActionInputPrefetcherTestBase.java +++ b/src/test/java/com/google/devtools/build/lib/remote/ActionInputPrefetcherTestBase.java @@ -90,7 +90,7 @@ protected Artifact createRemoteArtifact( String contents, @Nullable PathFragment materializationExecPath, Map metadata, - Map cas) { + @Nullable Map cas) { Path p = artifactRoot.getRoot().getRelative(pathFragment); Artifact a = ActionsTestUtil.createArtifact(artifactRoot, p); byte[] contentsBytes = contents.getBytes(UTF_8); @@ -102,7 +102,9 @@ protected Artifact createRemoteArtifact( /* locationIndex= */ 1, materializationExecPath); metadata.put(a, f); - cas.put(hashCode, contentsBytes); + if (cas != null) { + cas.put(hashCode, contentsBytes); + } return a; } @@ -110,7 +112,7 @@ protected Artifact createRemoteArtifact( String pathFragment, String contents, Map metadata, - Map cas) { + @Nullable Map cas) { return createRemoteArtifact( pathFragment, contents, /* materializationExecPath= */ null, metadata, cas); } @@ -441,7 +443,7 @@ public void downloadFile_downloadRemoteFiles() throws Exception { Artifact a1 = createRemoteArtifact("file1", "hello world", metadata, cas); AbstractActionInputPrefetcher prefetcher = createPrefetcher(cas); - prefetcher.downloadFile(a1.getPath(), metadata.get(a1)); + prefetcher.downloadFile(a1.getPath(), /* actionInput= */ null, metadata.get(a1)); assertThat(FileSystemUtils.readContent(a1.getPath(), UTF_8)).isEqualTo("hello world"); assertThat(a1.getPath().isExecutable()).isTrue(); @@ -470,7 +472,7 @@ public void downloadFile_onInterrupt_deletePartialDownloadedFile() throws Except new Thread( () -> { try { - prefetcher.downloadFile(a1.getPath(), metadata.get(a1)); + prefetcher.downloadFile(a1.getPath(), /* actionInput= */ null, metadata.get(a1)); } catch (IOException ignored) { // Intentionally left empty } catch (InterruptedException e) { diff --git a/src/test/java/com/google/devtools/build/lib/remote/BuildWithoutTheBytesIntegrationTest.java b/src/test/java/com/google/devtools/build/lib/remote/BuildWithoutTheBytesIntegrationTest.java index 957ec4d68f8c5f..c7db69aeb70dd9 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/BuildWithoutTheBytesIntegrationTest.java +++ b/src/test/java/com/google/devtools/build/lib/remote/BuildWithoutTheBytesIntegrationTest.java @@ -452,7 +452,7 @@ public void remoteCacheEvictBlobs_exitWithCode39() throws Exception { assertOutputDoesNotExist("a/foo.out"); // Act: Evict blobs from remote cache and do an incremental build - getFileSystem().getPath(worker.getCasPath().getSafePathString()).deleteTreesBelow(); + worker.restart(); write("a/bar.in", "updated bar"); var error = assertThrows(BuildFailedException.class, () -> buildTarget("//a:bar")); @@ -465,4 +465,51 @@ public void remoteCacheEvictBlobs_exitWithCode39() throws Exception { assertThat(error).hasMessageThat().contains(String.format("%s/%s", hashCode, bytes.length)); assertThat(error.getDetailedExitCode().getExitCode().getNumericExitCode()).isEqualTo(39); } + + @Test + public void remoteCacheEvictBlobs_incrementalBuildCanContinue() throws Exception { + // Arrange: Prepare workspace and populate remote cache + write( + "a/BUILD", + "genrule(", + " name = 'foo',", + " srcs = ['foo.in'],", + " outs = ['foo.out'],", + " cmd = 'cat $(SRCS) > $@',", + ")", + "genrule(", + " name = 'bar',", + " srcs = ['foo.out', 'bar.in'],", + " outs = ['bar.out'],", + " cmd = 'cat $(SRCS) > $@',", + " tags = ['no-remote-exec'],", + ")"); + write("a/foo.in", "foo"); + write("a/bar.in", "bar"); + + // Populate remote cache + buildTarget("//a:bar"); + getOutputPath("a/foo.out").delete(); + getOutputPath("a/bar.out").delete(); + getOutputBase().getRelative("action_cache").deleteTreesBelow(); + restartServer(); + + // Clean build, foo.out isn't downloaded + buildTarget("//a:bar"); + assertOutputDoesNotExist("a/foo.out"); + + // Evict blobs from remote cache + worker.restart(); + + // trigger build error + write("a/bar.in", "updated bar"); + // Build failed because of remote cache eviction + assertThrows(BuildFailedException.class, () -> buildTarget("//a:bar")); + + // Act: Do an incremental build without "clean" or "shutdown" + buildTarget("//a:bar"); + + // Assert: target was successfully built + assertValidOutputFile("a/bar.out", "foo" + lineSeparator() + "updated bar" + lineSeparator()); + } } diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemoteActionFileSystemTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemoteActionFileSystemTest.java index 684b3b0f070eed..9b33e0778a2a3d 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/RemoteActionFileSystemTest.java +++ b/src/test/java/com/google/devtools/build/lib/remote/RemoteActionFileSystemTest.java @@ -15,6 +15,7 @@ import static com.google.common.truth.Truth.assertThat; import static com.google.common.truth.Truth8.assertThat; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -117,7 +118,7 @@ public void testGetInputStream() throws Exception { return Futures.immediateFuture(null); }) .when(inputFetcher) - .downloadFile(eq(remoteArtifact.getPath()), eq(inputs.getMetadata(remoteArtifact))); + .downloadFile(eq(remoteArtifact.getPath()), any(), eq(inputs.getMetadata(remoteArtifact))); // act Path remoteActionFsPath = actionFs.getPath(remoteArtifact.getPath().asFragment()); @@ -132,7 +133,7 @@ public void testGetInputStream() throws Exception { assertThat(actualRemoteContents).isEqualTo("remote contents"); assertThat(actualLocalContents).isEqualTo("local contents"); verify(inputFetcher) - .downloadFile(eq(remoteArtifact.getPath()), eq(inputs.getMetadata(remoteArtifact))); + .downloadFile(eq(remoteArtifact.getPath()), any(), eq(inputs.getMetadata(remoteArtifact))); verifyNoMoreInteractions(inputFetcher); } diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemoteActionInputFetcherTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemoteActionInputFetcherTest.java index fca68db8b6f160..4f9a48bbf1cf85 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/RemoteActionInputFetcherTest.java +++ b/src/test/java/com/google/devtools/build/lib/remote/RemoteActionInputFetcherTest.java @@ -156,6 +156,21 @@ public void prefetchFiles_missingFiles_failsWithSpecificMessage() throws Excepti .contains(String.format("%s/%s", digest.getHash(), digest.getSizeBytes())); } + @Test + public void missingInputs_addedToList() { + Map metadata = new HashMap<>(); + Map cas = new HashMap<>(); + Artifact a = createRemoteArtifact("file", "hello world", metadata, /* cas= */ null); + MetadataProvider metadataProvider = new StaticMetadataProvider(metadata); + AbstractActionInputPrefetcher prefetcher = createPrefetcher(cas); + + assertThrows( + ExecException.class, + () -> wait(prefetcher.prefetchFiles(metadata.keySet(), metadataProvider))); + + assertThat(prefetcher.getMissingActionInputs()).contains(a); + } + private RemoteCache newCache( RemoteOptions options, DigestUtil digestUtil, Map cas) { Map cacheEntries = Maps.newHashMapWithExpectedSize(cas.size()); diff --git a/src/test/java/com/google/devtools/build/lib/remote/util/BUILD b/src/test/java/com/google/devtools/build/lib/remote/util/BUILD index ae61a0edbbfde8..39b1bf934f741e 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/util/BUILD +++ b/src/test/java/com/google/devtools/build/lib/remote/util/BUILD @@ -55,6 +55,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment", "//src/test/java/com/google/devtools/build/lib/testutil:TestUtils", "//third_party:guava", + "//third_party:jsr305", "@bazel_tools//tools/java/runfiles", ], ) diff --git a/src/test/java/com/google/devtools/build/lib/remote/util/IntegrationTestUtils.java b/src/test/java/com/google/devtools/build/lib/remote/util/IntegrationTestUtils.java index e2d2df95f93f8c..ded66dd44f2c62 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/util/IntegrationTestUtils.java +++ b/src/test/java/com/google/devtools/build/lib/remote/util/IntegrationTestUtils.java @@ -16,6 +16,7 @@ import static com.google.devtools.build.lib.testutil.TestUtils.tmpDirFile; import static java.nio.charset.StandardCharsets.UTF_8; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.devtools.build.lib.shell.Subprocess; import com.google.devtools.build.lib.shell.SubprocessBuilder; @@ -34,6 +35,7 @@ import java.nio.file.Paths; import java.util.Comparator; import java.util.Random; +import javax.annotation.Nullable; /** Integration test utilities. */ public final class IntegrationTestUtils { @@ -113,20 +115,9 @@ public static WorkerInstance startWorker(boolean useHttp) PathFragment workPath = testTmpDir.getRelative("remote.work_path"); PathFragment casPath = testTmpDir.getRelative("remote.cas_path"); int workerPort = pickUnusedRandomPort(); - ensureMkdir(workPath); - ensureMkdir(casPath); - String workerPath = Runfiles.create().rlocation(WORKER_PATH.getSafePathString()); - Subprocess workerProcess = - new SubprocessBuilder() - .setArgv( - ImmutableList.of( - workerPath, - "--work_path=" + workPath.getSafePathString(), - "--cas_path=" + casPath.getSafePathString(), - (useHttp ? "--http_listen_port=" : "--listen_port=") + workerPort)) - .start(); - waitForPortOpen(workerProcess, workerPort); - return new WorkerInstance(workerProcess, workerPort, workPath, casPath); + var worker = new WorkerInstance(useHttp, workerPort, workPath, casPath); + worker.start(); + return worker; } private static void ensureMkdir(PathFragment path) throws IOException { @@ -140,25 +131,50 @@ private static void ensureMkdir(PathFragment path) throws IOException { } public static class WorkerInstance { - private final Subprocess process; + @Nullable private Subprocess process; + private final boolean useHttp; private final int port; private final PathFragment workPath; private final PathFragment casPath; - private WorkerInstance( - Subprocess process, int port, PathFragment workPath, PathFragment casPath) { - this.process = process; + private WorkerInstance(boolean useHttp, int port, PathFragment workPath, PathFragment casPath) { + this.useHttp = useHttp; this.port = port; this.workPath = workPath; this.casPath = casPath; } + private void start() throws IOException, InterruptedException { + Preconditions.checkState(process == null); + ensureMkdir(workPath); + ensureMkdir(casPath); + String workerPath = Runfiles.create().rlocation(WORKER_PATH.getSafePathString()); + process = + new SubprocessBuilder() + .setArgv( + ImmutableList.of( + workerPath, + "--work_path=" + workPath.getSafePathString(), + "--cas_path=" + casPath.getSafePathString(), + (useHttp ? "--http_listen_port=" : "--listen_port=") + port)) + .start(); + waitForPortOpen(process, port); + } + public void stop() throws IOException { + Preconditions.checkNotNull(process); process.destroyAndWait(); + process = null; + deleteDir(workPath); deleteDir(casPath); } + public void restart() throws IOException, InterruptedException { + stop(); + start(); + } + private static void deleteDir(PathFragment path) throws IOException { try (var stream = Files.walk(Paths.get(path.getSafePathString()))) { stream.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); diff --git a/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/OnDiskBlobStoreCache.java b/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/OnDiskBlobStoreCache.java index 54a6dd94d28667..c67577c54b92b7 100644 --- a/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/OnDiskBlobStoreCache.java +++ b/src/tools/remote/src/main/java/com/google/devtools/build/remote/worker/OnDiskBlobStoreCache.java @@ -23,6 +23,7 @@ import build.bazel.remote.execution.v2.FileNode; import build.bazel.remote.execution.v2.SymlinkAbsolutePathStrategy; import build.bazel.remote.execution.v2.SymlinkNode; +import com.google.common.util.concurrent.ListenableFuture; import com.google.devtools.build.lib.remote.RemoteCache; import com.google.devtools.build.lib.remote.common.RemoteActionExecutionContext; import com.google.devtools.build.lib.remote.disk.DiskCacheClient; @@ -30,6 +31,7 @@ import com.google.devtools.build.lib.remote.util.DigestUtil; import com.google.devtools.build.lib.vfs.Path; import com.google.devtools.build.lib.vfs.PathFragment; +import com.google.protobuf.ByteString; import java.io.IOException; /** A {@link RemoteCache} backed by an {@link DiskCacheClient}. */ @@ -86,4 +88,16 @@ public DigestUtil getDigestUtil() { public RemoteOptions getRemoteOptions() { return options; } + + @Override + public ListenableFuture uploadBlob( + RemoteActionExecutionContext context, Digest digest, ByteString data) { + return uploadBlob(context, digest, data, /* force= */ true); + } + + @Override + public ListenableFuture uploadFile( + RemoteActionExecutionContext context, Digest digest, Path file) { + return uploadFile(context, digest, file, /* force= */ true); + } }