Consolidates options used in BuildDriver#evaluate, MemoizingEvaluator#evaluate and WalkableGraphFactory#prepareAndGet
RELNOTES: None
PiperOrigin-RevId: 218728701
diff --git a/src/main/java/com/google/devtools/build/lib/bazel/commands/SyncCommand.java b/src/main/java/com/google/devtools/build/lib/bazel/commands/SyncCommand.java
index 1f4cf12..73cd826 100644
--- a/src/main/java/com/google/devtools/build/lib/bazel/commands/SyncCommand.java
+++ b/src/main/java/com/google/devtools/build/lib/bazel/commands/SyncCommand.java
@@ -35,6 +35,7 @@
import com.google.devtools.build.lib.util.AbruptExitException;
import com.google.devtools.build.lib.util.ExitCode;
import com.google.devtools.build.lib.vfs.RootedPath;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.SkyValue;
@@ -75,11 +76,13 @@
// Obtain the key for the top-level WORKSPACE file
SkyKey packageLookupKey = PackageLookupValue.key(Label.EXTERNAL_PACKAGE_IDENTIFIER);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(env.getReporter())
+ .build();
EvaluationResult<SkyValue> packageLookupValue =
- skyframeExecutor.prepareAndGet(
- ImmutableSet.of(packageLookupKey),
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- env.getReporter());
+ skyframeExecutor.prepareAndGet(ImmutableSet.of(packageLookupKey), evaluationContext);
if (packageLookupValue.hasError()) {
reportError(env, packageLookupValue);
return BlazeCommandResult.exitCode(ExitCode.ANALYSIS_FAILURE);
@@ -93,10 +96,7 @@
WorkspaceFileValue fileValue = null;
while (workspace != null) {
EvaluationResult<SkyValue> value =
- skyframeExecutor.prepareAndGet(
- ImmutableSet.of(workspace),
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- env.getReporter());
+ skyframeExecutor.prepareAndGet(ImmutableSet.of(workspace), evaluationContext);
if (value.hasError()) {
reportError(env, value);
return BlazeCommandResult.exitCode(ExitCode.ANALYSIS_FAILURE);
@@ -119,11 +119,7 @@
}
}
EvaluationResult<SkyValue> fetchValue;
- fetchValue =
- skyframeExecutor.prepareAndGet(
- repositoriesToFetch.build(),
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- env.getReporter());
+ fetchValue = skyframeExecutor.prepareAndGet(repositoriesToFetch.build(), evaluationContext);
if (fetchValue.hasError()) {
reportError(env, fetchValue);
return BlazeCommandResult.exitCode(ExitCode.ANALYSIS_FAILURE);
diff --git a/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java b/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java
index c81febc..7844078 100644
--- a/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java
+++ b/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java
@@ -96,6 +96,7 @@
import com.google.devtools.build.lib.vfs.Path;
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.RootedPath;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InterruptibleSupplier;
import com.google.devtools.build.skyframe.SkyFunctionName;
@@ -243,7 +244,12 @@
EvaluationResult<SkyValue> result;
try (AutoProfiler p = AutoProfiler.logged("evaluation and walkable graph", logger)) {
- result = graphFactory.prepareAndGet(roots, loadingPhaseThreads, universeEvalEventHandler);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setNumThreads(loadingPhaseThreads)
+ .setEventHander(universeEvalEventHandler)
+ .build();
+ result = graphFactory.prepareAndGet(roots, evaluationContext);
}
if (graph == null || graph != result.getWalkableGraph()) {
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutor.java b/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutor.java
index bcac1c4..d290b66 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutor.java
@@ -80,6 +80,7 @@
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.skyframe.BuildDriver;
import com.google.devtools.build.skyframe.Differencer;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.GraphInconsistencyReceiver;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.Injectable;
@@ -463,8 +464,13 @@
// Before running the FilesystemValueChecker, ensure that all values marked for invalidation
// have actually been invalidated (recall that invalidation happens at the beginning of the
// next evaluate() call), because checking those is a waste of time.
- buildDriver.evaluate(ImmutableList.<SkyKey>of(), false,
- DEFAULT_THREAD_COUNT, eventHandler);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(DEFAULT_THREAD_COUNT)
+ .setEventHander(eventHandler)
+ .build();
+ buildDriver.evaluate(ImmutableList.<SkyKey>of(), evaluationContext);
FilesystemValueChecker fsvc = new FilesystemValueChecker(tsgm, null);
// We need to manually check for changes to known files. This entails finding all dirty file
@@ -877,14 +883,20 @@
// Run the invalidator to actually delete the values.
try {
progressReceiver.ignoreInvalidations = true;
- Uninterruptibles.callUninterruptibly(new Callable<Void>() {
- @Override
- public Void call() throws InterruptedException {
- buildDriver.evaluate(ImmutableList.<SkyKey>of(), false,
- ResourceUsage.getAvailableProcessors(), eventHandler);
- return null;
- }
- });
+ Uninterruptibles.callUninterruptibly(
+ new Callable<Void>() {
+ @Override
+ public Void call() throws InterruptedException {
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(ResourceUsage.getAvailableProcessors())
+ .setEventHander(eventHandler)
+ .build();
+ buildDriver.evaluate(ImmutableList.<SkyKey>of(), evaluationContext);
+ return null;
+ }
+ });
} catch (Exception e) {
throw new IllegalStateException(e);
} finally {
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyFunctionEnvironmentForTesting.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyFunctionEnvironmentForTesting.java
index e4cd196..52da7f0 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyFunctionEnvironmentForTesting.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyFunctionEnvironmentForTesting.java
@@ -21,6 +21,7 @@
import com.google.devtools.build.lib.util.ResourceUsage;
import com.google.devtools.build.skyframe.AbstractSkyFunctionEnvironment;
import com.google.devtools.build.skyframe.BuildDriver;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyFunction;
import com.google.devtools.build.skyframe.SkyKey;
@@ -51,9 +52,15 @@
Iterable<? extends SkyKey> depKeys) throws InterruptedException {
ImmutableMap.Builder<SkyKey, ValueOrUntypedException> resultMap = ImmutableMap.builder();
Iterable<SkyKey> keysToEvaluate = ImmutableList.copyOf(depKeys);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(true)
+ .setNumThreads(ResourceUsage.getAvailableProcessors())
+ .setEventHander(eventHandler)
+ .build();
EvaluationResult<SkyValue> evaluationResult =
skyframeExecutor.evaluateSkyKeys(eventHandler, keysToEvaluate, true);
- buildDriver.evaluate(depKeys, true, ResourceUsage.getAvailableProcessors(), eventHandler);
+ buildDriver.evaluate(depKeys, evaluationContext);
for (SkyKey depKey : ImmutableSet.copyOf(depKeys)) {
resultMap.put(depKey, ValueOrUntypedException.ofValueUntyped(evaluationResult.get(depKey)));
}
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
index aa36349..f0647a3 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
@@ -152,6 +152,7 @@
import com.google.devtools.build.skyframe.Differencer;
import com.google.devtools.build.skyframe.Differencer.DiffWithDelta.Delta;
import com.google.devtools.build.skyframe.ErrorInfo;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationProgressReceiver;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.EventFilter;
@@ -670,8 +671,8 @@
// We evaluate in keepGoing mode because in the case that the graph does not store its
// edges, nokeepGoing builds are not allowed, whereas keepGoing builds are always
// permitted.
- EvaluationResult<SkyValue> result =
- buildDriver.evaluate(
+ EvaluationResult result =
+ evaluate(
ImmutableList.of(key), true, ResourceUsage.getAvailableProcessors(), eventHandler);
if (!result.hasError()) {
return Preconditions.checkNotNull(result.get(key), "%s %s", result, key);
@@ -965,7 +966,7 @@
throws InterruptedException {
// Should already be present, unless the user didn't request any targets for analysis.
EvaluationResult<WorkspaceStatusValue> result =
- buildDriver.evaluate(
+ evaluate(
ImmutableList.of(WorkspaceStatusValue.BUILD_INFO_KEY),
/*keepGoing=*/ true,
/*numThreads=*/ 1,
@@ -1002,10 +1003,14 @@
}
EvaluationResult<ContainingPackageLookupValue> result;
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(true)
+ .setNumThreads(1)
+ .setEventHander(eventHandler)
+ .build();
synchronized (valueLookupLock) {
- result =
- buildDriver.evaluate(
- packageKeys.values(), /*keepGoing=*/ true, /*numThreads=*/ 1, eventHandler);
+ result = buildDriver.evaluate(packageKeys.values(), evaluationContext);
}
if (result.hasError()) {
@@ -1376,11 +1381,14 @@
Iterable<SkyKey> testKeys =
TestCompletionValue.keys(
parallelTests, topLevelArtifactContext, /*exclusiveTesting=*/ false);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(options.getOptions(KeepGoingOption.class).keepGoing)
+ .setNumThreads(options.getOptions(BuildRequestOptions.class).getJobs())
+ .setEventHander(reporter)
+ .build();
return buildDriver.evaluate(
- Iterables.concat(artifactsToBuild, targetKeys, aspectKeys, testKeys),
- options.getOptions(KeepGoingOption.class).keepGoing,
- options.getOptions(BuildRequestOptions.class).getJobs(),
- reporter);
+ Iterables.concat(artifactsToBuild, targetKeys, aspectKeys, testKeys), evaluationContext);
} finally {
progressReceiver.executionProgressReceiver = null;
// Also releases thread locks.
@@ -1414,10 +1422,10 @@
Iterable<SkyKey> testKeys =
TestCompletionValue.keys(
ImmutableSet.of(exclusiveTest), topLevelArtifactContext, /*exclusiveTesting=*/ true);
- return buildDriver.evaluate(
+ return evaluate(
testKeys,
- options.getOptions(KeepGoingOption.class).keepGoing,
- options.getOptions(BuildRequestOptions.class).getJobs(),
+ /*keepGoing=*/ options.getOptions(KeepGoingOption.class).keepGoing,
+ /*numThreads=*/ options.getOptions(BuildRequestOptions.class).getJobs(),
reporter);
} finally {
progressReceiver.executionProgressReceiver = null;
@@ -1441,7 +1449,13 @@
ExtendedEventHandler eventHandler)
throws InterruptedException {
checkActive();
- return buildDriver.evaluate(patternSkyKeys, keepGoing, numThreads, eventHandler);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(numThreads)
+ .setEventHander(eventHandler)
+ .build();
+ return buildDriver.evaluate(patternSkyKeys, evaluationContext);
}
@VisibleForTesting
@@ -1850,19 +1864,22 @@
final boolean keepGoing) {
EvaluationResult<SkyValue> result;
try {
- result = callUninterruptibly(new Callable<EvaluationResult<SkyValue>>() {
- @Override
- public EvaluationResult<SkyValue> call() throws Exception {
- synchronized (valueLookupLock) {
- try {
- skyframeBuildView.enableAnalysis(true);
- return buildDriver.evaluate(skyKeys, keepGoing, DEFAULT_THREAD_COUNT, eventHandler);
- } finally {
- skyframeBuildView.enableAnalysis(false);
- }
- }
- }
- });
+ result =
+ callUninterruptibly(
+ new Callable<EvaluationResult<SkyValue>>() {
+ @Override
+ public EvaluationResult<SkyValue> call() throws Exception {
+ synchronized (valueLookupLock) {
+ try {
+ skyframeBuildView.enableAnalysis(true);
+ return evaluate(
+ skyKeys, keepGoing, /*numThreads=*/ DEFAULT_THREAD_COUNT, eventHandler);
+ } finally {
+ skyframeBuildView.enableAnalysis(false);
+ }
+ }
+ }
+ });
} catch (Exception e) {
throw new IllegalStateException(e); // Should never happen.
}
@@ -1882,8 +1899,14 @@
SkyKey key =
BuildConfigurationValue.key(
fragments, BuildOptions.diffForReconstruction(defaultBuildOptions, options));
- BuildConfigurationValue result = (BuildConfigurationValue) buildDriver
- .evaluate(ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, eventHandler).get(key);
+ BuildConfigurationValue result =
+ (BuildConfigurationValue)
+ evaluate(
+ ImmutableList.of(key),
+ /*keepGoing=*/ false,
+ /*numThreads=*/ DEFAULT_THREAD_COUNT,
+ eventHandler)
+ .get(key);
return result.getConfiguration();
}
@@ -1968,12 +1991,14 @@
keys.add(aspectKey);
}
eventHandler.post(new ConfigurationPhaseStartedEvent(configuredTargetProgress));
- EvaluationResult<ActionLookupValue> result =
- buildDriver.evaluate(
- keys,
- keepGoing,
- () -> NamedForkJoinPool.newNamedPool("skyframe-evaluator", numThreads),
- eventHandler);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setExecutorServiceSupplier(
+ () -> NamedForkJoinPool.newNamedPool("skyframe-evaluator", numThreads))
+ .setEventHander(eventHandler)
+ .build();
+ EvaluationResult<ActionLookupValue> result = buildDriver.evaluate(keys, evaluationContext);
// Get rid of any memory retained by the cache -- all loading is done.
perBuildSyscallCache.clear();
return result;
@@ -1993,8 +2018,11 @@
PrecomputedValue.BAD_ACTIONS.set(injectable(), badActions);
// Make sure to not run too many analysis threads. This can cause memory thrashing.
EvaluationResult<PostConfiguredTargetValue> result =
- buildDriver.evaluate(PostConfiguredTargetValue.keys(values), keepGoing,
- ResourceUsage.getAvailableProcessors(), eventHandler);
+ evaluate(
+ PostConfiguredTargetValue.keys(values),
+ keepGoing,
+ /*numThreads=*/ ResourceUsage.getAvailableProcessors(),
+ eventHandler);
// Remove all post-configured target values immediately for memory efficiency. We are OK with
// this mini-phase being non-incremental as the failure mode of action conflict is rare.
@@ -2024,7 +2052,13 @@
for (Label label : labelsToVisit) {
valueNames.add(TransitiveTargetKey.of(label));
}
- return buildDriver.evaluate(valueNames, keepGoing, parallelThreads, eventHandler);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(parallelThreads)
+ .setEventHander(eventHandler)
+ .build();
+ return buildDriver.evaluate(valueNames, evaluationContext);
}
}
@@ -2038,10 +2072,11 @@
*/
@Override
public EvaluationResult<SkyValue> prepareAndGet(
- Set<SkyKey> roots, int numThreads, ExtendedEventHandler eventHandler)
- throws InterruptedException {
+ Set<SkyKey> roots, EvaluationContext evaluationContext) throws InterruptedException {
EvaluationResult<SkyValue> evaluationResult =
- buildDriver.evaluate(roots, true, numThreads, eventHandler);
+ buildDriver.evaluate(
+ roots,
+ EvaluationContext.newBuilder().copyFrom(evaluationContext).setKeepGoing(true).build());
return evaluationResult;
}
@@ -2081,9 +2116,12 @@
// analysis) after a failed --nokeep_going analysis in which the configured target that
// failed was a (transitive) dependency of the configured target that should generate
// this action. We don't expect callers to query generating actions in such cases.
- EvaluationResult<ActionLookupValue> result = buildDriver.evaluate(
- ImmutableList.of(actionLookupKey), false, ResourceUsage.getAvailableProcessors(),
- eventHandler);
+ EvaluationResult<ActionLookupValue> result =
+ evaluate(
+ ImmutableList.of(actionLookupKey),
+ /*keepGoing=*/ false,
+ /*numThreads=*/ ResourceUsage.getAvailableProcessors(),
+ eventHandler);
return result.hasError()
? null
: result.get(actionLookupKey).getGeneratingActionDangerousReadJavadoc(artifact);
@@ -2145,8 +2183,11 @@
// keep_going build, since otherwise the build would have failed during loading. Thus
// we set keepGoing=true unconditionally.
EvaluationResult<PackageValue> result =
- buildDriver.evaluate(ImmutableList.of(key), /*keepGoing=*/true,
- DEFAULT_THREAD_COUNT, eventHandler);
+ evaluate(
+ ImmutableList.of(key),
+ /*keepGoing=*/ true,
+ /*numThreads=*/ DEFAULT_THREAD_COUNT,
+ eventHandler);
ErrorInfo error = result.getError(key);
if (error != null) {
if (!Iterables.isEmpty(error.getCycleInfo())) {
@@ -2358,9 +2399,7 @@
TestFilter.forOptions(options, eventHandler, pkgFactory.getRuleClassNames()));
EvaluationResult<TargetPatternPhaseValue> evalResult;
eventHandler.post(new LoadingPhaseStartedEvent(packageProgress));
- evalResult =
- buildDriver.evaluate(
- ImmutableList.of(key), keepGoing, threadCount, eventHandler);
+ evalResult = evaluate(ImmutableList.of(key), keepGoing, threadCount, eventHandler);
if (evalResult.hasError()) {
ErrorInfo errorInfo = evalResult.getError(key);
TargetParsingException exc;
@@ -2418,11 +2457,10 @@
BuildOptions.diffForReconstruction(defaultBuildOptions, buildOptions),
multiCpu,
labels);
- EvaluationResult<PrepareAnalysisPhaseValue> evalResult;
- evalResult =
- buildDriver.evaluate(
+ EvaluationResult<PrepareAnalysisPhaseValue> evalResult =
+ evaluate(
ImmutableList.of(key),
- /* keepGoing= */ true,
+ /*keepGoing=*/ true,
/*numThreads=*/ DEFAULT_THREAD_COUNT,
eventHandler);
if (evalResult.hasError()) {
@@ -2513,4 +2551,19 @@
modifiedFilesDuringPreviousBuild = 0;
return result;
}
+
+ private <T extends SkyValue> EvaluationResult<T> evaluate(
+ Iterable<? extends SkyKey> roots,
+ boolean keepGoing,
+ int numThreads,
+ ExtendedEventHandler eventHandler)
+ throws InterruptedException {
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(numThreads)
+ .setEventHander(eventHandler)
+ .build();
+ return buildDriver.evaluate(roots, evaluationContext);
+ }
}
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java b/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
index ba0d65e..6fac2b8 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
@@ -77,6 +77,7 @@
import com.google.devtools.build.skyframe.BuildDriver;
import com.google.devtools.build.skyframe.Differencer;
import com.google.devtools.build.skyframe.ErrorInfo;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationProgressReceiver;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.GraphInconsistencyReceiver;
@@ -305,8 +306,13 @@
keys.add(PackageValue.key(pkgId));
}
- EvaluationResult<PackageValue> evalResult =
- makeFreshDriver().evaluate(keys, /*keepGoing=*/ true, skyframeThreads, reporter);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(true)
+ .setNumThreads(skyframeThreads)
+ .setEventHander(reporter)
+ .build();
+ EvaluationResult<PackageValue> evalResult = makeFreshDriver().evaluate(keys, evaluationContext);
ImmutableMap.Builder<PackageIdentifier, PackageLoader.PackageOrException> result =
ImmutableMap.builder();
diff --git a/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java b/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java
index d624fcf..099d52a 100644
--- a/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java
+++ b/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java
@@ -14,11 +14,8 @@
package com.google.devtools.build.skyframe;
-import com.google.devtools.build.lib.events.ExtendedEventHandler;
import com.google.devtools.build.lib.util.AbruptExitException;
import com.google.devtools.common.options.OptionsProvider;
-import java.util.concurrent.ExecutorService;
-import java.util.function.Supplier;
import javax.annotation.Nullable;
/** A BuildDriver wraps a MemoizingEvaluator, passing along the proper Version. */
@@ -28,19 +25,7 @@
* of a {@link Version} value.
*/
<T extends SkyValue> EvaluationResult<T> evaluate(
- Iterable<? extends SkyKey> roots, boolean keepGoing, int numThreads,
- ExtendedEventHandler reporter)
- throws InterruptedException;
-
- /**
- * See {@link MemoizingEvaluator#evaluate}, which has the same semantics except for the inclusion
- * of a {@link Version} value.
- */
- <T extends SkyValue> EvaluationResult<T> evaluate(
- Iterable<? extends SkyKey> roots,
- boolean keepGoing,
- Supplier<ExecutorService> executorService,
- ExtendedEventHandler reporter)
+ Iterable<? extends SkyKey> roots, EvaluationContext evaluationContext)
throws InterruptedException;
/**
diff --git a/src/main/java/com/google/devtools/build/skyframe/EvaluationContext.java b/src/main/java/com/google/devtools/build/skyframe/EvaluationContext.java
new file mode 100644
index 0000000..a72baf3
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/EvaluationContext.java
@@ -0,0 +1,114 @@
+// Copyright 2018 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.skyframe;
+
+import com.google.common.base.Preconditions;
+import com.google.devtools.build.lib.events.ExtendedEventHandler;
+import com.google.devtools.build.skyframe.WalkableGraph.WalkableGraphFactory;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+
+/**
+ * Includes options and states used by {@link MemoizingEvaluator#evaluate}, {@link
+ * BuildDriver#evaluate} and {@link WalkableGraphFactory#prepareAndGet}
+ */
+public class EvaluationContext {
+ @Nullable private final Integer numThreads;
+ @Nullable private final Supplier<ExecutorService> executorService;
+ private final boolean keepGoing;
+ private final ExtendedEventHandler eventHandler;
+
+ protected EvaluationContext(
+ @Nullable Integer numThread,
+ @Nullable Supplier<ExecutorService> executorService,
+ boolean keepGoing,
+ ExtendedEventHandler eventHandler) {
+ this.numThreads = numThread;
+ this.executorService = executorService;
+ this.keepGoing = keepGoing;
+ this.eventHandler = eventHandler;
+ }
+
+ public Integer getNumThreads() {
+ return numThreads;
+ }
+
+ public Supplier<ExecutorService> getExecutorService() {
+ return executorService;
+ }
+
+ public boolean getKeepGoing() {
+ return keepGoing;
+ }
+
+ public ExtendedEventHandler getEventHandler() {
+ return eventHandler;
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /** Builder for {@link EvaluationContext}. */
+ public static class Builder {
+ private Integer numThread;
+ private Supplier<ExecutorService> executorService;
+ private boolean keepGoing;
+ private ExtendedEventHandler eventHandler;
+
+ private Builder() {}
+
+ public Builder copyFrom(EvaluationContext evaluationContext) {
+ this.numThread = evaluationContext.numThreads;
+ this.executorService = evaluationContext.executorService;
+ this.keepGoing = evaluationContext.keepGoing;
+ this.eventHandler = evaluationContext.eventHandler;
+ return this;
+ }
+
+ public Builder setNumThreads(int numThread) {
+ this.numThread = numThread;
+ this.executorService = null;
+ return this;
+ }
+
+ public Builder setExecutorServiceSupplier(Supplier<ExecutorService> executorService) {
+ this.executorService = executorService;
+ this.numThread = null;
+ return this;
+ }
+
+ public Builder setKeepGoing(boolean keepGoing) {
+ this.keepGoing = keepGoing;
+ return this;
+ }
+
+ public Builder setEventHander(ExtendedEventHandler eventHandler) {
+ this.eventHandler = eventHandler;
+ return this;
+ }
+
+ public EvaluationContext build() {
+ Preconditions.checkState(
+ (numThread == null && executorService != null)
+ || (numThread != null && executorService == null),
+ "Exactly one of numThread and executorService must be set. %s %s",
+ numThread,
+ executorService);
+ return new EvaluationContext(numThread, executorService, keepGoing, eventHandler);
+ }
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
index d85decd..bab9d3d 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
@@ -23,7 +23,6 @@
import com.google.common.collect.Sets;
import com.google.devtools.build.lib.concurrent.AbstractQueueVisitor;
import com.google.devtools.build.lib.events.Event;
-import com.google.devtools.build.lib.events.ExtendedEventHandler;
import com.google.devtools.build.lib.profiler.Profiler;
import com.google.devtools.build.lib.profiler.SilentCloseable;
import com.google.devtools.build.skyframe.Differencer.Diff;
@@ -37,9 +36,7 @@
import java.util.LinkedHashSet;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Supplier;
import javax.annotation.Nullable;
/**
@@ -154,27 +151,7 @@
@Override
public <T extends SkyValue> EvaluationResult<T> evaluate(
- Iterable<? extends SkyKey> roots,
- Version version,
- boolean keepGoing,
- int numThreads,
- ExtendedEventHandler eventHandler)
- throws InterruptedException {
- return evaluate(
- roots,
- version,
- keepGoing,
- () -> AbstractQueueVisitor.createExecutorService(numThreads),
- eventHandler);
- }
-
- @Override
- public <T extends SkyValue> EvaluationResult<T> evaluate(
- Iterable<? extends SkyKey> roots,
- Version version,
- boolean keepGoing,
- Supplier<ExecutorService> executorService,
- ExtendedEventHandler eventHandler)
+ Iterable<? extends SkyKey> roots, Version version, EvaluationContext evaluationContext)
throws InterruptedException {
// NOTE: Performance critical code. See bug "Null build performance parity".
IntVersion intVersion = (IntVersion) version;
@@ -210,14 +187,18 @@
graph,
version,
skyFunctions,
- eventHandler,
+ evaluationContext.getEventHandler(),
emittedEventState,
eventFilter,
ErrorInfoManager.UseChildErrorInfoIfNecessary.INSTANCE,
- keepGoing,
+ evaluationContext.getKeepGoing(),
progressReceiver,
graphInconsistencyReceiver,
- executorService,
+ evaluationContext.getExecutorService() == null
+ ? () ->
+ AbstractQueueVisitor.createExecutorService(
+ evaluationContext.getNumThreads())
+ : evaluationContext.getExecutorService(),
new SimpleCycleDetector(),
EvaluationVersionBehavior.MAX_CHILD_VERSIONS);
result = evaluator.eval(roots);
diff --git a/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
index c30c2c6..21a1032 100644
--- a/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
@@ -18,12 +18,9 @@
import com.google.common.collect.ImmutableMap;
import com.google.devtools.build.lib.collect.nestedset.NestedSetVisitor;
import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadHostile;
-import com.google.devtools.build.lib.events.ExtendedEventHandler;
import com.google.devtools.build.lib.events.ExtendedEventHandler.Postable;
import java.io.PrintStream;
import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.function.Supplier;
import javax.annotation.Nullable;
/**
@@ -48,28 +45,7 @@
* missing.
*/
<T extends SkyValue> EvaluationResult<T> evaluate(
- Iterable<? extends SkyKey> roots,
- Version version,
- boolean keepGoing,
- int numThreads,
- ExtendedEventHandler reporter)
- throws InterruptedException;
-
- /**
- * Computes the transitive closure of a given set of values at the given {@link Version}. See
- * {@link EagerInvalidator#invalidate}.
- *
- * <p>The returned EvaluationResult is guaranteed to contain a result for at least one root if
- * keepGoing is false. It will contain a result for every root if keepGoing is true, <i>unless</i>
- * the evaluation failed with a "catastrophic" error. In that case, some or all results may be
- * missing.
- */
- <T extends SkyValue> EvaluationResult<T> evaluate(
- Iterable<? extends SkyKey> roots,
- Version version,
- boolean keepGoing,
- Supplier<ExecutorService> executorService,
- ExtendedEventHandler reporter)
+ Iterable<? extends SkyKey> roots, Version version, EvaluationContext evaluationContext)
throws InterruptedException;
/**
diff --git a/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java b/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java
index 2947e29..f814f55 100644
--- a/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java
+++ b/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java
@@ -15,10 +15,7 @@
import com.google.common.base.Preconditions;
import com.google.devtools.build.lib.concurrent.AbstractQueueVisitor;
-import com.google.devtools.build.lib.events.ExtendedEventHandler;
import com.google.devtools.common.options.OptionsProvider;
-import java.util.concurrent.ExecutorService;
-import java.util.function.Supplier;
import javax.annotation.Nullable;
/**
@@ -35,24 +32,21 @@
@Override
public <T extends SkyValue> EvaluationResult<T> evaluate(
- Iterable<? extends SkyKey> roots,
- boolean keepGoing,
- int numThreads,
- ExtendedEventHandler reporter)
- throws InterruptedException {
- return evaluate(
- roots, keepGoing, () -> AbstractQueueVisitor.createExecutorService(numThreads), reporter);
- }
-
- @Override
- public <T extends SkyValue> EvaluationResult<T> evaluate(
- Iterable<? extends SkyKey> roots,
- boolean keepGoing,
- Supplier<ExecutorService> executorService,
- ExtendedEventHandler reporter)
+ Iterable<? extends SkyKey> roots, EvaluationContext evaluationContext)
throws InterruptedException {
try {
- return memoizingEvaluator.evaluate(roots, curVersion, keepGoing, executorService, reporter);
+ return memoizingEvaluator.evaluate(
+ roots,
+ curVersion,
+ evaluationContext.getExecutorService() == null
+ ? EvaluationContext.newBuilder()
+ .copyFrom(evaluationContext)
+ .setExecutorServiceSupplier(
+ () ->
+ AbstractQueueVisitor.createExecutorService(
+ evaluationContext.getNumThreads()))
+ .build()
+ : evaluationContext);
} finally {
curVersion = curVersion.next();
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/WalkableGraph.java b/src/main/java/com/google/devtools/build/skyframe/WalkableGraph.java
index 9110249..50e51ca 100644
--- a/src/main/java/com/google/devtools/build/skyframe/WalkableGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/WalkableGraph.java
@@ -14,7 +14,6 @@
package com.google.devtools.build.skyframe;
import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
-import com.google.devtools.build.lib.events.ExtendedEventHandler;
import java.util.Collection;
import java.util.Map;
import java.util.Set;
@@ -89,8 +88,7 @@
/** Provides a WalkableGraph on demand after preparing it. */
interface WalkableGraphFactory {
- EvaluationResult<SkyValue> prepareAndGet(
- Set<SkyKey> roots, int numThreads, ExtendedEventHandler eventHandler)
+ EvaluationResult<SkyValue> prepareAndGet(Set<SkyKey> roots, EvaluationContext evaluationContext)
throws InterruptedException;
/** Returns the {@link SkyKey} that defines this universe. */
diff --git a/src/test/java/com/google/devtools/build/lib/actions/util/ActionsTestUtil.java b/src/test/java/com/google/devtools/build/lib/actions/util/ActionsTestUtil.java
index 32caf5b..41e90a9 100644
--- a/src/test/java/com/google/devtools/build/lib/actions/util/ActionsTestUtil.java
+++ b/src/test/java/com/google/devtools/build/lib/actions/util/ActionsTestUtil.java
@@ -77,6 +77,7 @@
import com.google.devtools.build.skyframe.AbstractSkyFunctionEnvironment;
import com.google.devtools.build.skyframe.BuildDriver;
import com.google.devtools.build.skyframe.ErrorInfo;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyFunction;
import com.google.devtools.build.skyframe.SkyKey;
@@ -217,12 +218,13 @@
EvaluationResult<SkyValue> evaluationResult;
Map<SkyKey, ValueOrUntypedException> result = new HashMap<>();
try {
- evaluationResult =
- driver.evaluate(
- depKeys, /*keepGoing=*/
- false,
- ResourceUsage.getAvailableProcessors(),
- new Reporter(new EventBus(), eventHandler));
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(ResourceUsage.getAvailableProcessors())
+ .setEventHander(new Reporter(new EventBus(), eventHandler))
+ .build();
+ evaluationResult = driver.evaluate(depKeys, evaluationContext);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
for (SkyKey key : depKeys) {
diff --git a/src/test/java/com/google/devtools/build/lib/packages/TestTargetUtilsTest.java b/src/test/java/com/google/devtools/build/lib/packages/TestTargetUtilsTest.java
index 2b7a749..9e07eed 100644
--- a/src/test/java/com/google/devtools/build/lib/packages/TestTargetUtilsTest.java
+++ b/src/test/java/com/google/devtools/build/lib/packages/TestTargetUtilsTest.java
@@ -33,6 +33,7 @@
import com.google.devtools.build.lib.pkgcache.TestFilter;
import com.google.devtools.build.lib.skyframe.TestSuiteExpansionValue;
import com.google.devtools.build.lib.vfs.PathFragment;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyKey;
import java.util.Collection;
@@ -218,10 +219,16 @@
ImmutableSet.copyOf(Iterables.transform(expected, TO_LABEL));
ImmutableSet<Label> suiteLabels = ImmutableSet.copyOf(Iterables.transform(suites, TO_LABEL));
SkyKey key = TestSuiteExpansionValue.key(suiteLabels);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(1)
+ .setEventHander(reporter)
+ .build();
EvaluationResult<TestSuiteExpansionValue> result =
getSkyframeExecutor()
.getDriverForTesting()
- .evaluate(ImmutableList.of(key), false, 1, reporter);
+ .evaluate(ImmutableList.of(key), evaluationContext);
ResolvedTargets<Label> actual = result.get(key).getLabels();
assertThat(actual.hasError()).isFalse();
assertThat(actual.getTargets()).containsExactlyElementsIn(expectedLabels);
diff --git a/src/test/java/com/google/devtools/build/lib/pkgcache/IOExceptionsTest.java b/src/test/java/com/google/devtools/build/lib/pkgcache/IOExceptionsTest.java
index ef6cda8..fde1286 100644
--- a/src/test/java/com/google/devtools/build/lib/pkgcache/IOExceptionsTest.java
+++ b/src/test/java/com/google/devtools/build/lib/pkgcache/IOExceptionsTest.java
@@ -30,6 +30,7 @@
import com.google.devtools.build.lib.vfs.Path;
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.inmemoryfs.InMemoryFileSystem;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.SkyValue;
@@ -64,8 +65,10 @@
private boolean visitTransitively(Label label) throws InterruptedException {
SkyKey key = TransitiveTargetKey.of(label);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder().setNumThreads(5).setEventHander(reporter).build();
EvaluationResult<SkyValue> result =
- skyframeExecutor.prepareAndGet(ImmutableSet.of(key), /*numThreads=*/ 5, reporter);
+ skyframeExecutor.prepareAndGet(ImmutableSet.of(key), evaluationContext);
TransitiveTargetValue value = (TransitiveTargetValue) result.get(key);
System.out.println(value);
boolean hasTransitiveError = (value == null) || value.getTransitiveRootCauses() != null;
diff --git a/src/test/java/com/google/devtools/build/lib/repository/ExternalPackageUtilTest.java b/src/test/java/com/google/devtools/build/lib/repository/ExternalPackageUtilTest.java
index db6cc5e..01ef6a7 100644
--- a/src/test/java/com/google/devtools/build/lib/repository/ExternalPackageUtilTest.java
+++ b/src/test/java/com/google/devtools/build/lib/repository/ExternalPackageUtilTest.java
@@ -57,6 +57,7 @@
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
import com.google.devtools.build.skyframe.AbstractSkyKey;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -81,6 +82,12 @@
/** Unit tests for {@link ExternalPackageUtil}. */
@RunWith(JUnit4.class)
public class ExternalPackageUtilTest extends BuildViewTestCase {
+ private static final EvaluationContext EVALUATION_OPTIONS =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
private SequentialBuildDriver driver;
@@ -170,11 +177,7 @@
}
EvaluationResult<GetRuleByNameValue> getRuleByName(SkyKey key) throws InterruptedException {
- return driver.<GetRuleByNameValue>evaluate(
- ImmutableList.of(key),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ return driver.<GetRuleByNameValue>evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
}
@Test
@@ -212,11 +215,7 @@
EvaluationResult<GetRegisteredToolchainsValue> getRegisteredToolchains(SkyKey key)
throws InterruptedException {
- return driver.<GetRegisteredToolchainsValue>evaluate(
- ImmutableList.of(key),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ return driver.<GetRegisteredToolchainsValue>evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
}
@Test
@@ -238,10 +237,7 @@
EvaluationResult<GetRegisteredExecutionPlatformsValue> getRegisteredExecutionPlatforms(SkyKey key)
throws InterruptedException {
return driver.<GetRegisteredExecutionPlatformsValue>evaluate(
- ImmutableList.of(key),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ ImmutableList.of(key), EVALUATION_OPTIONS);
}
// HELPER SKYFUNCTIONS
diff --git a/src/test/java/com/google/devtools/build/lib/rules/repository/RepositoryDelegatorTest.java b/src/test/java/com/google/devtools/build/lib/rules/repository/RepositoryDelegatorTest.java
index 4551358..6ddf0c7 100644
--- a/src/test/java/com/google/devtools/build/lib/rules/repository/RepositoryDelegatorTest.java
+++ b/src/test/java/com/google/devtools/build/lib/rules/repository/RepositoryDelegatorTest.java
@@ -49,6 +49,7 @@
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -151,8 +152,13 @@
public void testOverride() throws Exception {
StoredEventHandler eventHandler = new StoredEventHandler();
SkyKey key = RepositoryDirectoryValue.key(RepositoryName.createFromValidStrippedName("foo"));
- EvaluationResult<SkyValue> result =
- driver.evaluate(ImmutableList.of(key), false, 8, eventHandler);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(8)
+ .setEventHander(eventHandler)
+ .build();
+ EvaluationResult<SkyValue> result = driver.evaluate(ImmutableList.of(key), evaluationContext);
assertThat(result.hasError()).isFalse();
RepositoryDirectoryValue repositoryDirectoryValue = (RepositoryDirectoryValue) result.get(key);
Path expectedPath = scratch.dir("/outputbase/external/foo");
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/ActionTemplateExpansionFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/ActionTemplateExpansionFunctionTest.java
index 3a21b58b..e1ecf96 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/ActionTemplateExpansionFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/ActionTemplateExpansionFunctionTest.java
@@ -49,6 +49,7 @@
import com.google.devtools.build.lib.testutil.FoundationTestCase;
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.Root;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -199,12 +200,14 @@
differencer.inject(CTKEY, ctValue);
ActionTemplateExpansionKey templateKey = ActionTemplateExpansionValue.key(CTKEY, 0);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
EvaluationResult<ActionTemplateExpansionValue> result =
- driver.evaluate(
- ImmutableList.of(templateKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(templateKey), evaluationContext);
if (result.hasError()) {
throw result.getError().getException();
}
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/ArtifactFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/ArtifactFunctionTest.java
index a57755b..d52d6f3 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/ArtifactFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/ArtifactFunctionTest.java
@@ -48,6 +48,7 @@
import com.google.devtools.build.lib.vfs.Path;
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.Root;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyFunction;
import com.google.devtools.build.skyframe.SkyKey;
@@ -390,11 +391,13 @@
private <E extends SkyValue> EvaluationResult<E> evaluate(SkyKey... keys)
throws InterruptedException, ActionConflictException {
setGeneratingActions();
- return driver.evaluate(
- Arrays.asList(keys),
- /*keepGoing=*/false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
+ return driver.evaluate(Arrays.asList(keys), evaluationContext);
}
/** Value Builder for actions that just stats and stores the output file (which must exist). */
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/ContainingPackageLookupFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/ContainingPackageLookupFunctionTest.java
index 9827103..c453aae 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/ContainingPackageLookupFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/ContainingPackageLookupFunctionTest.java
@@ -45,6 +45,7 @@
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
import com.google.devtools.build.skyframe.RecordingDifferencer;
@@ -169,12 +170,14 @@
private ContainingPackageLookupValue lookupContainingPackage(PackageIdentifier packageIdentifier)
throws InterruptedException {
SkyKey key = ContainingPackageLookupValue.key(packageIdentifier);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
return driver
- .<ContainingPackageLookupValue>evaluate(
- ImmutableList.of(key),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE)
+ .<ContainingPackageLookupValue>evaluate(ImmutableList.of(key), evaluationContext)
.get(key);
}
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/FileFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/FileFunctionTest.java
index 3fc2b7a..3982276 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/FileFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/FileFunctionTest.java
@@ -65,6 +65,7 @@
import com.google.devtools.build.skyframe.BuildDriver;
import com.google.devtools.build.skyframe.ErrorInfo;
import com.google.devtools.build.skyframe.ErrorInfoSubject;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -98,6 +99,13 @@
/** Tests for {@link FileFunction}. */
@RunWith(JUnit4.class)
public class FileFunctionTest {
+ private static final EvaluationContext EVALUATION_OPTIONS =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
+
private CustomInMemoryFs fs;
private Root pkgRoot;
private Path outputBase;
@@ -201,9 +209,7 @@
RootedPath rootedPath = RootedPath.toRootedPath(root, pathFragment);
SequentialBuildDriver driver = makeDriver();
SkyKey key = FileValue.key(rootedPath);
- EvaluationResult<FileValue> result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<FileValue> result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
assertThat(result.hasError()).isFalse();
return result.get(key);
}
@@ -648,9 +654,7 @@
SequentialBuildDriver driver = makeDriver();
SkyKey key = skyKey("/outsideroot");
EvaluationResult<SkyValue> result;
- result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
if (result.hasError()) {
fail(String.format("Evaluation error for %s: %s", key, result.getError()));
}
@@ -659,9 +663,7 @@
file.delete();
differencer.invalidate(ImmutableList.of(fileStateSkyKey("/outsideroot")));
- result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
if (result.hasError()) {
fail(String.format("Evaluation error for %s: %s", key, result.getError()));
}
@@ -677,9 +679,7 @@
SequentialBuildDriver driver =
makeDriver(ExternalFileAction.ASSUME_NON_EXISTENT_AND_IMMUTABLE_FOR_EXTERNAL_PATHS);
SkyKey key = skyKey("/outsideroot");
- EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<SkyValue> result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
assertThatEvaluationResult(result).hasNoError();
FileValue value = (FileValue) result.get(key);
@@ -696,9 +696,7 @@
SequentialBuildDriver driver =
makeDriver(ExternalFileAction.ASSUME_NON_EXISTENT_AND_IMMUTABLE_FOR_EXTERNAL_PATHS);
SkyKey key = skyKey("a");
- EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<SkyValue> result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
assertThatEvaluationResult(result).hasNoError();
FileValue value = (FileValue) result.get(key);
@@ -715,9 +713,7 @@
SequentialBuildDriver driver =
makeDriver(ExternalFileAction.ASSUME_NON_EXISTENT_AND_IMMUTABLE_FOR_EXTERNAL_PATHS);
SkyKey key = skyKey("/root/src/foo/bar");
- EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<SkyValue> result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
assertThatEvaluationResult(result).hasNoError();
FileValue value = (FileValue) result.get(key);
@@ -733,9 +729,7 @@
SequentialBuildDriver driver =
makeDriver(ExternalFileAction.ASSUME_NON_EXISTENT_AND_IMMUTABLE_FOR_EXTERNAL_PATHS);
SkyKey key = skyKey("a");
- EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<SkyValue> result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
assertThatEvaluationResult(result).hasNoError();
FileValue value = (FileValue) result.get(key);
@@ -751,9 +745,7 @@
SequentialBuildDriver driver =
makeDriver(ExternalFileAction.ASSUME_NON_EXISTENT_AND_IMMUTABLE_FOR_EXTERNAL_PATHS);
SkyKey key = skyKey("a");
- EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<SkyValue> result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
assertThatEvaluationResult(result).hasNoError();
FileValue value = (FileValue) result.get(key);
@@ -907,8 +899,7 @@
SequentialBuildDriver driver = makeDriver();
SkyKey skyKey = skyKey("a");
EvaluationResult<FileValue> result =
- driver.evaluate(
- ImmutableList.of(skyKey), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(skyKey), EVALUATION_OPTIONS);
assertThat(result.hasError()).isTrue();
ErrorInfo errorInfo = result.getError(skyKey);
assertThat(errorInfo.getException()).isInstanceOf(InconsistentFilesystemException.class);
@@ -935,8 +926,7 @@
SequentialBuildDriver driver = makeDriver();
SkyKey skyKey = skyKey("unreadable");
EvaluationResult<FileValue> result =
- driver.evaluate(
- ImmutableList.of(skyKey), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(skyKey), EVALUATION_OPTIONS);
assertThat(result.hasError()).isTrue();
ErrorInfo errorInfo = result.getError(skyKey);
assertThat(errorInfo.getException()).isInstanceOf(InconsistentFilesystemException.class);
@@ -1006,8 +996,13 @@
}
StoredEventHandler eventHandler = new StoredEventHandler();
SequentialBuildDriver driver = makeDriver();
- EvaluationResult<FileValue> result =
- driver.evaluate(keys, /*keepGoing=*/ true, DEFAULT_THREAD_COUNT, eventHandler);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(true)
+ .setNumThreads(DEFAULT_THREAD_COUNT)
+ .setEventHander(eventHandler)
+ .build();
+ EvaluationResult<FileValue> result = driver.evaluate(keys, evaluationContext);
assertThat(result.hasError()).isTrue();
for (SkyKey key : keys) {
ErrorInfo errorInfo = result.getError(key);
@@ -1108,8 +1103,7 @@
file, symlink1, symlink2, symlink3, missing1, missing2, directory1, directory2);
SequentialBuildDriver driver = makeDriver();
- EvaluationResult<SkyValue> result =
- driver.evaluate(keys, false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<SkyValue> result = driver.evaluate(keys, EVALUATION_OPTIONS);
new EqualsTester()
.addEqualityGroup(result.get(file))
@@ -1177,8 +1171,14 @@
expectedChain =
ImmutableList.of(ancestorRootedPath, intermediateRootedPath, descendantRootedPath);
}
- EvaluationResult<FileValue> result =
- driver.evaluate(keys, /*keepGoing=*/ true, DEFAULT_THREAD_COUNT, eventHandler);
+
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(true)
+ .setNumThreads(DEFAULT_THREAD_COUNT)
+ .setEventHander(eventHandler)
+ .build();
+ EvaluationResult<FileValue> result = driver.evaluate(keys, evaluationContext);
assertThat(result.hasError()).isTrue();
for (SkyKey key : errorKeys) {
ErrorInfo errorInfo = result.getError(key);
@@ -1230,12 +1230,14 @@
SkyKey fooKey = skyKey("foo");
fs.stubStatError(foo, new IOException("bork"));
BuildDriver driver = makeDriver();
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(true)
+ .setNumThreads(1)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
EvaluationResult<FileValue> result =
- driver.evaluate(
- ImmutableList.of(fooKey),
- /*keepGoing=*/ true,
- /*numThreads=*/ 1,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(fooKey), evaluationContext);
ErrorInfoSubject errorInfoSubject = assertThatEvaluationResult(result)
.hasErrorEntryForKeyThat(fooKey);
errorInfoSubject.isTransient();
@@ -1249,12 +1251,7 @@
FileStateValue.create(
RootedPath.toRootedPath(pkgRoot, foo),
new TimestampGranularityMonitor(BlazeClock.instance())));
- result =
- driver.evaluate(
- ImmutableList.of(fooKey),
- /*keepGoing=*/ true,
- /*numThreads=*/ 1,
- NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.of(fooKey), evaluationContext);
assertThatEvaluationResult(result).hasNoError();
assertThat(result.get(fooKey).exists()).isTrue();
}
@@ -1267,9 +1264,7 @@
private void assertRealPath(String pathString, String expectedRealPathString) throws Exception {
SequentialBuildDriver driver = makeDriver();
SkyKey key = skyKey(pathString);
- EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<SkyValue> result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
if (result.hasError()) {
fail(String.format("Evaluation error for %s: %s", key, result.getError()));
}
@@ -1476,9 +1471,7 @@
SequentialBuildDriver driver = makeDriver();
SkyKey key = skyKey(pathString);
EvaluationResult<SkyValue> result;
- result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
if (result.hasError()) {
fail(String.format("Evaluation error for %s: %s", key, result.getError()));
}
@@ -1497,9 +1490,7 @@
}
}));
- result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
if (result.hasError()) {
fail(String.format("Evaluation error for %s: %s", key, result.getError()));
}
@@ -1528,9 +1519,7 @@
SequentialBuildDriver driver = makeDriver();
SkyKey key = skyKey(pathString);
EvaluationResult<FileValue> result;
- result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
assertWithMessage(
"Did not expect error while evaluating " + pathString + ", got " + result.get(key))
.that(result.hasError())
@@ -1546,9 +1535,7 @@
SequentialBuildDriver driver = makeDriver();
SkyKey key = skyKey(pathString);
EvaluationResult<FileValue> result;
- result =
- driver.evaluate(
- ImmutableList.of(key), false, DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.of(key), EVALUATION_OPTIONS);
assertWithMessage("Expected error while evaluating " + pathString + ", got " + result.get(key))
.that(result.hasError())
.isTrue();
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/FilesetEntryFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/FilesetEntryFunctionTest.java
index f82b763..bcee6cc 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/FilesetEntryFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/FilesetEntryFunctionTest.java
@@ -49,6 +49,7 @@
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -176,11 +177,13 @@
}
private <T extends SkyValue> EvaluationResult<T> eval(SkyKey key) throws Exception {
- return driver.evaluate(
- ImmutableList.of(key),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
+ return driver.evaluate(ImmutableList.of(key), evaluationContext);
}
private FilesetEntryValue evalFilesetTraversal(FilesetTraversalParams params) throws Exception {
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/FilesystemValueCheckerTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/FilesystemValueCheckerTest.java
index f4dfa24..7211185 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/FilesystemValueCheckerTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/FilesystemValueCheckerTest.java
@@ -61,6 +61,7 @@
import com.google.devtools.build.lib.vfs.Symlinks;
import com.google.devtools.build.lib.vfs.inmemoryfs.InMemoryFileSystem;
import com.google.devtools.build.skyframe.Differencer.Diff;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -91,6 +92,12 @@
*/
@RunWith(JUnit4.class)
public class FilesystemValueCheckerTest {
+ private static final EvaluationContext EVALUATION_OPTIONS =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
private RecordingDifferencer differencer;
private MemoizingEvaluator evaluator;
@@ -176,11 +183,7 @@
FileStateValue.key(
RootedPath.toRootedPath(Root.absoluteRoot(fs), PathFragment.create("/foo")));
EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(skyKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(skyKey), EVALUATION_OPTIONS);
assertThat(result.hasError()).isFalse();
assertEmptyDiff(getDirtyFilesystemKeys(evaluator, checker));
@@ -192,12 +195,7 @@
assertDiffWithNewValues(getDirtyFilesystemKeys(evaluator, checker), skyKey);
differencer.invalidate(ImmutableList.of(skyKey));
- result =
- driver.evaluate(
- ImmutableList.of(skyKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.of(skyKey), EVALUATION_OPTIONS);
assertThat(result.hasError()).isFalse();
assertEmptyDiff(getDirtyFilesystemKeys(evaluator, checker));
}
@@ -236,9 +234,7 @@
Iterable<SkyKey> allKeys = ImmutableList.of(symlinkKey, fooKey);
// First build -- prime the graph.
- EvaluationResult<FileValue> result =
- driver.evaluate(
- allKeys, false, SkyframeExecutor.DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<FileValue> result = driver.evaluate(allKeys, EVALUATION_OPTIONS);
assertThat(result.hasError()).isFalse();
FileValue symlinkValue = result.get(symlinkKey);
FileValue fooValue = result.get(fooKey);
@@ -253,12 +249,7 @@
assertDiffWithNewValues(getDirtyFilesystemKeys(evaluator, checker), sym1FileStateKey);
differencer.invalidate(ImmutableList.of(sym1FileStateKey));
- result =
- driver.evaluate(
- ImmutableList.<SkyKey>of(),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ result = driver.evaluate(ImmutableList.<SkyKey>of(), EVALUATION_OPTIONS);
assertThat(result.hasError()).isFalse();
assertDiffWithNewValues(getDirtyFilesystemKeys(evaluator, checker), sym1FileStateKey);
@@ -270,9 +261,7 @@
FileSystemUtils.writeContentAsLatin1(symlink, "new symlink contents");
assertDiffWithNewValues(getDirtyFilesystemKeys(evaluator, checker), symlinkFileStateKey);
differencer.invalidate(ImmutableList.of(symlinkFileStateKey));
- result =
- driver.evaluate(
- allKeys, false, SkyframeExecutor.DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ result = driver.evaluate(allKeys, EVALUATION_OPTIONS);
assertThat(result.hasError()).isFalse();
symlinkValue = result.get(symlinkKey);
assertWithMessage(symlinkValue.toString()).that(symlinkValue.isSymlink()).isFalse();
@@ -298,9 +287,7 @@
FileStateValue.key(
RootedPath.toRootedPath(Root.absoluteRoot(fs), PathFragment.create("/foo2")));
Iterable<SkyKey> skyKeys = ImmutableList.of(key1, key2);
- EvaluationResult<SkyValue> result =
- driver.evaluate(
- skyKeys, false, SkyframeExecutor.DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ EvaluationResult<SkyValue> result = driver.evaluate(skyKeys, EVALUATION_OPTIONS);
assertThat(result.hasError()).isFalse();
assertEmptyDiff(getDirtyFilesystemKeys(evaluator, checker));
@@ -320,9 +307,7 @@
assertDiffWithNewValues(getDirtyFilesystemKeys(evaluator, checker), key1, key2);
differencer.invalidate(skyKeys);
- result =
- driver.evaluate(
- skyKeys, false, SkyframeExecutor.DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ result = driver.evaluate(skyKeys, EVALUATION_OPTIONS);
assertThat(result.hasError()).isFalse();
assertEmptyDiff(getDirtyFilesystemKeys(evaluator, checker));
}
@@ -338,11 +323,7 @@
FileStateValue.key(
RootedPath.toRootedPath(Root.fromPath(pkgRoot), PathFragment.create("foo")));
EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(fileKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(fileKey), EVALUATION_OPTIONS);
assertThat(result.hasError()).isTrue();
fs.readlinkThrowsIoException = false;
@@ -363,11 +344,7 @@
SkyKey fileKey1 = FileValue.key(RootedPath.toRootedPath(Root.fromPath(pkgRoot), path1));
EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(fileKey1),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(fileKey1), EVALUATION_OPTIONS);
assertThat(result.hasError()).isTrue();
FilesystemValueChecker checker = new FilesystemValueChecker(null, null);
@@ -404,11 +381,13 @@
new TestAction(
Runnables.doNothing(), ImmutableSet.<Artifact>of(), ImmutableSet.of(out2)),
forceDigests)));
- assertThat(
- driver
- .evaluate(ImmutableList.<SkyKey>of(), false, 1, NullEventHandler.INSTANCE)
- .hasError())
- .isFalse();
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(1)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
+ assertThat(driver.evaluate(ImmutableList.<SkyKey>of(), evaluationContext).hasError()).isFalse();
assertThat(new FilesystemValueChecker(null, null).getDirtyActionValues(evaluator.getValues(),
batchStatter, ModifiedFileSet.EVERYTHING_MODIFIED)).isEmpty();
@@ -487,11 +466,13 @@
actionKeyLast,
actionValueWithEmptyDirectory(last)));
- assertThat(
- driver
- .evaluate(ImmutableList.<SkyKey>of(), false, 1, NullEventHandler.INSTANCE)
- .hasError())
- .isFalse();
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(1)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
+ assertThat(driver.evaluate(ImmutableList.<SkyKey>of(), evaluationContext).hasError()).isFalse();
assertThat(new FilesystemValueChecker(null, null).getDirtyActionValues(evaluator.getValues(),
batchStatter, ModifiedFileSet.EVERYTHING_MODIFIED)).isEmpty();
@@ -810,8 +791,7 @@
ImmutableList.of(
FileValue.key(
RootedPath.toRootedPath(Root.fromPath(pkgRoot), PathFragment.create("foo"))));
- driver.evaluate(
- values, false, SkyframeExecutor.DEFAULT_THREAD_COUNT, NullEventHandler.INSTANCE);
+ driver.evaluate(values, EVALUATION_OPTIONS);
FilesystemValueChecker checker = new FilesystemValueChecker(null, null);
assertEmptyDiff(getDirtyFilesystemKeys(evaluator, checker));
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/GlobFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/GlobFunctionTest.java
index a2d6e07d..72c0d35 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/GlobFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/GlobFunctionTest.java
@@ -55,6 +55,7 @@
import com.google.devtools.build.lib.vfs.UnixGlob;
import com.google.devtools.build.lib.vfs.inmemoryfs.InMemoryFileSystem;
import com.google.devtools.build.skyframe.ErrorInfo;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -80,6 +81,13 @@
* Tests for {@link GlobFunction}.
*/
public abstract class GlobFunctionTest {
+ private static final EvaluationContext EVALUATION_OPTIONS =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
+
@RunWith(JUnit4.class)
public static class GlobFunctionAlwaysUseDirListingTest extends GlobFunctionTest {
@Override
@@ -427,11 +435,7 @@
GlobValue.key(
PKG_ID, Root.fromPath(root), pattern, excludeDirs, PathFragment.EMPTY_FRAGMENT);
EvaluationResult<SkyValue> result =
- driver.evaluate(
- ImmutableList.of(skyKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(skyKey), EVALUATION_OPTIONS);
if (result.hasError()) {
throw result.getError().getException();
}
@@ -657,11 +661,7 @@
SkyKey skyKey =
GlobValue.key(PKG_ID, Root.fromPath(root), "*/foo", false, PathFragment.EMPTY_FRAGMENT);
EvaluationResult<GlobValue> result =
- driver.evaluate(
- ImmutableList.of(skyKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(skyKey), EVALUATION_OPTIONS);
assertThat(result.hasError()).isTrue();
ErrorInfo errorInfo = result.getError(skyKey);
assertThat(errorInfo.getException()).isInstanceOf(InconsistentFilesystemException.class);
@@ -685,11 +685,7 @@
SkyKey skyKey =
GlobValue.key(PKG_ID, Root.fromPath(root), "**/wiz", false, PathFragment.EMPTY_FRAGMENT);
EvaluationResult<GlobValue> result =
- driver.evaluate(
- ImmutableList.of(skyKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(skyKey), EVALUATION_OPTIONS);
assertThat(result.hasError()).isTrue();
ErrorInfo errorInfo = result.getError(skyKey);
assertThat(errorInfo.getException()).isInstanceOf(InconsistentFilesystemException.class);
@@ -760,11 +756,7 @@
GlobValue.key(
PKG_ID, Root.fromPath(root), "foo/bar/wiz/*", false, PathFragment.EMPTY_FRAGMENT);
EvaluationResult<GlobValue> result =
- driver.evaluate(
- ImmutableList.of(skyKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ driver.evaluate(ImmutableList.of(skyKey), EVALUATION_OPTIONS);
assertThat(result.hasError()).isTrue();
ErrorInfo errorInfo = result.getError(skyKey);
assertThat(errorInfo.getException()).isInstanceOf(InconsistentFilesystemException.class);
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/LocalRepositoryLookupFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/LocalRepositoryLookupFunctionTest.java
index 3e684e3..c6ad50e 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/LocalRepositoryLookupFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/LocalRepositoryLookupFunctionTest.java
@@ -42,6 +42,7 @@
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -141,11 +142,14 @@
private EvaluationResult<LocalRepositoryLookupValue> lookupDirectory(SkyKey directoryKey)
throws InterruptedException {
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
return driver.<LocalRepositoryLookupValue>evaluate(
- ImmutableList.of(directoryKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ ImmutableList.of(directoryKey), evaluationContext);
}
@Test
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/PackageErrorMessageFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/PackageErrorMessageFunctionTest.java
index 5679d69..605c8d1 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/PackageErrorMessageFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/PackageErrorMessageFunctionTest.java
@@ -19,6 +19,7 @@
import com.google.devtools.build.lib.analysis.util.BuildViewTestCase;
import com.google.devtools.build.lib.cmdline.PackageIdentifier;
import com.google.devtools.build.lib.skyframe.PackageErrorMessageValue.Result;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.SkyValue;
@@ -72,14 +73,14 @@
private PackageErrorMessageValue getPackageErrorMessageValue(boolean keepGoing)
throws InterruptedException {
SkyKey key = PackageErrorMessageValue.key(PackageIdentifier.createInMainRepo("a"));
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(SequencedSkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(reporter)
+ .build();
EvaluationResult<SkyValue> result =
- skyframeExecutor
- .getDriverForTesting()
- .evaluate(
- ImmutableList.of(key),
- /*keepGoing=*/ keepGoing,
- SequencedSkyframeExecutor.DEFAULT_THREAD_COUNT,
- reporter);
+ skyframeExecutor.getDriverForTesting().evaluate(ImmutableList.of(key), evaluationContext);
assertThat(result.hasError()).isFalse();
SkyValue value = result.get(key);
assertThat(value).isInstanceOf(PackageErrorMessageValue.class);
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/PackageLookupFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/PackageLookupFunctionTest.java
index 40da57f..7376c38 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/PackageLookupFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/PackageLookupFunctionTest.java
@@ -53,6 +53,7 @@
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -186,11 +187,14 @@
protected EvaluationResult<PackageLookupValue> lookupPackage(SkyKey packageIdentifierSkyKey)
throws InterruptedException {
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
return driver.<PackageLookupValue>evaluate(
- ImmutableList.of(packageIdentifierSkyKey),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ ImmutableList.of(packageIdentifierSkyKey), evaluationContext);
}
@Test
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfPatternsFunctionSmartNegationTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfPatternsFunctionSmartNegationTest.java
index e9dbfdf..b0c7fda 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfPatternsFunctionSmartNegationTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfPatternsFunctionSmartNegationTest.java
@@ -41,6 +41,7 @@
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.SkyValue;
@@ -218,14 +219,14 @@
ImmutableList<SkyKey> singletonTargetPattern = ImmutableList.of(independentTarget);
// When PrepareDepsOfPatternsFunction completes evaluation,
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(100)
+ .setEventHander(new Reporter(new EventBus(), eventCollector))
+ .build();
EvaluationResult<SkyValue> evaluationResult =
- skyframeExecutor
- .getDriverForTesting()
- .evaluate(
- singletonTargetPattern,
- keepGoing,
- /*numThreads=*/ 100,
- new Reporter(new EventBus(), eventCollector));
+ skyframeExecutor.getDriverForTesting().evaluate(singletonTargetPattern, evaluationContext);
// The evaluation has no errors if success was expected.
assertThat(evaluationResult.hasError()).isNotEqualTo(successExpected);
return Preconditions.checkNotNull(evaluationResult.getWalkableGraph());
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfPatternsFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfPatternsFunctionTest.java
index 689b035..9c801ab 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfPatternsFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfPatternsFunctionTest.java
@@ -25,6 +25,7 @@
import com.google.devtools.build.lib.events.Reporter;
import com.google.devtools.build.lib.packages.NoSuchPackageException;
import com.google.devtools.build.lib.packages.NoSuchTargetException;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.SkyValue;
@@ -203,14 +204,16 @@
ImmutableList<SkyKey> singletonTargetPattern = ImmutableList.of(independentTarget);
// When PrepareDepsOfPatternsFunction completes evaluation,
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(LOADING_PHASE_THREADS)
+ .setEventHander(new Reporter(new EventBus(), eventCollector))
+ .build();
EvaluationResult<SkyValue> evaluationResult =
getSkyframeExecutor()
.getDriverForTesting()
- .evaluate(
- singletonTargetPattern,
- keepGoing,
- LOADING_PHASE_THREADS,
- new Reporter(new EventBus(), eventCollector));
+ .evaluate(singletonTargetPattern, evaluationContext);
// Currently all callers either expect success or pass keepGoing=true, which implies success,
// since PrepareDepsOfPatternsFunction swallows all errors. Will need to be changed if a test
// that evaluates with keepGoing=false and expects errors is added.
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfTargetsUnderDirectoryFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfTargetsUnderDirectoryFunctionTest.java
index 68be10e..a577b7c 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfTargetsUnderDirectoryFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/PrepareDepsOfTargetsUnderDirectoryFunctionTest.java
@@ -31,6 +31,7 @@
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
import com.google.devtools.build.skyframe.BuildDriver;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.WalkableGraph;
@@ -80,12 +81,14 @@
private EvaluationResult<?> getEvaluationResult(SkyKey... keys) throws InterruptedException {
BuildDriver driver = skyframeExecutor.getDriverForTesting();
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SequencedSkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(reporter)
+ .build();
EvaluationResult<PrepareDepsOfTargetsUnderDirectoryValue> evaluationResult =
- driver.evaluate(
- ImmutableList.copyOf(keys),
- /*keepGoing=*/ false,
- SequencedSkyframeExecutor.DEFAULT_THREAD_COUNT,
- reporter);
+ driver.evaluate(ImmutableList.copyOf(keys), evaluationContext);
Preconditions.checkState(!evaluationResult.hasError());
return evaluationResult;
}
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/RecursiveFilesystemTraversalFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/RecursiveFilesystemTraversalFunctionTest.java
index 9268cce..6ec8cdc 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/RecursiveFilesystemTraversalFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/RecursiveFilesystemTraversalFunctionTest.java
@@ -58,6 +58,7 @@
import com.google.devtools.build.lib.vfs.RootedPath;
import com.google.devtools.build.skyframe.AbstractSkyKey;
import com.google.devtools.build.skyframe.ErrorInfo;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationProgressReceiver;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
@@ -267,11 +268,13 @@
}
private <T extends SkyValue> EvaluationResult<T> eval(SkyKey key) throws Exception {
- return driver.evaluate(
- ImmutableList.of(key),
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
+ return driver.evaluate(ImmutableList.of(key), evaluationContext);
}
private RecursiveFilesystemTraversalValue evalTraversalRequest(TraversalRequest params)
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/RecursivePkgFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/RecursivePkgFunctionTest.java
index be2e33b..16a1ff7 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/RecursivePkgFunctionTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/RecursivePkgFunctionTest.java
@@ -27,6 +27,7 @@
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.lib.vfs.RootedPath;
import com.google.devtools.build.skyframe.BuildDriver;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.WalkableGraph;
@@ -76,12 +77,14 @@
private EvaluationResult<RecursivePkgValue> getEvaluationResult(SkyKey key)
throws InterruptedException {
BuildDriver driver = skyframeExecutor.getDriverForTesting();
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SequencedSkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(reporter)
+ .build();
EvaluationResult<RecursivePkgValue> evaluationResult =
- driver.evaluate(
- ImmutableList.of(key),
- /*keepGoing=*/ false,
- SequencedSkyframeExecutor.DEFAULT_THREAD_COUNT,
- reporter);
+ driver.evaluate(ImmutableList.of(key), evaluationContext);
Preconditions.checkState(!evaluationResult.hasError());
return evaluationResult;
}
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/TimestampBuilderTestCase.java b/src/test/java/com/google/devtools/build/lib/skyframe/TimestampBuilderTestCase.java
index daf001f..298aaae 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/TimestampBuilderTestCase.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/TimestampBuilderTestCase.java
@@ -89,6 +89,7 @@
import com.google.devtools.build.lib.vfs.Root;
import com.google.devtools.build.skyframe.CycleInfo;
import com.google.devtools.build.skyframe.ErrorInfo;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationProgressReceiver;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
@@ -301,7 +302,14 @@
} catch (ActionConflictException e) {
throw new IllegalStateException(e);
}
- EvaluationResult<SkyValue> result = driver.evaluate(keys, keepGoing, threadCount, reporter);
+
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(threadCount)
+ .setEventHander(reporter)
+ .build();
+ EvaluationResult<SkyValue> result = driver.evaluate(keys, evaluationContext);
if (result.hasError()) {
boolean hasCycles = false;
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/TreeArtifactMetadataTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/TreeArtifactMetadataTest.java
index bb22054..7b052be 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/TreeArtifactMetadataTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/TreeArtifactMetadataTest.java
@@ -46,6 +46,7 @@
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.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
import com.google.devtools.build.skyframe.SkyFunction;
@@ -240,11 +241,13 @@
private <E extends SkyValue> EvaluationResult<E> evaluate(SkyKey... keys)
throws InterruptedException, ActionConflictException {
setGeneratingActions();
- return driver.evaluate(
- Arrays.asList(keys), /*keepGoing=*/
- false,
- SkyframeExecutor.DEFAULT_THREAD_COUNT,
- NullEventHandler.INSTANCE);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(NullEventHandler.INSTANCE)
+ .build();
+ return driver.evaluate(Arrays.asList(keys), evaluationContext);
}
private class TreeArtifactExecutionFunction implements SkyFunction {
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/util/SkyframeExecutorTestUtils.java b/src/test/java/com/google/devtools/build/lib/skyframe/util/SkyframeExecutorTestUtils.java
index 7b575ac..ff3cfb9 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/util/SkyframeExecutorTestUtils.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/util/SkyframeExecutorTestUtils.java
@@ -29,6 +29,7 @@
import com.google.devtools.build.lib.skyframe.SkyFunctions;
import com.google.devtools.build.lib.skyframe.SkyframeExecutor;
import com.google.devtools.build.skyframe.ErrorInfo;
+import com.google.devtools.build.skyframe.EvaluationContext;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
import com.google.devtools.build.skyframe.SkyKey;
@@ -68,8 +69,15 @@
boolean keepGoing,
ExtendedEventHandler errorEventListener)
throws InterruptedException {
- return skyframeExecutor.getDriverForTesting().evaluate(ImmutableList.of(key), keepGoing,
- SkyframeExecutor.DEFAULT_THREAD_COUNT, errorEventListener);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(SkyframeExecutor.DEFAULT_THREAD_COUNT)
+ .setEventHander(errorEventListener)
+ .build();
+ return skyframeExecutor
+ .getDriverForTesting()
+ .evaluate(ImmutableList.of(key), evaluationContext);
}
/**
diff --git a/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java b/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
index 75f70c5..607effa 100644
--- a/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
@@ -5192,7 +5192,13 @@
public <T extends SkyValue> EvaluationResult<T> eval(
boolean keepGoing, int numThreads, SkyKey... keys) throws InterruptedException {
assertThat(getModifiedValues()).isEmpty();
- return driver.evaluate(ImmutableList.copyOf(keys), keepGoing, numThreads, reporter);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(keepGoing)
+ .setNumThreads(numThreads)
+ .setEventHander(reporter)
+ .build();
+ return driver.evaluate(ImmutableList.copyOf(keys), evaluationContext);
}
public <T extends SkyValue> EvaluationResult<T> eval(boolean keepGoing, SkyKey... keys)
diff --git a/src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java b/src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java
index d974d82..a9cf718 100644
--- a/src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java
@@ -2127,7 +2127,13 @@
tester.set("d2", new StringValue("2"));
tester.set("d3", new StringValue("3"));
- driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top1")), false, 200, reporter);
+ EvaluationContext evaluationContext =
+ EvaluationContext.newBuilder()
+ .setKeepGoing(false)
+ .setNumThreads(200)
+ .setEventHander(reporter)
+ .build();
+ driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top1")), evaluationContext);
assertThat(enqueuedValues).containsExactlyElementsIn(
GraphTester.toSkyKeys("top1", "d1", "d2"));
assertThat(evaluatedValues).containsExactlyElementsIn(
@@ -2135,13 +2141,13 @@
enqueuedValues.clear();
evaluatedValues.clear();
- driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top2")), false, 200, reporter);
+ driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top2")), evaluationContext);
assertThat(enqueuedValues).containsExactlyElementsIn(GraphTester.toSkyKeys("top2", "d3"));
assertThat(evaluatedValues).containsExactlyElementsIn(GraphTester.toSkyKeys("top2", "d3"));
enqueuedValues.clear();
evaluatedValues.clear();
- driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top1")), false, 200, reporter);
+ driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top1")), evaluationContext);
assertThat(enqueuedValues).isEmpty();
assertThat(evaluatedValues).containsExactlyElementsIn(GraphTester.toSkyKeys("top1"));
}