From 52deefe54124c3cff0dd708704381aa4ac69c01d Mon Sep 17 00:00:00 2001 From: Googler Date: Wed, 1 Mar 2023 05:08:50 -0800 Subject: [PATCH] Correctly handle templated tree artifacts in the prefetcher. Due to the existence of templated tree artifacts (tree artifacts where each file is produced by a separate action) and nested artifacts (artifacts whose output path is a descendant of another artifact's), the prefetcher must not assume every tree artifact is fetched as a whole. In addition, concurrent actions calling the prefetcher might write to the same directory in the output tree, so they must synchronize when making the directory temporarily writable. Fixes #16333. PiperOrigin-RevId: 513205572 Change-Id: I827c4643643f63c9425e63bdf9177805c5f0f409 --- .../remote/AbstractActionInputPrefetcher.java | 444 +++++++++--------- .../google/devtools/build/lib/remote/BUILD | 2 +- .../remote/ActionInputPrefetcherTestBase.java | 177 +++++-- .../remote/build_without_the_bytes_test.sh | 18 + 4 files changed, 375 insertions(+), 266 deletions(-) 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 027188364cc6e1..78bbfd750abd34 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 @@ -13,6 +13,8 @@ // limitations under the License. package com.google.devtools.build.lib.remote; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; import static com.google.common.util.concurrent.Futures.addCallback; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; @@ -22,6 +24,7 @@ import static com.google.devtools.build.lib.remote.util.RxUtils.toTransferResult; import static com.google.devtools.build.lib.remote.util.Utils.getFromFuture; +import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -42,7 +45,6 @@ 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.BulkTransferException; 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; @@ -54,14 +56,16 @@ import io.reactivex.rxjava3.core.Completable; import io.reactivex.rxjava3.core.Flowable; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; +import java.util.Deque; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; import javax.annotation.Nullable; @@ -83,20 +87,77 @@ public abstract class AbstractActionInputPrefetcher implements ActionInputPrefet private final Set missingActionInputs = Sets.newConcurrentHashSet(); - private static class Context { - private final Set nonWritableDirs = Sets.newConcurrentHashSet(); + // Tracks the number of ongoing prefetcher calls temporarily making an output directory writable. + // Since concurrent calls may write to the same directory, it's not safe to make it non-writable + // until no other ongoing calls are writing to it. + private final ConcurrentHashMap temporarilyWritableDirectories = + new ConcurrentHashMap<>(); - public void addNonWritableDir(Path dir) { - nonWritableDirs.add(dir); + /** Keeps track of output directories written to by a single prefetcher call. */ + private final class DirectoryContext { + private final HashSet dirs = new HashSet<>(); + + /** + * Adds to the set of directories written to by the prefetcher call associated with this + * context. + */ + void add(Path dir) { + if (dirs.add(dir)) { + temporarilyWritableDirectories.compute(dir, (unused, count) -> count != null ? ++count : 1); + } } - public void finalizeContext() throws IOException { - for (Path path : nonWritableDirs) { - path.setWritable(false); + /** + * Signals that the prefetcher call associated with this context has finished. + * + *

The output permissions will be set on any directories written to by this call that are not + * being written to by other concurrent calls. + */ + void close() throws IOException { + AtomicReference caughtException = new AtomicReference<>(); + + for (Path dir : dirs) { + temporarilyWritableDirectories.compute( + dir, + (unused, count) -> { + checkState(count != null); + if (--count == 0) { + try { + dir.chmod(outputPermissions.getPermissionsMode()); + } catch (IOException e) { + // Store caught exceptions, but keep cleaning up the map. + if (caughtException.get() == null) { + caughtException.set(e); + } else { + caughtException.get().addSuppressed(e); + } + } + } + return count > 0 ? count : null; + }); + } + dirs.clear(); + + if (caughtException.get() != null) { + throw caughtException.get(); } } } + /** A symlink in the output tree. */ + @AutoValue + abstract static class Symlink { + + abstract PathFragment getLinkExecPath(); + + abstract PathFragment getTargetExecPath(); + + static Symlink of(PathFragment linkExecPath, PathFragment targetExecPath) { + checkArgument(!linkExecPath.equals(targetExecPath)); + return new AutoValue_AbstractActionInputPrefetcher_Symlink(linkExecPath, targetExecPath); + } + } + /** Priority for the staging task. */ protected enum Priority { /** @@ -196,8 +257,8 @@ protected ListenableFuture prefetchFiles( Iterable inputs, MetadataProvider metadataProvider, Priority priority) { - Map> trees = new HashMap<>(); List files = new ArrayList<>(); + for (ActionInput input : inputs) { // Source artifacts don't need to be fetched. if (input instanceof Artifact && ((Artifact) input).isSourceArtifact()) { @@ -209,225 +270,120 @@ protected ListenableFuture prefetchFiles( continue; } - if (input instanceof TreeFileArtifact) { - TreeFileArtifact treeFile = (TreeFileArtifact) input; - SpecialArtifact treeArtifact = treeFile.getParent(); - trees.computeIfAbsent(treeArtifact, unusedKey -> new ArrayList<>()).add(treeFile); - continue; - } - files.add(input); } - Context context = new Context(); + DirectoryContext dirCtx = new DirectoryContext(); - Flowable treeDownloads = - Flowable.fromIterable(trees.entrySet()) - .flatMapSingle( - entry -> - toTransferResult( - prefetchInputTreeOrSymlink( - context, - metadataProvider, - entry.getKey(), - entry.getValue(), - priority))); - - Flowable fileDownloads = + Flowable transfers = Flowable.fromIterable(files) .flatMapSingle( - input -> - toTransferResult( - prefetchInputFileOrSymlink(context, metadataProvider, input, priority))); + input -> toTransferResult(prefetchFile(dirCtx, metadataProvider, input, priority))); - Flowable transfers = Flowable.merge(treeDownloads, fileDownloads); Completable prefetch = Completable.using( - () -> context, ctx -> mergeBulkTransfer(transfers), Context::finalizeContext) + () -> dirCtx, ctx -> mergeBulkTransfer(transfers), DirectoryContext::close) .onErrorResumeNext(this::onErrorResumeNext); + return toListenableFuture(prefetch); } - private Completable prefetchInputTreeOrSymlink( - Context context, - MetadataProvider provider, - SpecialArtifact tree, - List treeFiles, + private Completable prefetchFile( + DirectoryContext dirCtx, + MetadataProvider metadataProvider, + ActionInput input, Priority priority) throws IOException { + if (input instanceof VirtualActionInput) { + prefetchVirtualActionInput((VirtualActionInput) input); + return Completable.complete(); + } - PathFragment execPath = tree.getExecPath(); + PathFragment execPath = input.getExecPath(); - FileArtifactValue treeMetadata = provider.getMetadata(tree); - // TODO(tjgq): Only download individual files that were requested within the tree. - // This isn't straightforward because multiple tree artifacts may share the same output tree - // when a ctx.actions.symlink is involved. - if (treeMetadata == null || !canDownloadAnyTreeFiles(treeFiles, treeMetadata)) { + FileArtifactValue metadata = metadataProvider.getMetadata(input); + if (metadata == null || !canDownloadFile(execRoot.getRelative(execPath), metadata)) { return Completable.complete(); } - PathFragment prefetchExecPath = treeMetadata.getMaterializationExecPath().orElse(execPath); + @Nullable Symlink symlink = maybeGetSymlink(input, metadata, metadataProvider); - Completable prefetch = - prefetchInputTree( - context, provider, prefetchExecPath, tree, treeFiles, treeMetadata, priority); - - // If prefetching to a different path, plant a symlink into it. - if (!prefetchExecPath.equals(execPath)) { - Completable prefetchAndSymlink = - prefetch.doOnComplete(() -> createSymlink(execPath, prefetchExecPath)); - return downloadCache.executeIfNot(execRoot.getRelative(execPath), prefetchAndSymlink); + if (symlink != null) { + checkState(execPath.startsWith(symlink.getLinkExecPath())); + execPath = + symlink.getTargetExecPath().getRelative(execPath.relativeTo(symlink.getLinkExecPath())); } - return prefetch; - } + @Nullable PathFragment treeRootExecPath = maybeGetTreeRoot(input, metadataProvider); - private boolean canDownloadAnyTreeFiles( - Iterable treeFiles, FileArtifactValue metadata) { - for (TreeFileArtifact treeFile : treeFiles) { - if (canDownloadFile(treeFile.getPath(), metadata)) { - return true; - } + Completable result = + downloadFileNoCheckRx( + dirCtx, + execRoot.getRelative(execPath), + treeRootExecPath != null ? execRoot.getRelative(treeRootExecPath) : null, + input, + metadata, + priority); + + if (symlink != null) { + result = result.andThen(plantSymlink(symlink)); } - return false; - } - private boolean shouldDownloadAnyTreeFiles( - Iterable treeFiles, FileArtifactValue metadata) { - for (TreeFileArtifact treeFile : treeFiles) { - if (shouldDownloadFile(treeFile.getPath(), metadata)) { - return true; - } - } - return false; + return result; } - private Completable prefetchInputTree( - Context context, - MetadataProvider provider, - PathFragment execPath, - SpecialArtifact tree, - List treeFiles, - FileArtifactValue treeMetadata, - Priority priority) { - Path treeRoot = execRoot.getRelative(execPath); - HashMap treeFileTmpPathMap = new HashMap<>(); - - Flowable transfers = - Flowable.fromIterable(treeFiles) - .flatMapSingle( - treeFile -> { - FileArtifactValue metadata = provider.getMetadata(treeFile); - - Path tempPath = tempPathGenerator.generateTempPath(); - treeFileTmpPathMap.put(treeFile, tempPath); - - return toTransferResult( - toCompletable( - () -> - doDownloadFile( - reporter, tempPath, treeFile.getExecPath(), metadata, priority), - directExecutor())); - }); - - AtomicBoolean completed = new AtomicBoolean(); - Completable download = - mergeBulkTransfer(transfers) - .doOnComplete( - () -> { - HashSet dirs = new HashSet<>(); - - // Even though the root directory for a tree artifact is created prior to action - // execution, we might be prefetching to a different directory that doesn't yet - // exist (when FileArtifactValue#getMaterializationExecPath() is present). - // In any case, we need to make it writable to move files into it. - createWritableDirectory(treeRoot); - dirs.add(treeRoot); - - for (Map.Entry entry : treeFileTmpPathMap.entrySet()) { - TreeFileArtifact treeFile = entry.getKey(); - Path tempPath = entry.getValue(); - - Path path = treeRoot.getRelative(treeFile.getParentRelativePath()); - Path dir = treeRoot; - for (String segment : treeFile.getParentRelativePath().segments()) { - dir = dir.getRelative(segment); - if (dir.equals(path)) { - break; - } - if (dirs.add(dir)) { - createWritableDirectory(dir); - } - } - checkState(dir.equals(path)); - finalizeDownload(context, tempPath, path); - } - - for (Path dir : dirs) { - // Change permission of all directories of a tree artifact (files are - // changed inside {@code finalizeDownload}) in order to match the behaviour when - // the tree artifact is generated locally. In that case, permission of all files - // and directories inside a tree artifact is changed within {@code - // checkOutputs()}. - dir.chmod(outputPermissions.getPermissionsMode()); - } - - completed.set(true); - }) - .doOnError( - error -> { - if (BulkTransferException.anyCausedByCacheNotFoundException(error)) { - missingActionInputs.add(tree); - } - }) - .doFinally( - () -> { - if (!completed.get()) { - for (Map.Entry entry : treeFileTmpPathMap.entrySet()) { - deletePartialDownload(entry.getValue()); - } - } - }); - return downloadCache.executeIfNot( - treeRoot, - Completable.defer( - () -> { - if (shouldDownloadAnyTreeFiles(treeFiles, treeMetadata)) { - return download; - } - return Completable.complete(); - })); + /** + * For an input belonging to a tree artifact, returns the prefetch exec path of the tree artifact + * root. Otherwise, returns null. + * + *

Some artifacts (notably, those created by {@code ctx.actions.symlink}) are materialized in + * the output tree as a symlink to another artifact, as indicated by the {@link + * FileArtifactValue#getMaterializationExecPath()} field in their metadata. + */ + @Nullable + private PathFragment maybeGetTreeRoot(ActionInput input, MetadataProvider metadataProvider) + throws IOException { + if (!(input instanceof TreeFileArtifact)) { + return null; + } + SpecialArtifact treeArtifact = ((TreeFileArtifact) input).getParent(); + FileArtifactValue treeMetadata = + checkNotNull( + metadataProvider.getMetadata(treeArtifact), + "input %s belongs to a tree artifact whose metadata is missing", + input); + return treeMetadata.getMaterializationExecPath().orElse(treeArtifact.getExecPath()); } - private Completable prefetchInputFileOrSymlink( - Context context, MetadataProvider metadataProvider, ActionInput input, Priority priority) + /** + * Returns the symlink to be planted in the output tree for artifacts that are prefetched into a + * different location. + * + *

Some artifacts (notably, those created by {@code ctx.actions.symlink}) are materialized in + * the output tree as a symlink to another artifact, as indicated by the {@link + * FileArtifactValue#getMaterializationExecPath()} field in their (or their parent tree + * artifact's) metadata. + */ + @Nullable + private Symlink maybeGetSymlink( + ActionInput input, FileArtifactValue metadata, MetadataProvider metadataProvider) throws IOException { - if (input instanceof VirtualActionInput) { - prefetchVirtualActionInput((VirtualActionInput) input); - return Completable.complete(); + if (input instanceof TreeFileArtifact) { + // Check whether the entire tree artifact should be prefetched into a separate location. + SpecialArtifact treeArtifact = ((TreeFileArtifact) input).getParent(); + FileArtifactValue treeMetadata = + checkNotNull( + metadataProvider.getMetadata(treeArtifact), + "input %s belongs to a tree artifact whose metadata is missing", + input); + return maybeGetSymlink(treeArtifact, treeMetadata, metadataProvider); } - PathFragment execPath = input.getExecPath(); - - FileArtifactValue metadata = metadataProvider.getMetadata(input); - if (metadata == null || !canDownloadFile(execRoot.getRelative(execPath), metadata)) { - return Completable.complete(); + PathFragment materializationExecPath = metadata.getMaterializationExecPath().orElse(execPath); + if (!materializationExecPath.equals(execPath)) { + return Symlink.of(execPath, materializationExecPath); } - - PathFragment prefetchExecPath = metadata.getMaterializationExecPath().orElse(execPath); - - Completable prefetch = - downloadFileNoCheckRx( - context, execRoot.getRelative(prefetchExecPath), input, metadata, priority); - - // If prefetching to a different path, plant a symlink into it. - if (!prefetchExecPath.equals(execPath)) { - Completable prefetchAndSymlink = - prefetch.doOnComplete(() -> createSymlink(execPath, prefetchExecPath)); - return downloadCache.executeIfNot(execRoot.getRelative(execPath), prefetchAndSymlink); - } - - return prefetch; + return null; } /** @@ -437,20 +393,22 @@ private Completable prefetchInputFileOrSymlink( * download finished. */ private Completable downloadFileRx( - Context context, + DirectoryContext dirCtx, Path path, + @Nullable Path treeRoot, @Nullable ActionInput actionInput, FileArtifactValue metadata, Priority priority) { if (!canDownloadFile(path, metadata)) { return Completable.complete(); } - return downloadFileNoCheckRx(context, path, actionInput, metadata, priority); + return downloadFileNoCheckRx(dirCtx, path, treeRoot, actionInput, metadata, priority); } private Completable downloadFileNoCheckRx( - Context context, + DirectoryContext dirCtx, Path path, + @Nullable Path treeRoot, @Nullable ActionInput actionInput, FileArtifactValue metadata, Priority priority) { @@ -480,7 +438,7 @@ private Completable downloadFileNoCheckRx( directExecutor()) .doOnComplete( () -> { - finalizeDownload(context, tempPath, finalPath); + finalizeDownload(dirCtx, treeRoot, tempPath, finalPath); completed.set(true); }), tempPath -> { @@ -526,33 +484,64 @@ protected ListenableFuture downloadFileAsync( @Nullable ActionInput actionInput, FileArtifactValue metadata, Priority priority) { - Context context = new Context(); return toListenableFuture( Completable.using( - () -> context, - ctx -> + DirectoryContext::new, + dirCtx -> downloadFileRx( - context, + dirCtx, execRoot.getFileSystem().getPath(path), + /* treeRoot= */ null, actionInput, metadata, priority), - Context::finalizeContext)); + DirectoryContext::close)); } - private void finalizeDownload(Context context, Path tmpPath, Path path) throws IOException { - Path parentDir = path.getParentDirectory(); - // In case the parent directory of the destination is not writable, temporarily change it to - // writable. b/254844173. - if (parentDir != null && !parentDir.isWritable()) { - context.addNonWritableDir(parentDir); - parentDir.setWritable(true); + private void finalizeDownload( + DirectoryContext dirCtx, @Nullable Path treeRoot, Path tmpPath, Path finalPath) + throws IOException { + Path parentDir = checkNotNull(finalPath.getParentDirectory()); + + if (treeRoot != null) { + checkState(parentDir.startsWith(treeRoot)); + + // Create intermediate tree artifact directories. + // In order to minimize filesystem calls when prefetching multiple files into the same tree, + // find the closest existing ancestor directory and only create its descendants. + Deque dirs = new ArrayDeque<>(); + for (Path dir = parentDir; ; dir = dir.getParentDirectory()) { + dirs.push(dir); + // The very last pushed directory already exists, but we still need to make it writable + // in case we previously prefetched into it and made it nonwritable. + if (dir.equals(treeRoot) || dir.exists()) { + break; + } + } + while (!dirs.isEmpty()) { + Path dir = dirs.pop(); + dirCtx.add(dir); + // Create directory or make existing directory writable. + var unused = dir.createWritableDirectory(); + } + } else { + // If the parent directory is not writable, temporarily make it so. + // This is needed when fetching a non-tree artifact nested inside a tree artifact, or a tree + // artifact inside a fileset (see b/254844173 for the latter). + // TODO(tjgq): Fix the TOCTTOU race between isWritable and setWritable. This requires keeping + // track of the original directory permissions. Note that nested artifacts are relatively rare + // and will eventually be disallowed (see issue #16729). + if (!parentDir.isWritable()) { + dirCtx.add(parentDir); + parentDir.setWritable(true); + } } - // The permission of output file is changed after action execution. We manually change - // the permission here for the downloaded file to keep this behaviour consistent. + // Set output permissions on files (tree subdirectories are handled in stopPrefetching), + // matching the behavior of SkyframeActionExecutor#checkOutputs for artifacts produced by local + // actions. tmpPath.chmod(outputPermissions.getPermissionsMode()); - FileSystemUtils.moveFile(tmpPath, path); + FileSystemUtils.moveFile(tmpPath, finalPath); } private void deletePartialDownload(Path path) { @@ -564,18 +553,19 @@ private void deletePartialDownload(Path path) { } } - private void createWritableDirectory(Path dir) throws IOException { - dir.createDirectory(); - dir.setWritable(true); - } - - private void createSymlink(PathFragment linkPath, PathFragment targetPath) throws IOException { - Path link = execRoot.getRelative(linkPath); - Path target = execRoot.getRelative(targetPath); - // Delete the link path if it already exists. - // This will happen for output directories, which get created before the action runs. - link.delete(); - link.createSymbolicLink(target); + private Completable plantSymlink(Symlink symlink) { + return downloadCache.executeIfNot( + execRoot.getRelative(symlink.getLinkExecPath()), + Completable.defer( + () -> { + Path link = execRoot.getRelative(symlink.getLinkExecPath()); + Path target = execRoot.getRelative(symlink.getTargetExecPath()); + // Delete the link path if it already exists. This is the case for tree artifacts, + // whose root directory is created before the action runs. + link.delete(); + link.createSymbolicLink(target); + return Completable.complete(); + })); } public ImmutableSet downloadedFiles() { 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 46173651621c7e..5b7f68acd4a5f6 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/BUILD +++ b/src/main/java/com/google/devtools/build/lib/remote/BUILD @@ -188,11 +188,11 @@ 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/common:cache_not_found_exception", "//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:auto_value", "//third_party:flogger", "//third_party:guava", "//third_party:jsr305", 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 8636e88d1d3c25..8c097089a78200 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 @@ -13,8 +13,10 @@ // limitations under the License. package com.google.devtools.build.lib.remote; +import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Throwables.throwIfInstanceOf; import static com.google.common.truth.Truth.assertThat; +import static com.google.common.truth.Truth.assertWithMessage; import static com.google.devtools.build.lib.actions.util.ActionsTestUtil.createTreeArtifactWithGeneratingAction; import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.Assert.assertThrows; @@ -48,10 +50,12 @@ import com.google.devtools.build.lib.skyframe.TreeArtifactValue; import com.google.devtools.build.lib.util.Pair; import com.google.devtools.build.lib.vfs.DigestHashFunction; +import com.google.devtools.build.lib.vfs.Dirent; import com.google.devtools.build.lib.vfs.FileSystem; import com.google.devtools.build.lib.vfs.FileSystemUtils; import com.google.devtools.build.lib.vfs.Path; import com.google.devtools.build.lib.vfs.PathFragment; +import com.google.devtools.build.lib.vfs.Symlinks; import com.google.devtools.build.lib.vfs.inmemoryfs.InMemoryFileSystem; import java.io.IOException; import java.util.HashMap; @@ -119,43 +123,66 @@ protected Artifact createRemoteArtifact( protected Pair> createRemoteTreeArtifact( String pathFragment, - Map contentMap, + Map localContentMap, + Map remoteContentMap, @Nullable PathFragment materializationExecPath, Map metadata, Map cas) throws IOException { SpecialArtifact parent = createTreeArtifactWithGeneratingAction(artifactRoot, pathFragment); + parent.getPath().createDirectoryAndParents(); parent.getPath().chmod(0555); + TreeArtifactValue.Builder treeBuilder = TreeArtifactValue.newBuilder(parent); - for (Map.Entry entry : contentMap.entrySet()) { - byte[] contentsBytes = entry.getValue().getBytes(UTF_8); - HashCode hashCode = HASH_FUNCTION.getHashFunction().hashBytes(contentsBytes); + for (Map.Entry entry : localContentMap.entrySet()) { + TreeFileArtifact child = + TreeFileArtifact.createTreeOutput(parent, PathFragment.create(entry.getKey())); + byte[] contents = entry.getValue().getBytes(UTF_8); + HashCode hashCode = HASH_FUNCTION.getHashFunction().hashBytes(contents); + FileArtifactValue childValue = + FileArtifactValue.createForNormalFile( + hashCode.asBytes(), /* proxy= */ null, contents.length); + treeBuilder.putChild(child, childValue); + metadata.put(child, childValue); + cas.put(hashCode, contents); + } + for (Map.Entry entry : remoteContentMap.entrySet()) { TreeFileArtifact child = TreeFileArtifact.createTreeOutput(parent, PathFragment.create(entry.getKey())); + byte[] contents = entry.getValue().getBytes(UTF_8); + HashCode hashCode = HASH_FUNCTION.getHashFunction().hashBytes(contents); RemoteFileArtifactValue childValue = RemoteFileArtifactValue.create( - hashCode.asBytes(), contentsBytes.length, /* locationIndex= */ 1); + hashCode.asBytes(), contents.length, /* locationIndex= */ 1); treeBuilder.putChild(child, childValue); metadata.put(child, childValue); - cas.put(hashCode, contentsBytes); + cas.put(hashCode, contents); } if (materializationExecPath != null) { treeBuilder.setMaterializationExecPath(materializationExecPath); } TreeArtifactValue treeValue = treeBuilder.build(); + metadata.put(parent, treeValue.getMetadata()); + return Pair.of(parent, treeValue.getChildren().asList()); } protected Pair> createRemoteTreeArtifact( String pathFragment, - Map contentMap, + Map localContentMap, + Map remoteContentMap, Map metadata, Map cas) throws IOException { return createRemoteTreeArtifact( - pathFragment, contentMap, /* materializationExecPath= */ null, metadata, cas); + pathFragment, + localContentMap, + remoteContentMap, + /* materializationExecPath= */ null, + metadata, + cas); } protected abstract AbstractActionInputPrefetcher createPrefetcher(Map cas); @@ -215,7 +242,7 @@ public void prefetchFiles_downloadRemoteFiles() throws Exception { } @Test - public void prefetchFiles_downloadRemoteFiles_withmaterializationExecPath() throws Exception { + public void prefetchFiles_downloadRemoteFiles_withMaterializationExecPath() throws Exception { Map metadata = new HashMap<>(); Map cas = new HashMap<>(); PathFragment targetExecPath = artifactRoot.getExecPath().getChild("target"); @@ -238,64 +265,99 @@ public void prefetchFiles_downloadRemoteFiles_withmaterializationExecPath() thro public void prefetchFiles_downloadRemoteTrees() throws Exception { Map metadata = new HashMap<>(); Map cas = new HashMap<>(); - Pair> tree = + Pair> treeAndChildren = createRemoteTreeArtifact( "dir", - ImmutableMap.of("file1", "content1", "nested_dir/file2", "content2"), + /* localContentMap= */ ImmutableMap.of(), + /* remoteContentMap= */ ImmutableMap.of( + "file1", "content1", "nested_dir/file2", "content2"), metadata, cas); - SpecialArtifact parent = tree.getFirst(); - Artifact firstChild = tree.getSecond().get(0); - Artifact secondChild = tree.getSecond().get(1); + SpecialArtifact tree = treeAndChildren.getFirst(); + ImmutableList children = treeAndChildren.getSecond(); + Artifact firstChild = children.get(0); + Artifact secondChild = children.get(1); MetadataProvider metadataProvider = new StaticMetadataProvider(metadata); AbstractActionInputPrefetcher prefetcher = createPrefetcher(cas); - wait(prefetcher.prefetchFiles(tree.getSecond(), metadataProvider)); + wait(prefetcher.prefetchFiles(children, metadataProvider)); - assertReadableNonWritableAndExecutable(parent.getPath()); assertThat(FileSystemUtils.readContent(firstChild.getPath(), UTF_8)).isEqualTo("content1"); - assertReadableNonWritableAndExecutable(firstChild.getPath()); assertThat(FileSystemUtils.readContent(secondChild.getPath(), UTF_8)).isEqualTo("content2"); - assertReadableNonWritableAndExecutable(secondChild.getPath()); - assertThat(prefetcher.downloadedFiles()).containsExactly(parent.getPath()); + + assertTreeReadableNonWritableAndExecutable(tree.getPath()); + + assertThat(prefetcher.downloadedFiles()) + .containsExactly(firstChild.getPath(), secondChild.getPath()); assertThat(prefetcher.downloadsInProgress()).isEmpty(); - assertReadableNonWritableAndExecutable(parent.getPath().getRelative("nested_dir")); } @Test - public void prefetchFiles_downloadRemoteTrees_withmaterializationExecPath() throws Exception { + public void prefetchFiles_downloadRemoteTrees_partial() throws Exception { + Map metadata = new HashMap<>(); + Map cas = new HashMap<>(); + Pair> treeAndChildren = + createRemoteTreeArtifact( + "dir", + /* localContentMap= */ ImmutableMap.of("file1", "content1"), + /* remoteContentMap= */ ImmutableMap.of("file2", "content2"), + metadata, + cas); + SpecialArtifact tree = treeAndChildren.getFirst(); + ImmutableList children = treeAndChildren.getSecond(); + Artifact firstChild = children.get(0); + Artifact secondChild = children.get(1); + + MetadataProvider metadataProvider = new StaticMetadataProvider(metadata); + AbstractActionInputPrefetcher prefetcher = createPrefetcher(cas); + + wait(prefetcher.prefetchFiles(ImmutableList.of(firstChild, secondChild), metadataProvider)); + + assertThat(firstChild.getPath().exists()).isFalse(); + assertThat(FileSystemUtils.readContent(secondChild.getPath(), UTF_8)).isEqualTo("content2"); + assertTreeReadableNonWritableAndExecutable(tree.getPath()); + assertThat(prefetcher.downloadedFiles()).containsExactly(secondChild.getPath()); + } + + @Test + public void prefetchFiles_downloadRemoteTrees_withMaterializationExecPath() throws Exception { Map metadata = new HashMap<>(); Map cas = new HashMap<>(); PathFragment targetExecPath = artifactRoot.getExecPath().getChild("target"); - Pair> tree = + Pair> treeAndChildren = createRemoteTreeArtifact( "dir", - ImmutableMap.of("file1", "content1", "nested_dir/file2", "content2"), + /* localContentMap= */ ImmutableMap.of(), + /* remoteContentMap= */ ImmutableMap.of( + "file1", "content1", "nested_dir/file2", "content2"), targetExecPath, metadata, cas); - SpecialArtifact parent = tree.getFirst(); - Artifact firstChild = tree.getSecond().get(0); - Artifact secondChild = tree.getSecond().get(1); + SpecialArtifact tree = treeAndChildren.getFirst(); + ImmutableList children = treeAndChildren.getSecond(); + Artifact firstChild = children.get(0); + Artifact secondChild = children.get(1); MetadataProvider metadataProvider = new StaticMetadataProvider(metadata); AbstractActionInputPrefetcher prefetcher = createPrefetcher(cas); - wait(prefetcher.prefetchFiles(tree.getSecond(), metadataProvider)); + wait(prefetcher.prefetchFiles(children, metadataProvider)); - assertThat(parent.getPath().isSymbolicLink()).isTrue(); - assertThat(parent.getPath().readSymbolicLink()) + assertThat(tree.getPath().isSymbolicLink()).isTrue(); + assertThat(tree.getPath().readSymbolicLink()) .isEqualTo(execRoot.getRelative(targetExecPath).asFragment()); - assertReadableNonWritableAndExecutable(parent.getPath()); assertThat(FileSystemUtils.readContent(firstChild.getPath(), UTF_8)).isEqualTo("content1"); - assertReadableNonWritableAndExecutable(firstChild.getPath()); assertThat(FileSystemUtils.readContent(secondChild.getPath(), UTF_8)).isEqualTo("content2"); - assertReadableNonWritableAndExecutable(secondChild.getPath()); + + assertTreeReadableNonWritableAndExecutable(execRoot.getRelative(targetExecPath)); + assertThat(prefetcher.downloadedFiles()) - .containsExactly(parent.getPath(), execRoot.getRelative(targetExecPath)); + .containsExactly( + tree.getPath(), + execRoot.getRelative(targetExecPath.getRelative(firstChild.getParentRelativePath())), + execRoot.getRelative(targetExecPath.getRelative(secondChild.getParentRelativePath()))); assertThat(prefetcher.downloadsInProgress()).isEmpty(); - assertReadableNonWritableAndExecutable(parent.getPath().getRelative("nested_dir")); } @Test @@ -315,7 +377,7 @@ public void prefetchFiles_missingFiles_fails() throws Exception { @Test public void prefetchFiles_ignoreNonRemoteFiles() throws Exception { - // Test that files that are not remote are not downloaded + // Test that non-remote files are not downloaded. Path p = execRoot.getRelative(artifactRoot.getExecPath()).getRelative("file1"); FileSystemUtils.writeContent(p, UTF_8, "hello world"); @@ -330,6 +392,35 @@ public void prefetchFiles_ignoreNonRemoteFiles() throws Exception { assertThat(prefetcher.downloadsInProgress()).isEmpty(); } + @Test + public void prefetchFiles_ignoreNonRemoteFiles_tree() throws Exception { + // Test that non-remote tree files are not downloaded, but other files in the tree are. + + Map metadata = new HashMap<>(); + Map cas = new HashMap<>(); + Pair> treeAndChildren = + createRemoteTreeArtifact( + "dir", + ImmutableMap.of("file1", "content1"), + ImmutableMap.of("file2", "content2"), + metadata, + cas); + SpecialArtifact tree = treeAndChildren.getFirst(); + ImmutableList children = treeAndChildren.getSecond(); + Artifact firstChild = children.get(0); + Artifact secondChild = children.get(1); + + MetadataProvider metadataProvider = new StaticMetadataProvider(metadata); + AbstractActionInputPrefetcher prefetcher = createPrefetcher(cas); + + wait(prefetcher.prefetchFiles(children, metadataProvider)); + + assertThat(firstChild.getPath().exists()).isFalse(); + assertThat(FileSystemUtils.readContent(secondChild.getPath(), UTF_8)).isEqualTo("content2"); + assertTreeReadableNonWritableAndExecutable(tree.getPath()); + assertThat(prefetcher.downloadedFiles()).containsExactly(secondChild.getPath()); + } + @Test public void prefetchFiles_multipleThreads_downloadIsCancelled() throws Exception { // Test shared downloads are cancelled if all threads/callers are interrupted @@ -544,8 +635,18 @@ protected static void mockDownload( } private void assertReadableNonWritableAndExecutable(Path path) throws IOException { - assertThat(path.isReadable()).isTrue(); - assertThat(path.isWritable()).isFalse(); - assertThat(path.isExecutable()).isTrue(); + assertWithMessage(path + " should be readable").that(path.isReadable()).isTrue(); + assertWithMessage(path + " should not be writable").that(path.isWritable()).isFalse(); + assertWithMessage(path + " should be executable").that(path.isExecutable()).isTrue(); + } + + private void assertTreeReadableNonWritableAndExecutable(Path path) throws IOException { + checkState(path.isDirectory()); + assertReadableNonWritableAndExecutable(path); + for (Dirent dirent : path.readdir(Symlinks.NOFOLLOW)) { + if (dirent.getType().equals(Dirent.Type.DIRECTORY)) { + assertTreeReadableNonWritableAndExecutable(path.getChild(dirent.getName())); + } + } } } diff --git a/src/test/shell/bazel/remote/build_without_the_bytes_test.sh b/src/test/shell/bazel/remote/build_without_the_bytes_test.sh index 096974263a7d21..21e015547dbff6 100755 --- a/src/test/shell/bazel/remote/build_without_the_bytes_test.sh +++ b/src/test/shell/bazel/remote/build_without_the_bytes_test.sh @@ -122,6 +122,24 @@ function test_cc_tree_remote_cache() { || fail "Failed to build //a:tree_cc with remote cache and minimal downloads" } +function test_cc_tree_prefetching() { + if [[ "$PLATFORM" == "darwin" ]]; then + # TODO(b/37355380): This test is disabled due to RemoteWorker not supporting + # setting SDKROOT and DEVELOPER_DIR appropriately, as is required of + # action executors in order to select the appropriate Xcode toolchain. + return 0 + fi + + setup_cc_tree + + bazel build \ + --remote_executor=grpc://localhost:${worker_port} \ + --modify_execution_info=CppCompile=+no-remote-exec \ + --remote_download_minimal \ + //a:tree_cc >& "$TEST_log" \ + || fail "Failed to build //a:tree_cc with prefetching and minimal downloads" +} + function test_cc_include_scanning_and_minimal_downloads() { cat > BUILD <<'EOF' cc_binary(