remote: introduce downloadMinimal
Adds downloadMinimal() which will only download action outputs that
are strictly required and inject the remaining action metadata instead.
This change also extracts the logic of retrieving complete action result
metadata into its own method so that it can be used by both
downloadMinimal() and download().
downloadMinimal is only introduced in this CL and will be enabled in
a follow up CL.
Progress towards #6862
Closes #7867.
PiperOrigin-RevId: 240987655
diff --git a/src/main/java/com/google/devtools/build/lib/remote/AbstractRemoteActionCache.java b/src/main/java/com/google/devtools/build/lib/remote/AbstractRemoteActionCache.java
index d5d315a..a5f3dad 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/AbstractRemoteActionCache.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/AbstractRemoteActionCache.java
@@ -13,6 +13,8 @@
// limitations under the License.
package com.google.devtools.build.lib.remote;
+import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
+
import build.bazel.remote.execution.v2.Action;
import build.bazel.remote.execution.v2.ActionResult;
import build.bazel.remote.execution.v2.Command;
@@ -27,36 +29,54 @@
import build.bazel.remote.execution.v2.Tree;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.SettableFuture;
+import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.Artifact;
import com.google.devtools.build.lib.actions.EnvironmentalExecException;
import com.google.devtools.build.lib.actions.ExecException;
+import com.google.devtools.build.lib.actions.FileArtifactValue.RemoteFileArtifactValue;
import com.google.devtools.build.lib.actions.UserExecException;
+import com.google.devtools.build.lib.actions.cache.MetadataInjector;
import com.google.devtools.build.lib.concurrent.ThreadSafety;
+import com.google.devtools.build.lib.profiler.Profiler;
+import com.google.devtools.build.lib.profiler.SilentCloseable;
+import com.google.devtools.build.lib.remote.AbstractRemoteActionCache.ActionResultMetadata.DirectoryMetadata;
+import com.google.devtools.build.lib.remote.AbstractRemoteActionCache.ActionResultMetadata.FileMetadata;
+import com.google.devtools.build.lib.remote.AbstractRemoteActionCache.ActionResultMetadata.SymlinkMetadata;
import com.google.devtools.build.lib.remote.options.RemoteOptions;
import com.google.devtools.build.lib.remote.util.DigestUtil;
import com.google.devtools.build.lib.remote.util.Utils;
+import com.google.devtools.build.lib.remote.util.Utils.InMemoryOutput;
import com.google.devtools.build.lib.util.io.FileOutErr;
+import com.google.devtools.build.lib.util.io.OutErr;
import com.google.devtools.build.lib.vfs.Dirent;
import com.google.devtools.build.lib.vfs.FileStatus;
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.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
import javax.annotation.Nullable;
/** A cache for storing artifacts (input and output) as well as the output of running an action. */
@@ -136,7 +156,7 @@
outerF.setException(t);
}
},
- MoreExecutors.directExecutor());
+ directExecutor());
return outerF;
}
@@ -151,46 +171,23 @@
*/
public void download(ActionResult result, Path execRoot, FileOutErr outErr)
throws ExecException, IOException, InterruptedException {
- List<FuturePathBooleanTuple> fileDownloads =
- Collections.synchronizedList(
- new ArrayList<>(result.getOutputFilesCount() + result.getOutputDirectoriesCount()));
- for (OutputFile file : result.getOutputFilesList()) {
- Path path = execRoot.getRelative(file.getPath());
- ListenableFuture<Void> download = downloadFile(path, file.getDigest());
- fileDownloads.add(new FuturePathBooleanTuple(download, path, file.getIsExecutable()));
- }
+ ActionResultMetadata metadata = parseActionResultMetadata(result, execRoot);
- List<ListenableFuture<Void>> dirDownloads = new ArrayList<>(result.getOutputDirectoriesCount());
- for (OutputDirectory dir : result.getOutputDirectoriesList()) {
- SettableFuture<Void> dirDownload = SettableFuture.create();
- ListenableFuture<byte[]> protoDownload = downloadBlob(dir.getTreeDigest());
- Futures.addCallback(
- protoDownload,
- new FutureCallback<byte[]>() {
- @Override
- public void onSuccess(byte[] b) {
- try {
- Tree tree = Tree.parseFrom(b);
- Map<Digest, Directory> childrenMap = new HashMap<>();
- for (Directory child : tree.getChildrenList()) {
- childrenMap.put(digestUtil.compute(child), child);
- }
- Path path = execRoot.getRelative(dir.getPath());
- fileDownloads.addAll(downloadDirectory(path, tree.getRoot(), childrenMap));
- dirDownload.set(null);
- } catch (IOException e) {
- dirDownload.setException(e);
- }
- }
-
- @Override
- public void onFailure(Throwable t) {
- dirDownload.setException(t);
- }
- },
- MoreExecutors.directExecutor());
- dirDownloads.add(dirDownload);
- }
+ List<ListenableFuture<FileMetadata>> downloads =
+ Stream.concat(
+ metadata.files().stream(),
+ metadata.directories().stream()
+ .flatMap((entry) -> entry.getValue().files().stream()))
+ .map(
+ (file) -> {
+ try {
+ ListenableFuture<Void> download = downloadFile(file.path(), file.digest());
+ return Futures.transform(download, (d) -> file, directExecutor());
+ } catch (IOException e) {
+ return Futures.<FileMetadata>immediateFailedFuture(e);
+ }
+ })
+ .collect(Collectors.toList());
// Subsequently we need to wait for *every* download to finish, even if we already know that
// one failed. That's so that when exiting this method we can be sure that all downloads have
@@ -198,33 +195,27 @@
// TODO(buchgr): Look into cancellation.
IOException downloadException = null;
+ InterruptedException interruptedException = null;
try {
- fileDownloads.addAll(downloadOutErr(result, outErr));
+ downloads.addAll(downloadOutErr(result, outErr));
} catch (IOException e) {
downloadException = e;
}
- for (ListenableFuture<Void> dirDownload : dirDownloads) {
- // Block on all directory download futures, so that we can be sure that we have discovered
- // all file downloads and can subsequently safely iterate over the list of file downloads.
- try {
- getFromFuture(dirDownload);
- } catch (IOException e) {
- downloadException = downloadException == null ? e : downloadException;
- }
- }
- for (FuturePathBooleanTuple download : fileDownloads) {
+ for (ListenableFuture<FileMetadata> download : downloads) {
try {
- getFromFuture(download.getFuture());
- if (download.getPath() != null) {
- download.getPath().setExecutable(download.isExecutable());
+ FileMetadata outputFile = getFromFuture(download);
+ if (outputFile != null) {
+ outputFile.path().setExecutable(outputFile.isExecutable());
}
} catch (IOException e) {
downloadException = downloadException == null ? e : downloadException;
+ } catch (InterruptedException e) {
+ interruptedException = interruptedException == null ? e : interruptedException;
}
}
- if (downloadException != null) {
+ if (downloadException != null || interruptedException != null) {
try {
// Delete any (partially) downloaded output files, since any subsequent local execution
// of this action may expect none of the output files to exist.
@@ -252,114 +243,47 @@
e,
true);
}
+ }
+
+ if (interruptedException != null) {
+ throw interruptedException;
+ }
+
+ if (downloadException != null) {
throw downloadException;
}
- // We create the symbolic links after all regular downloads are finished, because dangling
- // links will not work on Windows.
- createSymbolicLinks(
- execRoot,
- Iterables.concat(
- result.getOutputFileSymlinksList(), result.getOutputDirectorySymlinksList()));
+ List<SymlinkMetadata> symlinksInDirectories = new ArrayList<>();
+ for (Entry<Path, DirectoryMetadata> entry : metadata.directories()) {
+ entry.getKey().createDirectoryAndParents();
+ symlinksInDirectories.addAll(entry.getValue().symlinks());
+ }
+
+ Iterable<SymlinkMetadata> symlinks =
+ Iterables.concat(metadata.symlinks(), symlinksInDirectories);
+
+ // Create the symbolic links after all downloads are finished, because dangling symlinks
+ // might not be supported on all platforms
+ createSymlinks(symlinks);
}
- // Creates a local symbolic link. Only relative symlinks are supported.
- private void createSymbolicLink(Path path, String target) throws IOException {
- PathFragment targetPath = PathFragment.create(target);
- if (targetPath.isAbsolute()) {
- // Error, we do not support absolute symlinks as outputs.
- throw new IOException(
- String.format(
- "Action output %s is a symbolic link to an absolute path %s. "
- + "Symlinks to absolute paths in action outputs are not supported.",
- path, target));
- }
- path.createSymbolicLink(targetPath);
- }
-
- // Creates symbolic links locally as created remotely by the action. Only relative symbolic
- // links are supported, because absolute symlinks break action hermeticity.
- private void createSymbolicLinks(Path execRoot, Iterable<OutputSymlink> symlinks)
- throws IOException {
- for (OutputSymlink symlink : symlinks) {
- Path path = execRoot.getRelative(symlink.getPath());
- Preconditions.checkNotNull(
- path.getParentDirectory(), "Failed creating directory and parents for %s", path)
- .createDirectoryAndParents();
- createSymbolicLink(path, symlink.getTarget());
- }
- }
-
- @VisibleForTesting
- protected <T> T getFromFuture(ListenableFuture<T> f) throws IOException, InterruptedException {
- return Utils.getFromFuture(f);
- }
-
- /** Tuple of {@code ListenableFuture, Path, boolean}. */
- private static class FuturePathBooleanTuple {
- private final ListenableFuture<?> future;
- private final Path path;
- private final boolean isExecutable;
-
- public FuturePathBooleanTuple(ListenableFuture<?> future, Path path, boolean isExecutable) {
- this.future = future;
- this.path = path;
- this.isExecutable = isExecutable;
- }
-
- public ListenableFuture<?> getFuture() {
- return future;
- }
-
- public Path getPath() {
- return path;
- }
-
- public boolean isExecutable() {
- return isExecutable;
- }
- }
-
- /**
- * Download a directory recursively. The directory is represented by a {@link Directory} protobuf
- * message, and the descendant directories are in {@code childrenMap}, accessible through their
- * digest.
- */
- private List<FuturePathBooleanTuple> downloadDirectory(
- Path path, Directory dir, Map<Digest, Directory> childrenMap) throws IOException {
- // Ensure that the directory is created here even though the directory might be empty
- path.createDirectoryAndParents();
-
- for (SymlinkNode symlink : dir.getSymlinksList()) {
- createSymbolicLink(path.getRelative(symlink.getName()), symlink.getTarget());
- }
-
- List<FuturePathBooleanTuple> downloads = new ArrayList<>(dir.getFilesCount());
- for (FileNode child : dir.getFilesList()) {
- Path childPath = path.getRelative(child.getName());
- downloads.add(
- new FuturePathBooleanTuple(
- downloadFile(childPath, child.getDigest()), childPath, child.getIsExecutable()));
- }
-
- for (DirectoryNode child : dir.getDirectoriesList()) {
- Path childPath = path.getRelative(child.getName());
- Digest childDigest = child.getDigest();
- Directory childDir = childrenMap.get(childDigest);
- if (childDir == null) {
+ private void createSymlinks(Iterable<SymlinkMetadata> symlinks) throws IOException {
+ for (SymlinkMetadata symlink : symlinks) {
+ if (symlink.target().isAbsolute()) {
+ // We do not support absolute symlinks as outputs.
throw new IOException(
- "could not find subdirectory "
- + child.getName()
- + " of directory "
- + path
- + " for download: digest "
- + childDigest
- + "not found");
+ String.format(
+ "Action output %s is a symbolic link to an absolute path %s. "
+ + "Symlinks to absolute paths in action outputs are not supported.",
+ symlink.path(), symlink.target()));
}
- downloads.addAll(downloadDirectory(childPath, childDir, childrenMap));
+ Preconditions.checkNotNull(
+ symlink.path().getParentDirectory(),
+ "Failed creating directory and parents for %s",
+ symlink.path())
+ .createDirectoryAndParents();
+ symlink.path().createSymbolicLink(symlink.target());
}
-
- return downloads;
}
/** Download a file (that is not a directory). The content is fetched from the digest. */
@@ -399,32 +323,240 @@
}
}
},
- MoreExecutors.directExecutor());
+ directExecutor());
return outerF;
}
- private List<FuturePathBooleanTuple> downloadOutErr(ActionResult result, FileOutErr outErr)
+ private List<ListenableFuture<FileMetadata>> downloadOutErr(ActionResult result, OutErr outErr)
throws IOException {
- List<FuturePathBooleanTuple> downloads = new ArrayList<>();
+ List<ListenableFuture<FileMetadata>> downloads = new ArrayList<>();
if (!result.getStdoutRaw().isEmpty()) {
result.getStdoutRaw().writeTo(outErr.getOutputStream());
outErr.getOutputStream().flush();
} else if (result.hasStdoutDigest()) {
downloads.add(
- new FuturePathBooleanTuple(
- downloadBlob(result.getStdoutDigest(), outErr.getOutputStream()), null, false));
+ Futures.transform(
+ downloadBlob(result.getStdoutDigest(), outErr.getOutputStream()),
+ (d) -> null,
+ directExecutor()));
}
if (!result.getStderrRaw().isEmpty()) {
result.getStderrRaw().writeTo(outErr.getErrorStream());
outErr.getErrorStream().flush();
} else if (result.hasStderrDigest()) {
downloads.add(
- new FuturePathBooleanTuple(
- downloadBlob(result.getStderrDigest(), outErr.getErrorStream()), null, false));
+ Futures.transform(
+ downloadBlob(result.getStderrDigest(), outErr.getErrorStream()),
+ (d) -> null,
+ directExecutor()));
}
return downloads;
}
+ /**
+ * Avoids downloading the majority of action outputs but injects their metadata using {@link
+ * MetadataInjector} instead.
+ *
+ * <p>This method only downloads output directory metadata, stdout and stderr as well as the
+ * contents of {@code inMemoryOutputPath} if specified.
+ *
+ * @param result the action result metadata of a successfully executed action (exit code = 0).
+ * @param outputs the action's declared output files
+ * @param inMemoryOutputPath the path of an output file whose contents should be returned in
+ * memory by this method.
+ * @param outErr stdout and stderr of this action
+ * @param execRoot the execution root
+ * @param metadataInjector the action's metadata injector that allows this method to inject
+ * metadata about an action output instead of downloading the output
+ * @throws IOException in case of failure
+ * @throws InterruptedException in case of receiving an interrupt
+ */
+ @Nullable
+ public InMemoryOutput downloadMinimal(
+ ActionResult result,
+ Collection<? extends ActionInput> outputs,
+ @Nullable PathFragment inMemoryOutputPath,
+ OutErr outErr,
+ Path execRoot,
+ MetadataInjector metadataInjector)
+ throws IOException, InterruptedException {
+ Preconditions.checkState(
+ result.getExitCode() == 0,
+ "injecting remote metadata is only supported for successful actions (exit code 0).");
+
+ ActionResultMetadata metadata;
+ try (SilentCloseable c = Profiler.instance().profile("Remote.parseActionResultMetadata")) {
+ metadata = parseActionResultMetadata(result, execRoot);
+ }
+
+ if (!metadata.symlinks().isEmpty()) {
+ throw new IOException(
+ "Symlinks in action outputs are not yet supported by "
+ + "--experimental_remote_fetch_outputs");
+ }
+
+ ActionInput inMemoryOutput = null;
+ Digest inMemoryOutputDigest = null;
+ for (ActionInput output : outputs) {
+ if (inMemoryOutputPath != null && output.getExecPath().equals(inMemoryOutputPath)) {
+ Path p = execRoot.getRelative(output.getExecPath());
+ FileMetadata m = Preconditions.checkNotNull(metadata.file(p), "inMemoryOutputMetadata");
+ inMemoryOutputDigest = m.digest();
+ inMemoryOutput = output;
+ }
+ if (output instanceof Artifact) {
+ injectRemoteArtifact((Artifact) output, metadata, execRoot, metadataInjector);
+ }
+ }
+
+ try (SilentCloseable c = Profiler.instance().profile("Remote.download")) {
+ ListenableFuture<byte[]> inMemoryOutputDownload = null;
+ if (inMemoryOutput != null) {
+ inMemoryOutputDownload = downloadBlob(inMemoryOutputDigest);
+ }
+ for (ListenableFuture<FileMetadata> download : downloadOutErr(result, outErr)) {
+ getFromFuture(download);
+ }
+ if (inMemoryOutputDownload != null) {
+ byte[] data = getFromFuture(inMemoryOutputDownload);
+ return new InMemoryOutput(inMemoryOutput, ByteString.copyFrom(data));
+ }
+ }
+ return null;
+ }
+
+ private void injectRemoteArtifact(
+ Artifact output,
+ ActionResultMetadata metadata,
+ Path execRoot,
+ MetadataInjector metadataInjector)
+ throws IOException {
+ if (output.isTreeArtifact()) {
+ DirectoryMetadata directory =
+ metadata.directory(execRoot.getRelative(output.getExecPathString()));
+ if (directory == null) {
+ // A declared output wasn't created. It might have been an optional output and if not
+ // SkyFrame will make sure to fail.
+ return;
+ }
+ if (!directory.symlinks().isEmpty()) {
+ throw new IOException(
+ "Symlinks in action outputs are not yet supported by "
+ + "--experimental_remote_fetch_outputs");
+ }
+ ImmutableMap.Builder<PathFragment, RemoteFileArtifactValue> childMetadata =
+ ImmutableMap.builder();
+ for (FileMetadata file : directory.files()) {
+ PathFragment p = file.path().relativeTo(output.getPath());
+ RemoteFileArtifactValue r =
+ new RemoteFileArtifactValue(
+ DigestUtil.toBinaryDigest(file.digest()),
+ file.digest().getSizeBytes(),
+ /* locationIndex= */ 1);
+ childMetadata.put(p, r);
+ }
+ metadataInjector.injectRemoteDirectory(output, childMetadata.build());
+ } else {
+ FileMetadata outputMetadata = metadata.file(execRoot.getRelative(output.getExecPathString()));
+ if (outputMetadata == null) {
+ // A declared output wasn't created. It might have been an optional output and if not
+ // SkyFrame will make sure to fail.
+ return;
+ }
+ metadataInjector.injectRemoteFile(
+ output,
+ DigestUtil.toBinaryDigest(outputMetadata.digest()),
+ outputMetadata.digest().getSizeBytes(),
+ /* locationIndex= */ 1);
+ }
+ }
+
+ private DirectoryMetadata parseDirectory(
+ Path parent, Directory dir, Map<Digest, Directory> childDirectoriesMap) {
+ ImmutableList.Builder<FileMetadata> filesBuilder = ImmutableList.builder();
+ for (FileNode file : dir.getFilesList()) {
+ filesBuilder.add(
+ new FileMetadata(
+ parent.getRelative(file.getName()), file.getDigest(), file.getIsExecutable()));
+ }
+
+ ImmutableList.Builder<SymlinkMetadata> symlinksBuilder = ImmutableList.builder();
+ for (SymlinkNode symlink : dir.getSymlinksList()) {
+ symlinksBuilder.add(
+ new SymlinkMetadata(
+ parent.getRelative(symlink.getName()), PathFragment.create(symlink.getTarget())));
+ }
+
+ for (DirectoryNode directoryNode : dir.getDirectoriesList()) {
+ Path childPath = parent.getRelative(directoryNode.getName());
+ Directory childDir =
+ Preconditions.checkNotNull(childDirectoriesMap.get(directoryNode.getDigest()));
+ DirectoryMetadata childMetadata = parseDirectory(childPath, childDir, childDirectoriesMap);
+ filesBuilder.addAll(childMetadata.files());
+ symlinksBuilder.addAll(childMetadata.symlinks());
+ }
+
+ return new DirectoryMetadata(filesBuilder.build(), symlinksBuilder.build());
+ }
+
+ private ActionResultMetadata parseActionResultMetadata(ActionResult actionResult, Path execRoot)
+ throws IOException, InterruptedException {
+ Preconditions.checkNotNull(actionResult, "actionResult");
+ Map<Path, ListenableFuture<Tree>> dirMetadataDownloads =
+ Maps.newHashMapWithExpectedSize(actionResult.getOutputDirectoriesCount());
+ for (OutputDirectory dir : actionResult.getOutputDirectoriesList()) {
+ dirMetadataDownloads.put(
+ execRoot.getRelative(dir.getPath()),
+ Futures.transform(
+ downloadBlob(dir.getTreeDigest()),
+ (treeBytes) -> {
+ try {
+ return Tree.parseFrom(treeBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new RuntimeException(e);
+ }
+ },
+ directExecutor()));
+ }
+
+ ImmutableMap.Builder<Path, DirectoryMetadata> directories = ImmutableMap.builder();
+ for (Map.Entry<Path, ListenableFuture<Tree>> metadataDownload :
+ dirMetadataDownloads.entrySet()) {
+ Path path = metadataDownload.getKey();
+ Tree directoryTree = getFromFuture(metadataDownload.getValue());
+ Map<Digest, Directory> childrenMap = new HashMap<>();
+ for (Directory childDir : directoryTree.getChildrenList()) {
+ childrenMap.put(digestUtil.compute(childDir), childDir);
+ }
+
+ directories.put(path, parseDirectory(path, directoryTree.getRoot(), childrenMap));
+ }
+
+ ImmutableMap.Builder<Path, FileMetadata> files = ImmutableMap.builder();
+ for (OutputFile outputFile : actionResult.getOutputFilesList()) {
+ files.put(
+ execRoot.getRelative(outputFile.getPath()),
+ new FileMetadata(
+ execRoot.getRelative(outputFile.getPath()),
+ outputFile.getDigest(),
+ outputFile.getIsExecutable()));
+ }
+
+ ImmutableMap.Builder<Path, SymlinkMetadata> symlinks = ImmutableMap.builder();
+ Iterable<OutputSymlink> outputSymlinks =
+ Iterables.concat(
+ actionResult.getOutputFileSymlinksList(),
+ actionResult.getOutputDirectorySymlinksList());
+ for (OutputSymlink symlink : outputSymlinks) {
+ symlinks.put(
+ execRoot.getRelative(symlink.getPath()),
+ new SymlinkMetadata(
+ execRoot.getRelative(symlink.getPath()), PathFragment.create(symlink.getTarget())));
+ }
+
+ return new ActionResultMetadata(files.build(), symlinks.build(), directories.build());
+ }
+
/** UploadManifest adds output metadata to a {@link ActionResult}. */
static class UploadManifest {
private final DigestUtil digestUtil;
@@ -643,7 +775,7 @@
return b.build();
}
- private void illegalOutput(Path what) throws ExecException, IOException {
+ private void illegalOutput(Path what) throws ExecException {
String kind = what.isSymbolicLink() ? "symbolic link" : "special file";
throw new UserExecException(
String.format(
@@ -719,4 +851,109 @@
}
}
}
+
+ /** In-memory representation of action result metadata. */
+ static class ActionResultMetadata {
+
+ static class SymlinkMetadata {
+ private final Path path;
+ private final PathFragment target;
+
+ private SymlinkMetadata(Path path, PathFragment target) {
+ this.path = path;
+ this.target = target;
+ }
+
+ public Path path() {
+ return path;
+ }
+
+ public PathFragment target() {
+ return target;
+ }
+ }
+
+ static class FileMetadata {
+ private final Path path;
+ private final Digest digest;
+ private final boolean isExecutable;
+
+ private FileMetadata(Path path, Digest digest, boolean isExecutable) {
+ this.path = path;
+ this.digest = digest;
+ this.isExecutable = isExecutable;
+ }
+
+ public Path path() {
+ return path;
+ }
+
+ public Digest digest() {
+ return digest;
+ }
+
+ public boolean isExecutable() {
+ return isExecutable;
+ }
+ }
+
+ static class DirectoryMetadata {
+ private final ImmutableList<FileMetadata> files;
+ private final ImmutableList<SymlinkMetadata> symlinks;
+
+ private DirectoryMetadata(
+ ImmutableList<FileMetadata> files, ImmutableList<SymlinkMetadata> symlinks) {
+ this.files = files;
+ this.symlinks = symlinks;
+ }
+
+ public ImmutableList<FileMetadata> files() {
+ return files;
+ }
+
+ public ImmutableList<SymlinkMetadata> symlinks() {
+ return symlinks;
+ }
+ }
+
+ private final ImmutableMap<Path, FileMetadata> files;
+ private final ImmutableMap<Path, SymlinkMetadata> symlinks;
+ private final ImmutableMap<Path, DirectoryMetadata> directories;
+
+ private ActionResultMetadata(
+ ImmutableMap<Path, FileMetadata> files,
+ ImmutableMap<Path, SymlinkMetadata> symlinks,
+ ImmutableMap<Path, DirectoryMetadata> directories) {
+ this.files = files;
+ this.symlinks = symlinks;
+ this.directories = directories;
+ }
+
+ @Nullable
+ public FileMetadata file(Path path) {
+ return files.get(path);
+ }
+
+ @Nullable
+ public DirectoryMetadata directory(Path path) {
+ return directories.get(path);
+ }
+
+ public Collection<FileMetadata> files() {
+ return files.values();
+ }
+
+ public ImmutableSet<Entry<Path, DirectoryMetadata>> directories() {
+ return directories.entrySet();
+ }
+
+ public Collection<SymlinkMetadata> symlinks() {
+ return symlinks.values();
+ }
+ }
+
+ @VisibleForTesting
+ protected <T> T getFromFuture(ListenableFuture<T> f) throws IOException, InterruptedException {
+ return Utils.getFromFuture(f);
+ }
}