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"));
   }