From 16210f581a007e90af3916d326c42088cfbead10 Mon Sep 17 00:00:00 2001 From: Tiago Quelhas Date: Wed, 10 Jan 2024 18:00:13 +0000 Subject: [PATCH] [7.1.0] Introduce a SpawnLogContext interface. (#20842) For now, the sole implementation is ExpandedSpawnLogContext, which subsumes the old SpawnLogContext class, as well as some of the logic around log formats currently in SpawnLogModule. In a followup, a separate CompactSpawnLogContext implementation will be introduced. This will let us experiment with a new log format while minimizing the chance of accidentally breaking the existing formats. PiperOrigin-RevId: 588444807 Change-Id: I75bc2a577ec45202baf95c950a257eaf420cbeb0 --- .../com/google/devtools/build/lib/bazel/BUILD | 3 - .../build/lib/bazel/SpawnLogModule.java | 107 +---- .../com/google/devtools/build/lib/exec/BUILD | 7 +- .../lib/exec/ExpandedSpawnLogContext.java | 323 +++++++++++++++ .../build/lib/exec/SpawnLogContext.java | 322 ++++----------- ....java => ExpandedSpawnLogContextTest.java} | 377 +++++++++--------- 6 files changed, 608 insertions(+), 531 deletions(-) create mode 100644 src/main/java/com/google/devtools/build/lib/exec/ExpandedSpawnLogContext.java rename src/test/java/com/google/devtools/build/lib/exec/{SpawnLogContextTest.java => ExpandedSpawnLogContextTest.java} (70%) diff --git a/src/main/java/com/google/devtools/build/lib/bazel/BUILD b/src/main/java/com/google/devtools/build/lib/bazel/BUILD index 7de94c3cf50b2e..2372a76de605e0 100644 --- a/src/main/java/com/google/devtools/build/lib/bazel/BUILD +++ b/src/main/java/com/google/devtools/build/lib/bazel/BUILD @@ -129,7 +129,6 @@ java_library( ], deps = [ "//src/main/java/com/google/devtools/build/lib:runtime", - "//src/main/java/com/google/devtools/build/lib/bazel/execlog:stable_sort", "//src/main/java/com/google/devtools/build/lib/events", "//src/main/java/com/google/devtools/build/lib/exec:execution_options", "//src/main/java/com/google/devtools/build/lib/exec:executor_builder", @@ -138,10 +137,8 @@ java_library( "//src/main/java/com/google/devtools/build/lib/remote/options", "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/util:detailed_exit_code", - "//src/main/java/com/google/devtools/build/lib/util/io", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/protobuf:failure_details_java_proto", - "//src/main/protobuf:spawn_java_proto", "//third_party:guava", "//third_party:jsr305", ], diff --git a/src/main/java/com/google/devtools/build/lib/bazel/SpawnLogModule.java b/src/main/java/com/google/devtools/build/lib/bazel/SpawnLogModule.java index 99d457a5ad28da..9d21bc657f51f2 100644 --- a/src/main/java/com/google/devtools/build/lib/bazel/SpawnLogModule.java +++ b/src/main/java/com/google/devtools/build/lib/bazel/SpawnLogModule.java @@ -15,13 +15,13 @@ import static com.google.common.base.Preconditions.checkNotNull; -import com.google.devtools.build.lib.bazel.execlog.StableSort; import com.google.devtools.build.lib.buildtool.BuildRequest; import com.google.devtools.build.lib.events.Event; import com.google.devtools.build.lib.exec.ExecutionOptions; import com.google.devtools.build.lib.exec.ExecutorBuilder; +import com.google.devtools.build.lib.exec.ExpandedSpawnLogContext; +import com.google.devtools.build.lib.exec.ExpandedSpawnLogContext.Encoding; import com.google.devtools.build.lib.exec.ModuleActionContextRegistry; -import com.google.devtools.build.lib.exec.Protos.SpawnExec; import com.google.devtools.build.lib.exec.SpawnLogContext; import com.google.devtools.build.lib.remote.options.RemoteOptions; import com.google.devtools.build.lib.runtime.BlazeModule; @@ -32,41 +32,16 @@ import com.google.devtools.build.lib.server.FailureDetails.FailureDetail; import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.util.DetailedExitCode; -import com.google.devtools.build.lib.util.io.AsynchronousMessageOutputStream; -import com.google.devtools.build.lib.util.io.MessageOutputStream; -import com.google.devtools.build.lib.util.io.MessageOutputStreamWrapper.BinaryOutputStreamWrapper; -import com.google.devtools.build.lib.util.io.MessageOutputStreamWrapper.JsonOutputStreamWrapper; import com.google.devtools.build.lib.vfs.Path; import java.io.IOException; -import java.io.InputStream; import javax.annotation.Nullable; /** Module providing on-demand spawn logging. */ public final class SpawnLogModule extends BlazeModule { @Nullable private SpawnLogContext spawnLogContext; - /** Output path for the raw output stream. */ - @Nullable private Path rawOutputPath; - - /** Output stream to write directly into during execution. */ - @Nullable private MessageOutputStream rawOutputStream; - - /** - * Output stream to convert the raw output into after the execution is done. - * - *

We open the stream at the beginning of the command so that any errors (e.g., unwritable - * location) are surfaced before execution begins. - */ - @Nullable private MessageOutputStream convertedOutputStream; - - private CommandEnvironment env; - private void clear() { spawnLogContext = null; - rawOutputPath = null; - rawOutputStream = null; - convertedOutputStream = null; - env = null; } private void initOutputs(CommandEnvironment env) throws IOException { @@ -99,45 +74,30 @@ private void initOutputs(CommandEnvironment env) throws IOException { return; } - this.env = env; - Path workingDirectory = env.getWorkingDirectory(); Path outputBase = env.getOutputBase(); - // Set up the raw output stream. - // This stream performs the writes in a separate thread to avoid blocking execution. - // If the unsorted binary format was requested, use the respective output path to avoid a - // pointless conversion at the end. Otherwise, use a temporary path. - if (executionOptions.executionLogBinaryFile != null && !executionOptions.executionLogSort) { - rawOutputPath = workingDirectory.getRelative(executionOptions.executionLogBinaryFile); - } else { - rawOutputPath = outputBase.getRelative("execution.log"); - } - rawOutputStream = new AsynchronousMessageOutputStream<>(rawOutputPath); - - // Set up the binary output stream, if distinct from the raw output stream. - if (executionOptions.executionLogBinaryFile != null && executionOptions.executionLogSort) { - convertedOutputStream = - new BinaryOutputStreamWrapper<>( - workingDirectory - .getRelative(executionOptions.executionLogBinaryFile) - .getOutputStream()); + Path outputPath = null; + Encoding encoding = null; + if (executionOptions.executionLogBinaryFile != null) { + encoding = Encoding.BINARY; + outputPath = workingDirectory.getRelative(executionOptions.executionLogBinaryFile); + } else if (executionOptions.executionLogJsonFile != null) { + encoding = Encoding.JSON; + outputPath = workingDirectory.getRelative(executionOptions.executionLogJsonFile); } - // Set up the text output stream. - if (executionOptions.executionLogJsonFile != null) { - convertedOutputStream = - new JsonOutputStreamWrapper<>( - workingDirectory - .getRelative(executionOptions.executionLogJsonFile) - .getOutputStream()); - } + // Use a well-known temporary path to avoid accumulation of potentially large files in /tmp + // due to abnormally terminated invocations (e.g., when running out of memory). + Path tempPath = outputBase.getRelative("execution.log"); spawnLogContext = - new SpawnLogContext( + new ExpandedSpawnLogContext( + checkNotNull(outputPath), + tempPath, + checkNotNull(encoding), + /* sorted= */ executionOptions.executionLogSort, env.getExecRoot().asFragment(), - rawOutputStream, - env.getOptions().getOptions(ExecutionOptions.class), env.getOptions().getOptions(RemoteOptions.class), env.getRuntime().getFileSystem().getDigestFunction(), env.getXattrProvider()); @@ -180,44 +140,13 @@ public void afterCommand() throws AbruptExitException { return; } - checkNotNull(rawOutputPath); - - boolean done = false; try { spawnLogContext.close(); - if (convertedOutputStream != null) { - InputStream in = rawOutputPath.getInputStream(); - if (spawnLogContext.shouldSort()) { - StableSort.stableSort(in, convertedOutputStream); - } else { - while (in.available() > 0) { - SpawnExec ex = SpawnExec.parseDelimitedFrom(in); - convertedOutputStream.write(ex); - } - } - convertedOutputStream.close(); - } - done = true; } catch (IOException e) { String message = e.getMessage() == null ? "Error writing execution log" : e.getMessage(); throw new AbruptExitException( createDetailedExitCode(message, Code.EXECUTION_LOG_WRITE_FAILURE), e); } finally { - if (convertedOutputStream != null) { - if (!done) { - env.getReporter() - .handle( - Event.warn( - "Execution log might not have been populated. Raw execution log is at " - + rawOutputPath)); - } else { - try { - rawOutputPath.delete(); - } catch (IOException e) { - // Intentionally ignored. - } - } - } clear(); } } diff --git a/src/main/java/com/google/devtools/build/lib/exec/BUILD b/src/main/java/com/google/devtools/build/lib/exec/BUILD index b1b02b40f36159..424882321ebf20 100644 --- a/src/main/java/com/google/devtools/build/lib/exec/BUILD +++ b/src/main/java/com/google/devtools/build/lib/exec/BUILD @@ -257,13 +257,16 @@ java_library( java_library( name = "spawn_log_context", - srcs = ["SpawnLogContext.java"], + srcs = [ + "ExpandedSpawnLogContext.java", + "SpawnLogContext.java", + ], deps = [ - ":execution_options", "//src/main/java/com/google/devtools/build/lib/actions", "//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/analysis/platform:platform_utils", + "//src/main/java/com/google/devtools/build/lib/bazel/execlog:stable_sort", "//src/main/java/com/google/devtools/build/lib/profiler", "//src/main/java/com/google/devtools/build/lib/remote/options", "//src/main/java/com/google/devtools/build/lib/util/io", diff --git a/src/main/java/com/google/devtools/build/lib/exec/ExpandedSpawnLogContext.java b/src/main/java/com/google/devtools/build/lib/exec/ExpandedSpawnLogContext.java new file mode 100644 index 00000000000000..8f5a6c451b1c6b --- /dev/null +++ b/src/main/java/com/google/devtools/build/lib/exec/ExpandedSpawnLogContext.java @@ -0,0 +1,323 @@ +// 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.exec; + +import static com.google.devtools.build.lib.exec.SpawnLogContext.computeDigest; +import static com.google.devtools.build.lib.exec.SpawnLogContext.getSpawnMetricsProto; + +import build.bazel.remote.execution.v2.Platform; +import com.google.common.collect.ImmutableSet; +import com.google.common.flogger.GoogleLogger; +import com.google.devtools.build.lib.actions.ActionInput; +import com.google.devtools.build.lib.actions.Artifact.TreeFileArtifact; +import com.google.devtools.build.lib.actions.ExecException; +import com.google.devtools.build.lib.actions.InputMetadataProvider; +import com.google.devtools.build.lib.actions.Spawn; +import com.google.devtools.build.lib.actions.SpawnResult; +import com.google.devtools.build.lib.actions.Spawns; +import com.google.devtools.build.lib.actions.cache.VirtualActionInput; +import com.google.devtools.build.lib.analysis.platform.PlatformUtils; +import com.google.devtools.build.lib.bazel.execlog.StableSort; +import com.google.devtools.build.lib.exec.Protos.Digest; +import com.google.devtools.build.lib.exec.Protos.File; +import com.google.devtools.build.lib.exec.Protos.SpawnExec; +import com.google.devtools.build.lib.profiler.Profiler; +import com.google.devtools.build.lib.profiler.SilentCloseable; +import com.google.devtools.build.lib.remote.options.RemoteOptions; +import com.google.devtools.build.lib.util.io.AsynchronousMessageOutputStream; +import com.google.devtools.build.lib.util.io.MessageOutputStream; +import com.google.devtools.build.lib.util.io.MessageOutputStreamWrapper.BinaryOutputStreamWrapper; +import com.google.devtools.build.lib.util.io.MessageOutputStreamWrapper.JsonOutputStreamWrapper; +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.Path; +import com.google.devtools.build.lib.vfs.PathFragment; +import com.google.devtools.build.lib.vfs.Symlinks; +import com.google.devtools.build.lib.vfs.XattrProvider; +import java.io.IOException; +import java.io.InputStream; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.function.Consumer; +import javax.annotation.Nullable; + +/** A {@link SpawnLogContext} implementation that produces a log in expanded format. */ +public class ExpandedSpawnLogContext implements SpawnLogContext { + + /** The log encoding. */ + public enum Encoding { + /** Length-delimited binary protos. */ + BINARY, + /** Newline-delimited JSON messages. */ + JSON + } + + private static final GoogleLogger logger = GoogleLogger.forEnclosingClass(); + + private final Path tempPath; + private final boolean sorted; + + private final PathFragment execRoot; + @Nullable private final RemoteOptions remoteOptions; + private final DigestHashFunction digestHashFunction; + private final XattrProvider xattrProvider; + + /** Output stream to write directly into during execution. */ + private final MessageOutputStream rawOutputStream; + + /** Output stream to convert the raw output stream into after execution, if required. */ + @Nullable private final MessageOutputStream convertedOutputStream; + + public ExpandedSpawnLogContext( + Path outputPath, + Path tempPath, + Encoding encoding, + boolean sorted, + PathFragment execRoot, + @Nullable RemoteOptions remoteOptions, + DigestHashFunction digestHashFunction, + XattrProvider xattrProvider) + throws IOException { + this.tempPath = tempPath; + this.sorted = sorted; + this.execRoot = execRoot; + this.remoteOptions = remoteOptions; + this.digestHashFunction = digestHashFunction; + this.xattrProvider = xattrProvider; + + if (encoding == Encoding.BINARY && !sorted) { + // The unsorted binary format can be written directly into the output path during execution. + rawOutputStream = getRawOutputStream(outputPath); + convertedOutputStream = null; + } else { + // Otherwise, write the unsorted binary format into a temporary path first, then convert into + // the output format after execution. + rawOutputStream = getRawOutputStream(tempPath); + convertedOutputStream = getConvertedOutputStream(encoding, outputPath); + } + } + + private static MessageOutputStream getRawOutputStream(Path path) throws IOException { + // Use an AsynchronousMessageOutputStream so that writes occur in a separate thread. + // This ensures concurrent writes don't tear and avoids blocking execution. + return new AsynchronousMessageOutputStream<>(path); + } + + private static MessageOutputStream getConvertedOutputStream( + Encoding encoding, Path path) throws IOException { + switch (encoding) { + case BINARY: + return new BinaryOutputStreamWrapper<>(path.getOutputStream()); + case JSON: + return new JsonOutputStreamWrapper<>(path.getOutputStream()); + } + throw new IllegalArgumentException( + String.format("invalid execution log encoding: %s", encoding)); + } + + @Override + public void logSpawn( + Spawn spawn, + InputMetadataProvider inputMetadataProvider, + SortedMap inputMap, + FileSystem fileSystem, + Duration timeout, + SpawnResult result) + throws IOException, ExecException { + SortedMap existingOutputs = listExistingOutputs(spawn, fileSystem); + SpawnExec.Builder builder = SpawnExec.newBuilder(); + builder.addAllCommandArgs(spawn.getArguments()); + + Map env = spawn.getEnvironment(); + // Sorting the environment pairs by variable name. + TreeSet variables = new TreeSet<>(env.keySet()); + for (String var : variables) { + builder.addEnvironmentVariablesBuilder().setName(var).setValue(env.get(var)); + } + + ImmutableSet toolFiles = spawn.getToolFiles().toSet(); + + try (SilentCloseable c = Profiler.instance().profile("logSpawn/inputs")) { + for (Map.Entry e : inputMap.entrySet()) { + ActionInput input = e.getValue(); + if (input instanceof VirtualActionInput.EmptyActionInput) { + continue; + } + Path inputPath = fileSystem.getPath(execRoot.getRelative(input.getExecPathString())); + if (inputPath.isDirectory()) { + listDirectoryContents(inputPath, builder::addInputs, inputMetadataProvider); + continue; + } + Digest digest = + computeDigest( + input, inputPath, inputMetadataProvider, xattrProvider, digestHashFunction); + boolean isTool = + toolFiles.contains(input) + || (input instanceof TreeFileArtifact + && toolFiles.contains(((TreeFileArtifact) input).getParent())); + builder + .addInputsBuilder() + .setPath(input.getExecPathString()) + .setDigest(digest) + .setIsTool(isTool); + } + } catch (IOException e) { + logger.atWarning().withCause(e).log("Error computing spawn inputs"); + } + try (SilentCloseable c = Profiler.instance().profile("logSpawn/outputs")) { + ArrayList outputPaths = new ArrayList<>(); + for (ActionInput output : spawn.getOutputFiles()) { + outputPaths.add(output.getExecPathString()); + } + Collections.sort(outputPaths); + builder.addAllListedOutputs(outputPaths); + for (Map.Entry e : existingOutputs.entrySet()) { + Path path = e.getKey(); + if (path.isDirectory()) { + listDirectoryContents(path, builder::addActualOutputs, inputMetadataProvider); + } else { + File.Builder outputBuilder = builder.addActualOutputsBuilder(); + outputBuilder.setPath(path.relativeTo(fileSystem.getPath(execRoot)).toString()); + try { + outputBuilder.setDigest( + computeDigest( + e.getValue(), path, inputMetadataProvider, xattrProvider, digestHashFunction)); + } catch (IOException ex) { + logger.atWarning().withCause(ex).log("Error computing spawn event output properties"); + } + } + } + } + builder.setRemotable(Spawns.mayBeExecutedRemotely(spawn)); + + Platform execPlatform = PlatformUtils.getPlatformProto(spawn, remoteOptions); + if (execPlatform != null) { + builder.setPlatform(buildPlatform(execPlatform)); + } + if (result.status() != SpawnResult.Status.SUCCESS) { + builder.setStatus(result.status().toString()); + } + if (!timeout.isZero()) { + builder.setTimeoutMillis(timeout.toMillis()); + } + builder.setCacheable(Spawns.mayBeCached(spawn)); + builder.setRemoteCacheable(Spawns.mayBeCachedRemotely(spawn)); + builder.setExitCode(result.exitCode()); + builder.setCacheHit(result.isCacheHit()); + builder.setRunner(result.getRunnerName()); + + if (result.getDigest() != null) { + builder.setDigest(result.getDigest()); + } + + builder.setMnemonic(spawn.getMnemonic()); + + if (spawn.getTargetLabel() != null) { + builder.setTargetLabel(spawn.getTargetLabel().toString()); + } + + builder.setMetrics(getSpawnMetricsProto(result)); + + try (SilentCloseable c = Profiler.instance().profile("logSpawn/write")) { + rawOutputStream.write(builder.build()); + } + } + + @Override + public void close() throws IOException { + rawOutputStream.close(); + + if (convertedOutputStream == null) { + // No conversion required. + return; + } + + try (InputStream in = tempPath.getInputStream()) { + if (sorted) { + StableSort.stableSort(in, convertedOutputStream); + } else { + while (in.available() > 0) { + SpawnExec ex = SpawnExec.parseDelimitedFrom(in); + convertedOutputStream.write(ex); + } + } + } finally { + try { + tempPath.delete(); + } catch (IOException e) { + // Intentionally ignored. + } + } + } + + private static Protos.Platform buildPlatform(Platform platform) { + Protos.Platform.Builder platformBuilder = Protos.Platform.newBuilder(); + for (Platform.Property p : platform.getPropertiesList()) { + platformBuilder.addPropertiesBuilder().setName(p.getName()).setValue(p.getValue()); + } + return platformBuilder.build(); + } + + private SortedMap listExistingOutputs(Spawn spawn, FileSystem fileSystem) { + TreeMap result = new TreeMap<>(); + for (ActionInput output : spawn.getOutputFiles()) { + Path outputPath = fileSystem.getPath(execRoot.getRelative(output.getExecPathString())); + // TODO(olaola): once symlink API proposal is implemented, report symlinks here. + if (outputPath.exists()) { + result.put(outputPath, output); + } + } + return result; + } + + private void listDirectoryContents( + Path path, Consumer addFile, InputMetadataProvider inputMetadataProvider) { + try { + // TODO(olaola): once symlink API proposal is implemented, report symlinks here. + List sortedDirent = new ArrayList<>(path.readdir(Symlinks.NOFOLLOW)); + sortedDirent.sort(Comparator.comparing(Dirent::getName)); + for (Dirent dirent : sortedDirent) { + String name = dirent.getName(); + Path child = path.getRelative(name); + if (dirent.getType() == Dirent.Type.DIRECTORY) { + listDirectoryContents(child, addFile, inputMetadataProvider); + } else { + String pathString; + if (child.startsWith(execRoot)) { + pathString = child.asFragment().relativeTo(execRoot).getPathString(); + } else { + pathString = child.getPathString(); + } + addFile.accept( + File.newBuilder() + .setPath(pathString) + .setDigest( + computeDigest( + null, child, inputMetadataProvider, xattrProvider, digestHashFunction)) + .build()); + } + } + } catch (IOException e) { + logger.atWarning().withCause(e).log("Error computing spawn event file properties"); + } + } +} diff --git a/src/main/java/com/google/devtools/build/lib/exec/SpawnLogContext.java b/src/main/java/com/google/devtools/build/lib/exec/SpawnLogContext.java index 765034996d46ce..bb655b530be28d 100644 --- a/src/main/java/com/google/devtools/build/lib/exec/SpawnLogContext.java +++ b/src/main/java/com/google/devtools/build/lib/exec/SpawnLogContext.java @@ -13,306 +13,70 @@ // limitations under the License. package com.google.devtools.build.lib.exec; -import build.bazel.remote.execution.v2.Platform; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableSet; -import com.google.common.flogger.GoogleLogger; import com.google.common.hash.HashCode; import com.google.devtools.build.lib.actions.ActionContext; import com.google.devtools.build.lib.actions.ActionInput; -import com.google.devtools.build.lib.actions.Artifact.TreeFileArtifact; import com.google.devtools.build.lib.actions.ExecException; import com.google.devtools.build.lib.actions.FileArtifactValue; import com.google.devtools.build.lib.actions.InputMetadataProvider; import com.google.devtools.build.lib.actions.Spawn; import com.google.devtools.build.lib.actions.SpawnMetrics; import com.google.devtools.build.lib.actions.SpawnResult; -import com.google.devtools.build.lib.actions.Spawns; import com.google.devtools.build.lib.actions.cache.VirtualActionInput; -import com.google.devtools.build.lib.analysis.platform.PlatformUtils; import com.google.devtools.build.lib.exec.Protos.Digest; -import com.google.devtools.build.lib.exec.Protos.File; -import com.google.devtools.build.lib.exec.Protos.SpawnExec; -import com.google.devtools.build.lib.profiler.Profiler; -import com.google.devtools.build.lib.profiler.SilentCloseable; -import com.google.devtools.build.lib.remote.options.RemoteOptions; -import com.google.devtools.build.lib.util.io.MessageOutputStream; import com.google.devtools.build.lib.vfs.DigestHashFunction; import com.google.devtools.build.lib.vfs.DigestUtils; -import com.google.devtools.build.lib.vfs.Dirent; import com.google.devtools.build.lib.vfs.FileSystem; 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.XattrProvider; import com.google.protobuf.util.Durations; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; import java.util.SortedMap; -import java.util.TreeMap; -import java.util.TreeSet; -import java.util.function.Consumer; import javax.annotation.Nullable; -/** - * A logging utility for spawns. - */ -public class SpawnLogContext implements ActionContext { - - private static final GoogleLogger logger = GoogleLogger.forEnclosingClass(); - - private final PathFragment execRoot; - private final MessageOutputStream executionLog; - @Nullable private final ExecutionOptions executionOptions; - @Nullable private final RemoteOptions remoteOptions; - private final DigestHashFunction digestHashFunction; - private final XattrProvider xattrProvider; - - public SpawnLogContext( - PathFragment execRoot, - MessageOutputStream executionLog, - @Nullable ExecutionOptions executionOptions, - @Nullable RemoteOptions remoteOptions, - DigestHashFunction digestHashFunction, - XattrProvider xattrProvider) { - this.execRoot = execRoot; - this.executionLog = executionLog; - this.executionOptions = executionOptions; - this.remoteOptions = remoteOptions; - this.digestHashFunction = digestHashFunction; - this.xattrProvider = xattrProvider; - } - +/** An {@link ActionContext} providing the ability to log executed spawns. */ +public interface SpawnLogContext extends ActionContext { /** - * Logs the executed spawn to the output stream. + * Logs an executed spawn. + * + *

May be called concurrently. * * @param spawn the spawn to log * @param inputMetadataProvider provides metadata for the spawn inputs + * @param inputMap the mapping from input paths to action inputs * @param fileSystem the filesystem containing the spawn inputs and outputs, which might be an * action filesystem when building without the bytes * @param timeout the timeout the spawn was run under * @param result the spawn result */ - public void logSpawn( + void logSpawn( Spawn spawn, InputMetadataProvider inputMetadataProvider, SortedMap inputMap, FileSystem fileSystem, Duration timeout, SpawnResult result) - throws IOException, ExecException { - SortedMap existingOutputs = listExistingOutputs(spawn, fileSystem); - SpawnExec.Builder builder = SpawnExec.newBuilder(); - builder.addAllCommandArgs(spawn.getArguments()); - - Map env = spawn.getEnvironment(); - // Sorting the environment pairs by variable name. - TreeSet variables = new TreeSet<>(env.keySet()); - for (String var : variables) { - builder.addEnvironmentVariablesBuilder().setName(var).setValue(env.get(var)); - } - - ImmutableSet toolFiles = spawn.getToolFiles().toSet(); - - try (SilentCloseable c = Profiler.instance().profile("logSpawn/inputs")) { - for (Map.Entry e : inputMap.entrySet()) { - ActionInput input = e.getValue(); - if (input instanceof VirtualActionInput.EmptyActionInput) { - continue; - } - Path inputPath = fileSystem.getPath(execRoot.getRelative(input.getExecPathString())); - if (inputPath.isDirectory()) { - listDirectoryContents(inputPath, builder::addInputs, inputMetadataProvider); - continue; - } - Digest digest = computeDigest(input, inputPath, inputMetadataProvider, xattrProvider); - boolean isTool = - toolFiles.contains(input) - || (input instanceof TreeFileArtifact - && toolFiles.contains(((TreeFileArtifact) input).getParent())); - builder - .addInputsBuilder() - .setPath(input.getExecPathString()) - .setDigest(digest) - .setIsTool(isTool); - } - } catch (IOException e) { - logger.atWarning().withCause(e).log("Error computing spawn inputs"); - } - try (SilentCloseable c = Profiler.instance().profile("logSpawn/outputs")) { - ArrayList outputPaths = new ArrayList<>(); - for (ActionInput output : spawn.getOutputFiles()) { - outputPaths.add(output.getExecPathString()); - } - Collections.sort(outputPaths); - builder.addAllListedOutputs(outputPaths); - for (Map.Entry e : existingOutputs.entrySet()) { - Path path = e.getKey(); - if (path.isDirectory()) { - listDirectoryContents(path, builder::addActualOutputs, inputMetadataProvider); - } else { - File.Builder outputBuilder = builder.addActualOutputsBuilder(); - outputBuilder.setPath(path.relativeTo(fileSystem.getPath(execRoot)).toString()); - try { - outputBuilder.setDigest( - computeDigest(e.getValue(), path, inputMetadataProvider, xattrProvider)); - } catch (IOException ex) { - logger.atWarning().withCause(ex).log("Error computing spawn event output properties"); - } - } - } - } - builder.setRemotable(Spawns.mayBeExecutedRemotely(spawn)); - - Platform execPlatform = PlatformUtils.getPlatformProto(spawn, remoteOptions); - if (execPlatform != null) { - builder.setPlatform(buildPlatform(execPlatform)); - } - if (result.status() != SpawnResult.Status.SUCCESS) { - builder.setStatus(result.status().toString()); - } - if (!timeout.isZero()) { - builder.setTimeoutMillis(timeout.toMillis()); - } - builder.setCacheable(Spawns.mayBeCached(spawn)); - builder.setRemoteCacheable(Spawns.mayBeCachedRemotely(spawn)); - builder.setExitCode(result.exitCode()); - builder.setCacheHit(result.isCacheHit()); - builder.setRunner(result.getRunnerName()); - - if (result.getDigest() != null) { - builder.setDigest(result.getDigest()); - } - - builder.setMnemonic(spawn.getMnemonic()); - - if (spawn.getTargetLabel() != null) { - builder.setTargetLabel(spawn.getTargetLabel().toString()); - } - - SpawnMetrics metrics = result.getMetrics(); - Protos.SpawnMetrics.Builder metricsBuilder = builder.getMetricsBuilder(); - if (metrics.totalTimeInMs() != 0L) { - metricsBuilder.setTotalTime(millisToProto(metrics.totalTimeInMs())); - } - if (metrics.parseTimeInMs() != 0L) { - metricsBuilder.setParseTime(millisToProto(metrics.parseTimeInMs())); - } - if (metrics.networkTimeInMs() != 0L) { - metricsBuilder.setNetworkTime(millisToProto(metrics.networkTimeInMs())); - } - if (metrics.fetchTimeInMs() != 0L) { - metricsBuilder.setFetchTime(millisToProto(metrics.fetchTimeInMs())); - } - if (metrics.queueTimeInMs() != 0L) { - metricsBuilder.setQueueTime(millisToProto(metrics.queueTimeInMs())); - } - if (metrics.setupTimeInMs() != 0L) { - metricsBuilder.setSetupTime(millisToProto(metrics.setupTimeInMs())); - } - if (metrics.uploadTimeInMs() != 0L) { - metricsBuilder.setUploadTime(millisToProto(metrics.uploadTimeInMs())); - } - if (metrics.executionWallTimeInMs() != 0L) { - metricsBuilder.setExecutionWallTime(millisToProto(metrics.executionWallTimeInMs())); - } - if (metrics.processOutputsTimeInMs() != 0L) { - metricsBuilder.setProcessOutputsTime(millisToProto(metrics.processOutputsTimeInMs())); - } - if (metrics.retryTimeInMs() != 0L) { - metricsBuilder.setRetryTime(millisToProto(metrics.retryTimeInMs())); - } - metricsBuilder.setInputBytes(metrics.inputBytes()); - metricsBuilder.setInputFiles(metrics.inputFiles()); - metricsBuilder.setMemoryEstimateBytes(metrics.memoryEstimate()); - metricsBuilder.setInputBytesLimit(metrics.inputBytesLimit()); - metricsBuilder.setInputFilesLimit(metrics.inputFilesLimit()); - metricsBuilder.setOutputBytesLimit(metrics.outputBytesLimit()); - metricsBuilder.setOutputFilesLimit(metrics.outputFilesLimit()); - metricsBuilder.setMemoryBytesLimit(metrics.memoryLimit()); - if (metrics.timeLimitInMs() != 0L) { - metricsBuilder.setTimeLimit(millisToProto(metrics.timeLimitInMs())); - } - - try (SilentCloseable c = Profiler.instance().profile("logSpawn/write")) { - executionLog.write(builder.build()); - } - } + throws IOException, ExecException; - @VisibleForTesting - static com.google.protobuf.Duration millisToProto(int t) { - return Durations.fromMillis(t); - } - - public void close() throws IOException { - executionLog.close(); - } - - private static Protos.Platform buildPlatform(Platform platform) { - Protos.Platform.Builder platformBuilder = Protos.Platform.newBuilder(); - for (Platform.Property p : platform.getPropertiesList()) { - platformBuilder.addPropertiesBuilder().setName(p.getName()).setValue(p.getValue()); - } - return platformBuilder.build(); - } - - private SortedMap listExistingOutputs(Spawn spawn, FileSystem fileSystem) { - TreeMap result = new TreeMap<>(); - for (ActionInput output : spawn.getOutputFiles()) { - Path outputPath = fileSystem.getPath(execRoot.getRelative(output.getExecPathString())); - // TODO(olaola): once symlink API proposal is implemented, report symlinks here. - if (outputPath.exists()) { - result.put(outputPath, output); - } - } - return result; - } - - private void listDirectoryContents( - Path path, Consumer addFile, InputMetadataProvider inputMetadataProvider) { - try { - // TODO(olaola): once symlink API proposal is implemented, report symlinks here. - List sortedDirent = new ArrayList<>(path.readdir(Symlinks.NOFOLLOW)); - sortedDirent.sort(Comparator.comparing(Dirent::getName)); - for (Dirent dirent : sortedDirent) { - String name = dirent.getName(); - Path child = path.getRelative(name); - if (dirent.getType() == Dirent.Type.DIRECTORY) { - listDirectoryContents(child, addFile, inputMetadataProvider); - } else { - String pathString; - if (child.startsWith(execRoot)) { - pathString = child.asFragment().relativeTo(execRoot).getPathString(); - } else { - pathString = child.getPathString(); - } - addFile.accept( - File.newBuilder() - .setPath(pathString) - .setDigest(computeDigest(null, child, inputMetadataProvider, xattrProvider)) - .build()); - } - } - } catch (IOException e) { - logger.atWarning().withCause(e).log("Error computing spawn event file properties"); - } - } + /** Finishes writing the log and performs any required post-processing. */ + void close() throws IOException; /** - * Computes the digest of the given ActionInput or corresponding path. Will try to access the - * Metadata cache first, if it is available, and fall back to digesting the contents manually. + * Computes the digest of an ActionInput or its path. + * + *

Will try to obtain the digest from cached metadata first, falling back to digesting the + * contents manually. */ - private Digest computeDigest( + static Digest computeDigest( @Nullable ActionInput input, Path path, InputMetadataProvider inputMetadataProvider, - XattrProvider xattrProvider) + XattrProvider xattrProvider, + DigestHashFunction digestHashFunction) throws IOException { Digest.Builder builder = Digest.newBuilder().setHashFunctionName(digestHashFunction.toString()); @@ -353,7 +117,55 @@ private Digest computeDigest( .build(); } - public boolean shouldSort() { - return executionOptions.executionLogSort; + static Protos.SpawnMetrics getSpawnMetricsProto(SpawnResult result) { + SpawnMetrics metrics = result.getMetrics(); + Protos.SpawnMetrics.Builder builder = Protos.SpawnMetrics.newBuilder(); + if (metrics.totalTimeInMs() != 0L) { + builder.setTotalTime(millisToProto(metrics.totalTimeInMs())); + } + if (metrics.parseTimeInMs() != 0L) { + builder.setParseTime(millisToProto(metrics.parseTimeInMs())); + } + if (metrics.networkTimeInMs() != 0L) { + builder.setNetworkTime(millisToProto(metrics.networkTimeInMs())); + } + if (metrics.fetchTimeInMs() != 0L) { + builder.setFetchTime(millisToProto(metrics.fetchTimeInMs())); + } + if (metrics.queueTimeInMs() != 0L) { + builder.setQueueTime(millisToProto(metrics.queueTimeInMs())); + } + if (metrics.setupTimeInMs() != 0L) { + builder.setSetupTime(millisToProto(metrics.setupTimeInMs())); + } + if (metrics.uploadTimeInMs() != 0L) { + builder.setUploadTime(millisToProto(metrics.uploadTimeInMs())); + } + if (metrics.executionWallTimeInMs() != 0L) { + builder.setExecutionWallTime(millisToProto(metrics.executionWallTimeInMs())); + } + if (metrics.processOutputsTimeInMs() != 0L) { + builder.setProcessOutputsTime(millisToProto(metrics.processOutputsTimeInMs())); + } + if (metrics.retryTimeInMs() != 0L) { + builder.setRetryTime(millisToProto(metrics.retryTimeInMs())); + } + builder.setInputBytes(metrics.inputBytes()); + builder.setInputFiles(metrics.inputFiles()); + builder.setMemoryEstimateBytes(metrics.memoryEstimate()); + builder.setInputBytesLimit(metrics.inputBytesLimit()); + builder.setInputFilesLimit(metrics.inputFilesLimit()); + builder.setOutputBytesLimit(metrics.outputBytesLimit()); + builder.setOutputFilesLimit(metrics.outputFilesLimit()); + builder.setMemoryBytesLimit(metrics.memoryLimit()); + if (metrics.timeLimitInMs() != 0L) { + builder.setTimeLimit(millisToProto(metrics.timeLimitInMs())); + } + return builder.build(); + } + + @VisibleForTesting + static com.google.protobuf.Duration millisToProto(int t) { + return Durations.fromMillis(t); } } diff --git a/src/test/java/com/google/devtools/build/lib/exec/SpawnLogContextTest.java b/src/test/java/com/google/devtools/build/lib/exec/ExpandedSpawnLogContextTest.java similarity index 70% rename from src/test/java/com/google/devtools/build/lib/exec/SpawnLogContextTest.java rename to src/test/java/com/google/devtools/build/lib/exec/ExpandedSpawnLogContextTest.java index 9cfd8d6e110028..a5dc8c2c8e92ee 100644 --- a/src/test/java/com/google/devtools/build/lib/exec/SpawnLogContextTest.java +++ b/src/test/java/com/google/devtools/build/lib/exec/ExpandedSpawnLogContextTest.java @@ -14,9 +14,9 @@ package com.google.devtools.build.lib.exec; import static com.google.common.base.Preconditions.checkState; +import static com.google.common.truth.Truth.assertThat; import static com.google.devtools.build.lib.exec.SpawnLogContext.millisToProto; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.mockito.Mockito.verify; import com.google.common.base.Utf8; import com.google.common.collect.ImmutableList; @@ -38,6 +38,7 @@ import com.google.devtools.build.lib.actions.StaticInputMetadataProvider; import com.google.devtools.build.lib.actions.cache.VirtualActionInput; import com.google.devtools.build.lib.actions.util.ActionsTestUtil; +import com.google.devtools.build.lib.exec.ExpandedSpawnLogContext.Encoding; import com.google.devtools.build.lib.exec.Protos.Digest; import com.google.devtools.build.lib.exec.Protos.EnvironmentVariable; import com.google.devtools.build.lib.exec.Protos.File; @@ -48,7 +49,6 @@ import com.google.devtools.build.lib.server.FailureDetails.Crash; import com.google.devtools.build.lib.server.FailureDetails.FailureDetail; import com.google.devtools.build.lib.skyframe.TreeArtifactValue; -import com.google.devtools.build.lib.util.io.MessageOutputStream; import com.google.devtools.build.lib.vfs.DelegateFileSystem; import com.google.devtools.build.lib.vfs.DigestHashFunction; import com.google.devtools.build.lib.vfs.Dirent; @@ -63,22 +63,17 @@ import com.google.testing.junit.testparameterinjector.TestParameter; import com.google.testing.junit.testparameterinjector.TestParameterInjector; import java.io.IOException; +import java.io.InputStream; import java.time.Duration; +import java.util.ArrayList; import java.util.Map; import java.util.SortedMap; -import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnit; -import org.mockito.junit.MockitoRule; -/** Tests for {@link SpawnLogContext}. */ +/** Tests for {@link ExpandedSpawnLogContext}. */ @RunWith(TestParameterInjector.class) -public final class SpawnLogContextTest { - @Rule public final MockitoRule mockito = MockitoJUnit.rule(); - @Mock private MessageOutputStream outputStream; - +public final class ExpandedSpawnLogContextTest { private final DigestHashFunction digestHashFunction = DigestHashFunction.SHA256; private final FileSystem fs = new InMemoryFileSystem(digestHashFunction); private final Path execRoot = fs.getPath("/execroot"); @@ -86,6 +81,9 @@ public final class SpawnLogContextTest { private final ArtifactRoot outputDir = ArtifactRoot.asDerivedRoot(execRoot, RootType.Output, "out"); + private final Path logPath = fs.getPath("/log"); + private final Path tempPath = fs.getPath("/temp"); + // A fake action filesystem that provides a fast digest, but refuses to compute it from the // file contents (which won't be available when building without the bytes). private static final class FakeActionFileSystem extends DelegateFileSystem { @@ -145,9 +143,9 @@ public void testFileInput(@TestParameter InputsMode inputsMode) throws Exception spawn.withTools(fileInput); } - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( spawn.build(), createInputMetadataProvider(fileInput), createInputMap(fileInput), @@ -155,15 +153,15 @@ public void testFileInput(@TestParameter InputsMode inputsMode) throws Exception defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addInputs( - File.newBuilder() - .setPath("file") - .setDigest(getDigest("abc")) - .setIsTool(inputsMode.isTool())) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addInputs( + File.newBuilder() + .setPath("file") + .setDigest(getDigest("abc")) + .setIsTool(inputsMode.isTool())) + .build()); } @Test @@ -182,9 +180,9 @@ public void testDirectoryInput( spawn.withTools(dirInput); } - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( spawn.build(), createInputMetadataProvider(dirInput), createInputMap(dirInput), @@ -193,18 +191,15 @@ public void testDirectoryInput( defaultSpawnResult()); // TODO(tjgq): Propagate tool bit to files inside source directories. - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addAllInputs( - dirContents.isEmpty() - ? ImmutableList.of() - : ImmutableList.of( - File.newBuilder() - .setPath("dir/file") - .setDigest(getDigest("abc")) - .build())) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addAllInputs( + dirContents.isEmpty() + ? ImmutableList.of() + : ImmutableList.of( + File.newBuilder().setPath("dir/file").setDigest(getDigest("abc")).build())) + .build()); } @Test @@ -224,9 +219,9 @@ public void testTreeInput( spawn.withTools(treeInput); } - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( spawn.build(), createInputMetadataProvider(treeInput), createInputMap(treeInput), @@ -234,19 +229,19 @@ public void testTreeInput( defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addAllInputs( - dirContents.isEmpty() - ? ImmutableList.of() - : ImmutableList.of( - File.newBuilder() - .setPath("out/tree/child") - .setDigest(getDigest("abc")) - .setIsTool(inputsMode.isTool()) - .build())) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addAllInputs( + dirContents.isEmpty() + ? ImmutableList.of() + : ImmutableList.of( + File.newBuilder() + .setPath("out/tree/child") + .setDigest(getDigest("abc")) + .setIsTool(inputsMode.isTool()) + .build())) + .build()); } @Test @@ -255,9 +250,9 @@ public void testRunfilesInput() throws Exception { writeFile(runfilesInput, "abc"); - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( // In reality, the spawn would have a RunfilesSupplier and a runfiles middleman input. defaultSpawn(), createInputMetadataProvider(runfilesInput), @@ -268,11 +263,11 @@ public void testRunfilesInput() throws Exception { defaultSpawnResult()); // TODO(tjgq): The path should be foo.runfiles/data.txt. - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addInputs(File.newBuilder().setPath("data.txt").setDigest(getDigest("abc"))) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addInputs(File.newBuilder().setPath("data.txt").setDigest(getDigest("abc"))) + .build()); } @Test @@ -283,9 +278,9 @@ public void testAbsolutePathInput() throws Exception { writeFile(absolutePath, "abc"); - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( defaultSpawn(), new StaticInputMetadataProvider( ImmutableMap.of(absolutePathInput, FileArtifactValue.createForTesting(absolutePath))), @@ -294,18 +289,18 @@ public void testAbsolutePathInput() throws Exception { defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addInputs(File.newBuilder().setPath("/some/file.txt").setDigest(getDigest("abc"))) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addInputs(File.newBuilder().setPath("/some/file.txt").setDigest(getDigest("abc"))) + .build()); } @Test public void testEmptyInput() throws Exception { - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( defaultSpawn(), createInputMetadataProvider(), /* inputMap= */ ImmutableSortedMap.of( @@ -315,7 +310,7 @@ public void testEmptyInput() throws Exception { defaultSpawnResult()); // TODO(tjgq): It would make more sense to report an empty file. - verify(outputStream).write(defaultSpawnExec()); + closeAndAssertLog(context, defaultSpawnExec()); } @Test @@ -326,9 +321,9 @@ public void testFileOutput(@TestParameter OutputsMode outputsMode) throws Except Spawn spawn = defaultSpawnBuilder().withOutputs(fileOutput).build(); - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( spawn, createInputMetadataProvider(), createInputMap(), @@ -336,12 +331,12 @@ public void testFileOutput(@TestParameter OutputsMode outputsMode) throws Except defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addListedOutputs("out/file") - .addActualOutputs(File.newBuilder().setPath("out/file").setDigest(getDigest("abc"))) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addListedOutputs("out/file") + .addActualOutputs(File.newBuilder().setPath("out/file").setDigest(getDigest("abc"))) + .build()); } @Test @@ -357,9 +352,9 @@ public void testDirectoryOutput( SpawnBuilder spawn = defaultSpawnBuilder().withOutputs(dirOutput); - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( spawn.build(), createInputMetadataProvider(), createInputMap(), @@ -367,19 +362,19 @@ public void testDirectoryOutput( defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addListedOutputs("out/dir") - .addAllActualOutputs( - dirContents.isEmpty() - ? ImmutableList.of() - : ImmutableList.of( - File.newBuilder() - .setPath("out/dir/file") - .setDigest(getDigest("abc")) - .build())) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addListedOutputs("out/dir") + .addAllActualOutputs( + dirContents.isEmpty() + ? ImmutableList.of() + : ImmutableList.of( + File.newBuilder() + .setPath("out/dir/file") + .setDigest(getDigest("abc")) + .build())) + .build()); } @Test @@ -396,9 +391,9 @@ public void testTreeOutput( Spawn spawn = defaultSpawnBuilder().withOutputs(treeOutput).build(); - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( spawn, createInputMetadataProvider(), createInputMap(), @@ -406,19 +401,19 @@ public void testTreeOutput( defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addListedOutputs("out/tree") - .addAllActualOutputs( - dirContents.isEmpty() - ? ImmutableList.of() - : ImmutableList.of( - File.newBuilder() - .setPath("out/tree/child") - .setDigest(getDigest("abc")) - .build())) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addListedOutputs("out/tree") + .addAllActualOutputs( + dirContents.isEmpty() + ? ImmutableList.of() + : ImmutableList.of( + File.newBuilder() + .setPath("out/tree/child") + .setDigest(getDigest("abc")) + .build())) + .build()); } @Test @@ -426,9 +421,9 @@ public void testEnvironment() throws Exception { Spawn spawn = defaultSpawnBuilder().withEnvironment("SPAM", "eggs").withEnvironment("FOO", "bar").build(); - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( spawn, createInputMetadataProvider(), createInputMap(), @@ -436,21 +431,21 @@ public void testEnvironment() throws Exception { defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .addEnvironmentVariables( - EnvironmentVariable.newBuilder().setName("FOO").setValue("bar")) - .addEnvironmentVariables( - EnvironmentVariable.newBuilder().setName("SPAM").setValue("eggs")) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .addEnvironmentVariables( + EnvironmentVariable.newBuilder().setName("FOO").setValue("bar")) + .addEnvironmentVariables( + EnvironmentVariable.newBuilder().setName("SPAM").setValue("eggs")) + .build()); } @Test public void testDefaultPlatformProperties() throws Exception { - SpawnLogContext spawnLogContext = createSpawnLogContext(ImmutableMap.of("a", "1", "b", "2")); + SpawnLogContext context = createSpawnLogContext(ImmutableMap.of("a", "1", "b", "2")); - spawnLogContext.logSpawn( + context.logSpawn( defaultSpawn(), createInputMetadataProvider(), createInputMap(), @@ -458,15 +453,15 @@ public void testDefaultPlatformProperties() throws Exception { defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .setPlatform( - Platform.newBuilder() - .addProperties(Platform.Property.newBuilder().setName("a").setValue("1")) - .addProperties(Platform.Property.newBuilder().setName("b").setValue("2")) - .build()) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties(Platform.Property.newBuilder().setName("a").setValue("1")) + .addProperties(Platform.Property.newBuilder().setName("b").setValue("2")) + .build()) + .build()); } @Test @@ -474,9 +469,9 @@ public void testSpawnPlatformProperties() throws Exception { Spawn spawn = defaultSpawnBuilder().withExecProperties(ImmutableMap.of("a", "3", "c", "4")).build(); - SpawnLogContext spawnLogContext = createSpawnLogContext(ImmutableMap.of("a", "1", "b", "2")); + SpawnLogContext context = createSpawnLogContext(ImmutableMap.of("a", "1", "b", "2")); - spawnLogContext.logSpawn( + context.logSpawn( spawn, createInputMetadataProvider(), createInputMap(), @@ -485,16 +480,16 @@ public void testSpawnPlatformProperties() throws Exception { defaultSpawnResult()); // The spawn properties should override the default properties. - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .setPlatform( - Platform.newBuilder() - .addProperties(Platform.Property.newBuilder().setName("a").setValue("3")) - .addProperties(Platform.Property.newBuilder().setName("b").setValue("2")) - .addProperties(Platform.Property.newBuilder().setName("c").setValue("4")) - .build()) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties(Platform.Property.newBuilder().setName("a").setValue("3")) + .addProperties(Platform.Property.newBuilder().setName("b").setValue("2")) + .addProperties(Platform.Property.newBuilder().setName("c").setValue("4")) + .build()) + .build()); } @Test @@ -503,9 +498,9 @@ public void testExecutionInfo( throws Exception { Spawn spawn = defaultSpawnBuilder().withExecutionInfo(requirement, "").build(); - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( spawn, createInputMetadataProvider(), createInputMap(), @@ -513,25 +508,25 @@ public void testExecutionInfo( defaultTimeout(), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .setRemotable(!requirement.equals("no-remote")) - .setCacheable(!requirement.equals("no-cache")) - .setRemoteCacheable( - !requirement.equals("no-cache") - && !requirement.equals("no-remote") - && !requirement.equals("no-remote-cache")) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .setRemotable(!requirement.equals("no-remote")) + .setCacheable(!requirement.equals("no-cache")) + .setRemoteCacheable( + !requirement.equals("no-cache") + && !requirement.equals("no-remote") + && !requirement.equals("no-remote-cache")) + .build()); } @Test public void testCacheHit() throws Exception { - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); SpawnResult result = defaultSpawnResultBuilder().setCacheHit(true).build(); - spawnLogContext.logSpawn( + context.logSpawn( defaultSpawn(), createInputMetadataProvider(), createInputMap(), @@ -539,16 +534,16 @@ public void testCacheHit() throws Exception { defaultTimeout(), result); - verify(outputStream).write(defaultSpawnExecBuilder().setCacheHit(true).build()); + closeAndAssertLog(context, defaultSpawnExecBuilder().setCacheHit(true).build()); } @Test public void testDigest() throws Exception { - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); SpawnResult result = defaultSpawnResultBuilder().setDigest(getDigest("abc")).build(); - spawnLogContext.logSpawn( + context.logSpawn( defaultSpawn(), createInputMetadataProvider(), createInputMap(), @@ -556,14 +551,14 @@ public void testDigest() throws Exception { defaultTimeout(), result); - verify(outputStream).write(defaultSpawnExecBuilder().setDigest(getDigest("abc")).build()); + closeAndAssertLog(context, defaultSpawnExecBuilder().setDigest(getDigest("abc")).build()); } @Test public void testTimeout() throws Exception { - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( defaultSpawn(), createInputMetadataProvider(), createInputMap(), @@ -571,18 +566,18 @@ public void testTimeout() throws Exception { /* timeout= */ Duration.ofSeconds(42), defaultSpawnResult()); - verify(outputStream) - .write( - defaultSpawnExecBuilder().setTimeoutMillis(Duration.ofSeconds(42).toMillis()).build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder().setTimeoutMillis(Duration.ofSeconds(42).toMillis()).build()); } @Test public void testSpawnMetrics() throws Exception { SpawnMetrics metrics = SpawnMetrics.Builder.forLocalExec().setTotalTimeInMs(1).build(); - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); - spawnLogContext.logSpawn( + context.logSpawn( defaultSpawn(), createInputMetadataProvider(), createInputMap(), @@ -590,16 +585,16 @@ public void testSpawnMetrics() throws Exception { defaultTimeout(), defaultSpawnResultBuilder().setSpawnMetrics(metrics).build()); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .setMetrics(Protos.SpawnMetrics.newBuilder().setTotalTime(millisToProto(1))) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .setMetrics(Protos.SpawnMetrics.newBuilder().setTotalTime(millisToProto(1))) + .build()); } @Test public void testStatus() throws Exception { - SpawnLogContext spawnLogContext = createSpawnLogContext(); + SpawnLogContext context = createSpawnLogContext(); // SpawnResult requires a non-zero exit code and non-null failure details when the status isn't // successful. @@ -614,7 +609,7 @@ public void testStatus() throws Exception { .build()) .build(); - spawnLogContext.logSpawn( + context.logSpawn( defaultSpawn(), createInputMetadataProvider(), createInputMap(), @@ -622,12 +617,12 @@ public void testStatus() throws Exception { defaultTimeout(), result); - verify(outputStream) - .write( - defaultSpawnExecBuilder() - .setExitCode(37) - .setStatus(Status.NON_ZERO_EXIT.toString()) - .build()); + closeAndAssertLog( + context, + defaultSpawnExecBuilder() + .setExitCode(37) + .setStatus(Status.NON_ZERO_EXIT.toString()) + .build()); } private static Duration defaultTimeout() { @@ -724,18 +719,21 @@ private static TreeArtifactValue createTreeArtifactValue(Artifact tree) throws E return builder.build(); } - private SpawnLogContext createSpawnLogContext() { + private SpawnLogContext createSpawnLogContext() throws IOException { return createSpawnLogContext(ImmutableSortedMap.of()); } - private SpawnLogContext createSpawnLogContext(ImmutableMap platformProperties) { + private SpawnLogContext createSpawnLogContext(ImmutableMap platformProperties) + throws IOException { RemoteOptions remoteOptions = Options.getDefaults(RemoteOptions.class); remoteOptions.remoteDefaultExecProperties = platformProperties.entrySet().asList(); - return new SpawnLogContext( + return new ExpandedSpawnLogContext( + logPath, + tempPath, + Encoding.BINARY, + /* sorted= */ false, execRoot.asFragment(), - outputStream, - Options.getDefaults(ExecutionOptions.class), remoteOptions, DigestHashFunction.SHA256, SyscallCache.NO_CACHE); @@ -757,4 +755,19 @@ private static void writeFile(Path path, String contents) throws IOException { path.getParentDirectory().createDirectoryAndParents(); FileSystemUtils.writeContent(path, UTF_8, contents); } + + private void closeAndAssertLog(SpawnLogContext context, SpawnExec... expected) + throws IOException { + context.close(); + + ArrayList actual = new ArrayList<>(); + try (InputStream in = logPath.getInputStream()) { + while (in.available() > 0) { + SpawnExec ex = SpawnExec.parseDelimitedFrom(in); + actual.add(ex); + } + } + + assertThat(actual).containsExactlyElementsIn(expected).inOrder(); + } }