From 537f6505a357aff955585204e3bd3ed2b9ebe5b8 Mon Sep 17 00:00:00 2001 From: Ed Schouten Date: Sat, 3 Dec 2022 22:35:01 +0100 Subject: [PATCH] Make Bazel more responsive and use less memory when --jobs is high When using Bazel in combination with a larger remote execution cluster, it's not uncommon to call it with something like --jobs=512. We have observed that this is currently problematic for a couple of reasons: 1. It causes Bazel to launch 512 local threads, each being responsible for running one action remotely. All of these local threads may spend a lot of time in buildRemoteAction(), generating input roots in the form of Merkle trees. As the local system tends to have fewer than 512 CPUs, all of these threads will unnecessarily compete with each other. One practical downside of that is that interrupting Bazel using ^C takes a very long time, as it first wants to complete the computation of all 512 Merkle trees. Let's put a semaphore in place, limiting the number of concurrent Merkle tree computations to the number of CPU cores available. 2. Related to the above, Bazel will end up keeping 512 Merkle trees in memory throughout all stages of execution. This makes sense, as we may get cache misses, requiring us to upload the input root afterwards. Or the execution of a remote action may fail, requiring us to upload the input root. That said, generally speaking these cases are fairly uncommon. Most builds have relatively high cache hit rates and execution retries only happen rarely. It is therefore not worth keeping these Merkle trees in memory constantly. We only need it when computing the action digest for GetActionResult(), and while uploading it into the CAS. 3. AbstractSpawnStrategy.getInputMapping() has some smartness to memoize its results. This makes a lot of sense for local execution, where the input mapping is used in a couple of places. For remote caching/execution it is not evident that this is a good idea. Assuming you end up having a remote cache hit, you don't need it. Let's make the memoization optional, only using it in cases where we do local execution (which may also happen when you get a cache miss when doing remote caching). Similar changes against Bazel 5.x have allowed me to successfully do builds of a large monorepo using --jobs=512 using the default heap size limits, whereas I would normally see occasional OOM behaviour when providing --host_jvm_args=-Xmx64g. --- .../build/lib/exec/AbstractSpawnStrategy.java | 44 +++-- .../devtools/build/lib/exec/SpawnRunner.java | 2 +- .../build/lib/remote/RemoteAction.java | 18 +- .../lib/remote/RemoteExecutionService.java | 170 +++++++++++------- .../build/lib/remote/RemoteSpawnCache.java | 2 +- .../build/lib/remote/RemoteSpawnRunner.java | 4 +- .../lib/remote/common/RemotePathResolver.java | 10 +- .../sandbox/DarwinSandboxedSpawnRunner.java | 2 +- .../sandbox/DockerSandboxedSpawnRunner.java | 2 +- .../sandbox/LinuxSandboxedSpawnRunner.java | 5 +- .../ProcessWrapperSandboxedSpawnRunner.java | 2 +- .../sandbox/WindowsSandboxedSpawnRunner.java | 2 +- .../build/lib/worker/WorkerSpawnRunner.java | 3 +- .../lib/exec/local/LocalSpawnRunnerTest.java | 3 +- .../remote/RemoteExecutionServiceTest.java | 26 +-- .../lib/remote/RemotePathResolverTest.java | 7 +- .../lib/remote/RemoteSpawnCacheTest.java | 3 +- .../util/FakeSpawnExecutionContext.java | 3 +- .../lib/sandbox/SpawnRunnerTestUtil.java | 3 +- 19 files changed, 175 insertions(+), 136 deletions(-) diff --git a/src/main/java/com/google/devtools/build/lib/exec/AbstractSpawnStrategy.java b/src/main/java/com/google/devtools/build/lib/exec/AbstractSpawnStrategy.java index 6f827296406fe7..c2fc0bcab93827 100644 --- a/src/main/java/com/google/devtools/build/lib/exec/AbstractSpawnStrategy.java +++ b/src/main/java/com/google/devtools/build/lib/exec/AbstractSpawnStrategy.java @@ -185,7 +185,8 @@ public ImmutableList exec( spawnLogContext.logSpawn( spawn, actionExecutionContext.getMetadataProvider(), - context.getInputMapping(PathFragment.EMPTY_FRAGMENT), + context.getInputMapping(PathFragment.EMPTY_FRAGMENT, + /* willAccessRepeatedly = */ false), context.getTimeout(), spawnResult); } catch (IOException | ForbiddenActionInputException e) { @@ -246,7 +247,9 @@ public ListenableFuture prefetchInputs() return actionExecutionContext .getActionInputPrefetcher() .prefetchFiles( - getInputMapping(PathFragment.EMPTY_FRAGMENT).values(), getMetadataProvider()); + getInputMapping(PathFragment.EMPTY_FRAGMENT, + /* willAccessRepeatedly = */ true).values(), + getMetadataProvider()); } return immediateVoidFuture(); @@ -306,22 +309,33 @@ public FileOutErr getFileOutErr() { } @Override - public SortedMap getInputMapping(PathFragment baseDirectory) + public SortedMap getInputMapping(PathFragment baseDirectory, + boolean willAccessRepeatedly) throws IOException, ForbiddenActionInputException { - if (lazyInputMapping == null || !inputMappingBaseDirectory.equals(baseDirectory)) { - try (SilentCloseable c = - Profiler.instance().profile("AbstractSpawnStrategy.getInputMapping")) { - inputMappingBaseDirectory = baseDirectory; - lazyInputMapping = - spawnInputExpander.getInputMapping( - spawn, - actionExecutionContext.getArtifactExpander(), - baseDirectory, - actionExecutionContext.getMetadataProvider()); - } + // Return previously computed copy if present. + if (lazyInputMapping != null && inputMappingBaseDirectory.equals(baseDirectory)) { + return lazyInputMapping; + } + + SortedMap inputMapping; + try (SilentCloseable c = + Profiler.instance().profile("AbstractSpawnStrategy.getInputMapping")) { + inputMapping = + spawnInputExpander.getInputMapping( + spawn, + actionExecutionContext.getArtifactExpander(), + baseDirectory, + actionExecutionContext.getMetadataProvider()); } - return lazyInputMapping; + // Don't cache the input mapping if it is unlikely that it is used again. + // This reduces memory usage in the case where remote caching/execution is + // used, and the expected cache hit rate is high. + if (willAccessRepeatedly) { + inputMappingBaseDirectory = baseDirectory; + lazyInputMapping = inputMapping; + } + return inputMapping; } @Override diff --git a/src/main/java/com/google/devtools/build/lib/exec/SpawnRunner.java b/src/main/java/com/google/devtools/build/lib/exec/SpawnRunner.java index 44bd3f38c44e35..e7c83bf3196dd6 100644 --- a/src/main/java/com/google/devtools/build/lib/exec/SpawnRunner.java +++ b/src/main/java/com/google/devtools/build/lib/exec/SpawnRunner.java @@ -250,7 +250,7 @@ void lockOutputFiles(int exitCode, String errorMessage, FileOutErr outErr) * mapping is used in a context where the directory relative to which the keys are interpreted * is not the same as the execroot. */ - SortedMap getInputMapping(PathFragment baseDirectory) + SortedMap getInputMapping(PathFragment baseDirectory, boolean willAccessRepeatedly) throws IOException, ForbiddenActionInputException; /** Reports a progress update to the Spawn strategy. */ diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteAction.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteAction.java index e680ea484d818c..7b3786311d3671 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/RemoteAction.java +++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteAction.java @@ -36,7 +36,8 @@ public class RemoteAction { private final SpawnExecutionContext spawnExecutionContext; private final RemoteActionExecutionContext remoteActionExecutionContext; private final RemotePathResolver remotePathResolver; - private final MerkleTree merkleTree; + private final long inputBytes; + private final long inputFiles; private final Digest commandHash; private final Command command; private final Action action; @@ -56,7 +57,8 @@ public class RemoteAction { this.spawnExecutionContext = spawnExecutionContext; this.remoteActionExecutionContext = remoteActionExecutionContext; this.remotePathResolver = remotePathResolver; - this.merkleTree = merkleTree; + this.inputBytes = merkleTree.getInputBytes(); + this.inputFiles = merkleTree.getInputFiles(); this.commandHash = commandHash; this.command = command; this.action = action; @@ -80,12 +82,12 @@ public Spawn getSpawn() { * Returns the sum of file sizes plus protobuf sizes used to represent the inputs of this action. */ public long getInputBytes() { - return merkleTree.getInputBytes(); + return inputBytes; } /** Returns the number of input files of this action. */ public long getInputFiles() { - return merkleTree.getInputFiles(); + return inputFiles; } /** Returns the id this is action. */ @@ -111,17 +113,13 @@ public Command getCommand() { return command; } - public MerkleTree getMerkleTree() { - return merkleTree; - } - /** * Returns a {@link SortedMap} which maps from input paths for remote action to {@link * ActionInput}. */ - public SortedMap getInputMap() + public SortedMap getInputMap(boolean willAccessRepeatedly) throws IOException, ForbiddenActionInputException { - return remotePathResolver.getInputMapping(spawnExecutionContext); + return remotePathResolver.getInputMapping(spawnExecutionContext, willAccessRepeatedly); } /** diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionService.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionService.java index ffa3871a369970..6aa0d5210bee60 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionService.java +++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteExecutionService.java @@ -126,6 +126,7 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.schedulers.Schedulers; import java.io.IOException; +import java.lang.Runtime; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -142,6 +143,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Executor; import java.util.concurrent.Phaser; +import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Predicate; import javax.annotation.Nullable; @@ -377,7 +379,8 @@ private MerkleTree buildInputMerkleTree( } return MerkleTree.merge(subMerkleTrees, digestUtil); } else { - SortedMap inputMap = remotePathResolver.getInputMapping(context); + SortedMap inputMap = + remotePathResolver.getInputMapping(context, /* willAccessRepeatedly = */ false); if (!outputDirMap.isEmpty()) { // The map returned by getInputMapping is mutable, but must not be mutated here as it is // shared with all other strategies. @@ -436,63 +439,89 @@ private static ByteString buildSalt(Spawn spawn) { return null; } + /** + * Semaphore for limiting the concurrent number of Merkle tree input roots we + * compute and keep in memory. + * + * When --jobs is set to a high value to let the remote execution service runs + * many actions in parallel, there is no point in letting the local system + * compute Merkle trees of input roots with the same amount of parallelism. + * Not only does this make Bazel feel sluggish and slow to respond to being + * interrupted, it causes it to exhaust memory. + * + * As there is no point in letting Merkle tree input root computation use a + * higher concurrency than the number of CPUs in the system, use a semaphore + * to limit the concurrency of buildRemoteAction(). + */ + private final Semaphore remoteActionBuildingSemaphore = + new Semaphore(Runtime.getRuntime().availableProcessors(), true); + + private ToolSignature getToolSignature(Spawn spawn, SpawnExecutionContext context) + throws IOException, ExecException, ForbiddenActionInputException, InterruptedException { + return remoteOptions.markToolInputs + && Spawns.supportsWorkers(spawn) + && !spawn.getToolFiles().isEmpty() + ? computePersistentWorkerSignature(spawn, context) + : null; + } + /** Creates a new {@link RemoteAction} instance from spawn. */ public RemoteAction buildRemoteAction(Spawn spawn, SpawnExecutionContext context) throws IOException, ExecException, ForbiddenActionInputException, InterruptedException { - ToolSignature toolSignature = - remoteOptions.markToolInputs - && Spawns.supportsWorkers(spawn) - && !spawn.getToolFiles().isEmpty() - ? computePersistentWorkerSignature(spawn, context) - : null; - final MerkleTree merkleTree = buildInputMerkleTree(spawn, context, toolSignature); - - // Get the remote platform properties. - Platform platform = PlatformUtils.getPlatformProto(spawn, remoteOptions); - if (toolSignature != null) { - platform = - PlatformUtils.getPlatformProto( - spawn, remoteOptions, ImmutableMap.of("persistentWorkerKey", toolSignature.key)); - } else { - platform = PlatformUtils.getPlatformProto(spawn, remoteOptions); - } - - Command command = - buildCommand( - spawn.getOutputFiles(), - spawn.getArguments(), - spawn.getEnvironment(), - platform, - remotePathResolver); - Digest commandHash = digestUtil.compute(command); - Action action = - Utils.buildAction( - commandHash, - merkleTree.getRootDigest(), - platform, - context.getTimeout(), - Spawns.mayBeCachedRemotely(spawn), - buildSalt(spawn)); - - ActionKey actionKey = digestUtil.computeActionKey(action); - - RequestMetadata metadata = - TracingMetadataUtils.buildMetadata( - buildRequestId, commandId, actionKey.getDigest().getHash(), spawn.getResourceOwner()); - RemoteActionExecutionContext remoteActionExecutionContext = - RemoteActionExecutionContext.create( - spawn, metadata, getWriteCachePolicy(spawn), getReadCachePolicy(spawn)); - - return new RemoteAction( - spawn, - context, - remoteActionExecutionContext, - remotePathResolver, - merkleTree, - commandHash, - command, - action, - actionKey); + remoteActionBuildingSemaphore.acquire(); + try { + ToolSignature toolSignature = getToolSignature(spawn, context); + final MerkleTree merkleTree = buildInputMerkleTree(spawn, context, toolSignature); + + // Get the remote platform properties. + Platform platform = PlatformUtils.getPlatformProto(spawn, remoteOptions); + if (toolSignature != null) { + platform = + PlatformUtils.getPlatformProto( + spawn, remoteOptions, ImmutableMap.of("persistentWorkerKey", toolSignature.key)); + } else { + platform = PlatformUtils.getPlatformProto(spawn, remoteOptions); + } + + Command command = + buildCommand( + spawn.getOutputFiles(), + spawn.getArguments(), + spawn.getEnvironment(), + platform, + remotePathResolver); + Digest commandHash = digestUtil.compute(command); + Action action = + Utils.buildAction( + commandHash, + merkleTree.getRootDigest(), + platform, + context.getTimeout(), + Spawns.mayBeCachedRemotely(spawn), + buildSalt(spawn)); + + ActionKey actionKey = digestUtil.computeActionKey(action); + + RequestMetadata metadata = + TracingMetadataUtils.buildMetadata( + buildRequestId, commandId, actionKey.getDigest().getHash(), spawn.getResourceOwner()); + RemoteActionExecutionContext remoteActionExecutionContext = + RemoteActionExecutionContext.create( + spawn, metadata, getWriteCachePolicy(spawn), getReadCachePolicy(spawn)); + + return new RemoteAction( + spawn, + context, + remoteActionExecutionContext, + remotePathResolver, + merkleTree, + commandHash, + command, + action, + actionKey); + } finally { + remoteActionBuildingSemaphore.release(); + } } @Nullable @@ -1338,7 +1367,7 @@ private void reportUploadError(Throwable error) { *

Must be called before calling {@link #executeRemotely}. */ public void uploadInputsIfNotPresent(RemoteAction action, boolean force) - throws IOException, InterruptedException { + throws IOException, ExecException, ForbiddenActionInputException, InterruptedException { checkState(!shutdown.get(), "shutdown"); checkState(mayBeExecutedRemotely(action.getSpawn()), "spawn can't be executed remotely"); @@ -1347,13 +1376,28 @@ public void uploadInputsIfNotPresent(RemoteAction action, boolean force) Map additionalInputs = Maps.newHashMapWithExpectedSize(2); additionalInputs.put(action.getActionKey().getDigest(), action.getAction()); additionalInputs.put(action.getCommandHash(), action.getCommand()); - remoteExecutionCache.ensureInputsPresent( - action - .getRemoteActionExecutionContext() - .withWriteCachePolicy(CachePolicy.REMOTE_CACHE_ONLY), // Only upload to remote cache - action.getMerkleTree(), - additionalInputs, - force); + + // As uploading depends on having the full input root in memory, limit + // concurrency. This prevents memory exhaustion. We assume that + // ensureInputsPresent() provides enough parallelism to saturate the + // network connection. + remoteActionBuildingSemaphore.acquire(); + try { + Spawn spawn = action.getSpawn(); + SpawnExecutionContext context = action.getSpawnExecutionContext(); + ToolSignature toolSignature = getToolSignature(spawn, context); + MerkleTree merkleTree = buildInputMerkleTree(spawn, context, toolSignature); + + remoteExecutionCache.ensureInputsPresent( + action + .getRemoteActionExecutionContext() + .withWriteCachePolicy(CachePolicy.REMOTE_CACHE_ONLY), // Only upload to remote cache + merkleTree, + additionalInputs, + force); + } finally { + remoteActionBuildingSemaphore.release(); + } } /** diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnCache.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnCache.java index 8da2b2b0455411..09d821a0108e11 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnCache.java +++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnCache.java @@ -202,7 +202,7 @@ public void close() {} private void checkForConcurrentModifications() throws IOException, ForbiddenActionInputException { - for (ActionInput input : action.getInputMap().values()) { + for (ActionInput input : action.getInputMap(true).values()) { if (input instanceof VirtualActionInput) { continue; } diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnRunner.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnRunner.java index aa2557d3770dd5..49f2b6b73295ae 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnRunner.java +++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnRunner.java @@ -621,9 +621,9 @@ private Map getInputCtimes(SortedMap inpu SpawnResult execLocallyAndUpload( RemoteAction action, Spawn spawn, SpawnExecutionContext context, boolean uploadLocalResults) throws ExecException, IOException, ForbiddenActionInputException, InterruptedException { - Map ctimesBefore = getInputCtimes(action.getInputMap()); + Map ctimesBefore = getInputCtimes(action.getInputMap(true)); SpawnResult result = execLocally(spawn, context); - Map ctimesAfter = getInputCtimes(action.getInputMap()); + Map ctimesAfter = getInputCtimes(action.getInputMap(true)); uploadLocalResults = uploadLocalResults && Status.SUCCESS.equals(result.status()) && result.exitCode() == 0; if (!uploadLocalResults) { diff --git a/src/main/java/com/google/devtools/build/lib/remote/common/RemotePathResolver.java b/src/main/java/com/google/devtools/build/lib/remote/common/RemotePathResolver.java index d2c1c2c5448a38..d37d2a0c9b1b0f 100644 --- a/src/main/java/com/google/devtools/build/lib/remote/common/RemotePathResolver.java +++ b/src/main/java/com/google/devtools/build/lib/remote/common/RemotePathResolver.java @@ -42,7 +42,7 @@ public interface RemotePathResolver { * Returns a {@link SortedMap} which maps from input paths for remote action to {@link * ActionInput}. */ - SortedMap getInputMapping(SpawnExecutionContext context) + SortedMap getInputMapping(SpawnExecutionContext context, boolean willAccessRepeatedly) throws IOException, ForbiddenActionInputException; void walkInputs( @@ -100,9 +100,9 @@ public String getWorkingDirectory() { } @Override - public SortedMap getInputMapping(SpawnExecutionContext context) + public SortedMap getInputMapping(SpawnExecutionContext context, boolean willAccessRepeatedly) throws IOException, ForbiddenActionInputException { - return context.getInputMapping(PathFragment.EMPTY_FRAGMENT); + return context.getInputMapping(PathFragment.EMPTY_FRAGMENT, willAccessRepeatedly); } @Override @@ -171,12 +171,12 @@ public String getWorkingDirectory() { } @Override - public SortedMap getInputMapping(SpawnExecutionContext context) + public SortedMap getInputMapping(SpawnExecutionContext context, boolean willAccessRepeatedly) throws IOException, ForbiddenActionInputException { // The "root directory" of the action from the point of view of RBE is the parent directory of // the execroot locally. This is so that paths of artifacts in external repositories don't // start with an uplevel reference. - return context.getInputMapping(PathFragment.create(checkNotNull(getWorkingDirectory()))); + return context.getInputMapping(PathFragment.create(checkNotNull(getWorkingDirectory())), willAccessRepeatedly); } @Override diff --git a/src/main/java/com/google/devtools/build/lib/sandbox/DarwinSandboxedSpawnRunner.java b/src/main/java/com/google/devtools/build/lib/sandbox/DarwinSandboxedSpawnRunner.java index 831932b0dc4cc0..45a109ec6c076a 100644 --- a/src/main/java/com/google/devtools/build/lib/sandbox/DarwinSandboxedSpawnRunner.java +++ b/src/main/java/com/google/devtools/build/lib/sandbox/DarwinSandboxedSpawnRunner.java @@ -236,7 +236,7 @@ protected SandboxedSpawn prepareSpawn(Spawn spawn, SpawnExecutionContext context SandboxInputs inputs = helpers.processInputFiles( - context.getInputMapping(PathFragment.EMPTY_FRAGMENT), + context.getInputMapping(PathFragment.EMPTY_FRAGMENT, /* willAccessRepeatedly = */ true), execRoot, execRoot, packageRoots, diff --git a/src/main/java/com/google/devtools/build/lib/sandbox/DockerSandboxedSpawnRunner.java b/src/main/java/com/google/devtools/build/lib/sandbox/DockerSandboxedSpawnRunner.java index c9cd49fbb52318..423d2b76334a76 100644 --- a/src/main/java/com/google/devtools/build/lib/sandbox/DockerSandboxedSpawnRunner.java +++ b/src/main/java/com/google/devtools/build/lib/sandbox/DockerSandboxedSpawnRunner.java @@ -225,7 +225,7 @@ protected SandboxedSpawn prepareSpawn(Spawn spawn, SpawnExecutionContext context SandboxInputs inputs = helpers.processInputFiles( - context.getInputMapping(PathFragment.EMPTY_FRAGMENT), + context.getInputMapping(PathFragment.EMPTY_FRAGMENT, /* willAccessRepeatedly = */ true), execRoot, execRoot, packageRoots, diff --git a/src/main/java/com/google/devtools/build/lib/sandbox/LinuxSandboxedSpawnRunner.java b/src/main/java/com/google/devtools/build/lib/sandbox/LinuxSandboxedSpawnRunner.java index 95b7ace9991f22..4546d096d8b61c 100644 --- a/src/main/java/com/google/devtools/build/lib/sandbox/LinuxSandboxedSpawnRunner.java +++ b/src/main/java/com/google/devtools/build/lib/sandbox/LinuxSandboxedSpawnRunner.java @@ -299,7 +299,7 @@ protected SandboxedSpawn prepareSpawn(Spawn spawn, SpawnExecutionContext context SandboxInputs inputs = helpers.processInputFiles( - context.getInputMapping(PathFragment.EMPTY_FRAGMENT), + context.getInputMapping(PathFragment.EMPTY_FRAGMENT, /* willAccessRepeatedly = */ true), execRoot, withinSandboxExecRoot, packageRoots, @@ -598,7 +598,8 @@ public void verifyPostCondition( private void checkForConcurrentModifications(SpawnExecutionContext context) throws IOException, ForbiddenActionInputException { - for (ActionInput input : context.getInputMapping(PathFragment.EMPTY_FRAGMENT).values()) { + for (ActionInput input : context.getInputMapping(PathFragment.EMPTY_FRAGMENT, + /* willAccessRepeatedly = */ true).values()) { if (input instanceof VirtualActionInput) { // Virtual inputs are not existing in file system and can't be tampered with via sandbox. No // need to check them. diff --git a/src/main/java/com/google/devtools/build/lib/sandbox/ProcessWrapperSandboxedSpawnRunner.java b/src/main/java/com/google/devtools/build/lib/sandbox/ProcessWrapperSandboxedSpawnRunner.java index f82df416777660..1905e1c5ddb72a 100644 --- a/src/main/java/com/google/devtools/build/lib/sandbox/ProcessWrapperSandboxedSpawnRunner.java +++ b/src/main/java/com/google/devtools/build/lib/sandbox/ProcessWrapperSandboxedSpawnRunner.java @@ -115,7 +115,7 @@ protected SandboxedSpawn prepareSpawn(Spawn spawn, SpawnExecutionContext context SandboxInputs inputs = helpers.processInputFiles( - context.getInputMapping(PathFragment.EMPTY_FRAGMENT), + context.getInputMapping(PathFragment.EMPTY_FRAGMENT, /* willAccessRepeatedly = */ true), execRoot, execRoot, packageRoots, diff --git a/src/main/java/com/google/devtools/build/lib/sandbox/WindowsSandboxedSpawnRunner.java b/src/main/java/com/google/devtools/build/lib/sandbox/WindowsSandboxedSpawnRunner.java index 6b584f0c4b3a3b..abf62633463997 100644 --- a/src/main/java/com/google/devtools/build/lib/sandbox/WindowsSandboxedSpawnRunner.java +++ b/src/main/java/com/google/devtools/build/lib/sandbox/WindowsSandboxedSpawnRunner.java @@ -75,7 +75,7 @@ protected SandboxedSpawn prepareSpawn(Spawn spawn, SpawnExecutionContext context SandboxInputs readablePaths = helpers.processInputFiles( - context.getInputMapping(PathFragment.EMPTY_FRAGMENT), + context.getInputMapping(PathFragment.EMPTY_FRAGMENT, /* willAccessRepeatedly = */ true), execRoot, execRoot, packageRoots, diff --git a/src/main/java/com/google/devtools/build/lib/worker/WorkerSpawnRunner.java b/src/main/java/com/google/devtools/build/lib/worker/WorkerSpawnRunner.java index 0d1c14ee808b48..5cbcd4f5184ad7 100644 --- a/src/main/java/com/google/devtools/build/lib/worker/WorkerSpawnRunner.java +++ b/src/main/java/com/google/devtools/build/lib/worker/WorkerSpawnRunner.java @@ -192,7 +192,8 @@ public SpawnResult exec(Spawn spawn, SpawnExecutionContext context) Profiler.instance().profile(ProfilerTask.WORKER_SETUP, "Setting up inputs")) { inputFiles = helpers.processInputFiles( - context.getInputMapping(PathFragment.EMPTY_FRAGMENT), + context.getInputMapping(PathFragment.EMPTY_FRAGMENT, + /* willAccessRepeatedly = */ true), execRoot, execRoot, packageRoots, diff --git a/src/test/java/com/google/devtools/build/lib/exec/local/LocalSpawnRunnerTest.java b/src/test/java/com/google/devtools/build/lib/exec/local/LocalSpawnRunnerTest.java index 939fc951ef9d13..da987212571b0c 100644 --- a/src/test/java/com/google/devtools/build/lib/exec/local/LocalSpawnRunnerTest.java +++ b/src/test/java/com/google/devtools/build/lib/exec/local/LocalSpawnRunnerTest.java @@ -284,7 +284,8 @@ public FileOutErr getFileOutErr() { } @Override - public SortedMap getInputMapping(PathFragment baseDirectory) { + public SortedMap getInputMapping(PathFragment baseDirectory, + boolean willAccessRepeatedly) { return inputMapping; } diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemoteExecutionServiceTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemoteExecutionServiceTest.java index 1808b86d66c5ef..4efb46e2879330 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/RemoteExecutionServiceTest.java +++ b/src/test/java/com/google/devtools/build/lib/remote/RemoteExecutionServiceTest.java @@ -1701,7 +1701,7 @@ public void uploadInputsIfNotPresent_interrupted_requestCancelled() throws Excep service.uploadInputsIfNotPresent(action, /*force=*/ false); } catch (InterruptedException ignored) { interrupted.countDown(); - } catch (IOException ignored) { + } catch (Exception ignored) { // intentionally ignored } }); @@ -1834,30 +1834,6 @@ public void buildRemoteActionForRemotePersistentWorkers() throws Exception { .setValue( "b22d48cd55755474eae27e63a79306a64146bd5947d5bd3423d78f001cf7b3de")) .build()); - var merkleTree = remoteAction.getMerkleTree(); - var outputDirectory = - merkleTree.getDirectoryByDigest(merkleTree.getRootProto().getDirectories(0).getDigest()); - var inputFile = - FileNode.newBuilder() - .setName("input") - .setDigest( - Digest.newBuilder() - .setHash("cd42404d52ad55ccfa9aca4adc828aa5800ad9d385a0671fbcbf724118320619") - .setSizeBytes(5)) - .setIsExecutable(true); - var toolFile = - FileNode.newBuilder() - .setName("worker_input") - .setDigest( - Digest.newBuilder() - .setHash("bbd21d9e9b2bbadb2bb67202833df0edc8d14baf38be49388ffc71831eb88ac4") - .setSizeBytes(12)) - .setIsExecutable(true) - .setNodeProperties( - NodeProperties.newBuilder() - .addProperties(NodeProperty.newBuilder().setName("bazel_tool_input"))); - assertThat(outputDirectory) - .isEqualTo(Directory.newBuilder().addFiles(inputFile).addFiles(toolFile).build()); // Check that if an non-tool input changes, the persistent worker key does not change. fakeFileCache.createScratchInput(input, "value2"); diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemotePathResolverTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemotePathResolverTest.java index 3805649a1e0bab..2a735faaa6ee0e 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/RemotePathResolverTest.java +++ b/src/test/java/com/google/devtools/build/lib/remote/RemotePathResolverTest.java @@ -15,6 +15,7 @@ import static com.google.common.truth.Truth.assertThat; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -50,7 +51,7 @@ public void setup() throws Exception { input = ActionInputHelper.fromPath("foo"); spawnExecutionContext = mock(SpawnExecutionContext.class); - when(spawnExecutionContext.getInputMapping(any())) + when(spawnExecutionContext.getInputMapping(any(), anyBoolean())) .thenAnswer( invocationOnMock -> { PathFragment baseDirectory = invocationOnMock.getArgument(0); @@ -83,7 +84,7 @@ public void getInputMapping_default_inputsRelativeToExecRoot() throws Exception RemotePathResolver remotePathResolver = RemotePathResolver.createDefault(execRoot); SortedMap inputs = - remotePathResolver.getInputMapping(spawnExecutionContext); + remotePathResolver.getInputMapping(spawnExecutionContext, false); assertThat(inputs).containsExactly(PathFragment.create("foo"), input); } @@ -93,7 +94,7 @@ public void getInputMapping_sibling_inputsRelativeToInputRoot() throws Exception RemotePathResolver remotePathResolver = new SiblingRepositoryLayoutResolver(execRoot); SortedMap inputs = - remotePathResolver.getInputMapping(spawnExecutionContext); + remotePathResolver.getInputMapping(spawnExecutionContext, false); assertThat(inputs).containsExactly(PathFragment.create("main/foo"), input); } diff --git a/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnCacheTest.java b/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnCacheTest.java index 59f8b8ad4bc1cf..8ef831af7b9984 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnCacheTest.java +++ b/src/test/java/com/google/devtools/build/lib/remote/RemoteSpawnCacheTest.java @@ -172,7 +172,8 @@ public FileOutErr getFileOutErr() { } @Override - public SortedMap getInputMapping(PathFragment baseDirectory) + public SortedMap getInputMapping(PathFragment baseDirectory, + boolean willAccessRepeatedly) throws IOException, ForbiddenActionInputException { return getSpawnInputExpander() .getInputMapping(simpleSpawn, SIMPLE_ARTIFACT_EXPANDER, baseDirectory, fakeFileCache); diff --git a/src/test/java/com/google/devtools/build/lib/remote/util/FakeSpawnExecutionContext.java b/src/test/java/com/google/devtools/build/lib/remote/util/FakeSpawnExecutionContext.java index 3c25136a3b5509..0ee7d943b6841d 100644 --- a/src/test/java/com/google/devtools/build/lib/remote/util/FakeSpawnExecutionContext.java +++ b/src/test/java/com/google/devtools/build/lib/remote/util/FakeSpawnExecutionContext.java @@ -133,7 +133,8 @@ public FileOutErr getFileOutErr() { } @Override - public SortedMap getInputMapping(PathFragment baseDirectory) + public SortedMap getInputMapping(PathFragment baseDirectory, + boolean willAccessRepeatedly) throws IOException, ForbiddenActionInputException { return getSpawnInputExpander() .getInputMapping(spawn, this::artifactExpander, baseDirectory, metadataProvider); diff --git a/src/test/java/com/google/devtools/build/lib/sandbox/SpawnRunnerTestUtil.java b/src/test/java/com/google/devtools/build/lib/sandbox/SpawnRunnerTestUtil.java index 562d7eabb90aa0..01f703b06662da 100644 --- a/src/test/java/com/google/devtools/build/lib/sandbox/SpawnRunnerTestUtil.java +++ b/src/test/java/com/google/devtools/build/lib/sandbox/SpawnRunnerTestUtil.java @@ -131,7 +131,8 @@ public FileOutErr getFileOutErr() { } @Override - public SortedMap getInputMapping(PathFragment baseDirectory) { + public SortedMap getInputMapping(PathFragment baseDirectory, + boolean willAccessRepeatedly) { TreeMap inputMapping = new TreeMap<>(); for (ActionInput actionInput : spawn.getInputFiles().toList()) { inputMapping.put(baseDirectory.getRelative(actionInput.getExecPath()), actionInput);