Add DONE_CHECKING as a reason for requesting nodes.

Split registering the unique new deps of a node between those where we're enqueueing a known dependency from a prior build and one where we're adding a new dependency.

Replace prefetchBatch with getBatchAsync and add createIfAbsentBatchAsync.

PiperOrigin-RevId: 190471980
diff --git a/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java
index 6051f60..92ad4d6 100644
--- a/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java
@@ -18,6 +18,7 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
 import com.google.devtools.build.lib.clock.BlazeClock;
 import com.google.devtools.build.lib.events.Event;
 import com.google.devtools.build.lib.events.ExtendedEventHandler;
@@ -388,7 +389,7 @@
 
             Map<SkyKey, ? extends NodeEntry> newlyRequestedDeps =
                 evaluatorContext.getBatchValues(
-                    skyKey, Reason.RDEP_ADDITION, env.getNewlyRequestedDeps());
+                    skyKey, Reason.DONE_CHECKING, env.getNewlyRequestedDeps());
             boolean isTransitivelyTransient = reifiedBuilderException.isTransient();
             for (NodeEntry depEntry :
                 Iterables.concat(env.getDirectDepsValues(), newlyRequestedDeps.values())) {
@@ -405,7 +406,7 @@
                 reifiedBuilderException,
                 isTransitivelyTransient);
             registerNewlyDiscoveredDepsForDoneEntry(
-                skyKey, state, newlyRequestedDeps, oldDeps, env, evaluatorContext.keepGoing());
+                skyKey, state, oldDeps, env, evaluatorContext.keepGoing());
             env.setError(state, errorInfo);
             Set<SkyKey> rdepsToBubbleUpTo =
                 env.commit(
@@ -427,6 +428,8 @@
           env.doneBuilding();
         }
 
+        // Helper objects for all the newly requested deps that weren't known to the environment,
+        // and may contain duplicate elements.
         GroupedListHelper<SkyKey> newDirectDeps = env.getNewlyRequestedDeps();
 
         if (value != null) {
@@ -441,7 +444,6 @@
           registerNewlyDiscoveredDepsForDoneEntry(
               skyKey,
               state,
-              graph.getBatch(skyKey, Reason.RDEP_ADDITION, env.getNewlyRequestedDeps()),
               oldDeps,
               env,
               evaluatorContext.keepGoing());
@@ -494,7 +496,9 @@
         // TODO(bazel-team): An ill-behaved SkyFunction can throw us into an infinite loop where we
         // add more dependencies on every run. [skyframe-core]
 
-        // Add all new keys to the set of known deps.
+        // Add all the newly requested dependencies to the temporary direct deps. Note that
+        // newDirectDeps does not contain any elements in common with the already existing temporary
+        // direct deps. uniqueNewDeps will be the set of unique keys contained in newDirectDeps.
         Set<SkyKey> uniqueNewDeps = state.addTemporaryDirectDeps(newDirectDeps);
 
         // If there were no newly requested dependencies, at least one of them was in error or there
@@ -520,16 +524,34 @@
           return;
         }
 
+        // We want to split apart the dependencies that existed for this node the last time we did
+        // an evaluation and those that were introduced in this evaluation. To be clear, the prefix
+        // "newDeps" refers to newly discovered this time around after a SkyFunction#compute call
+        // and not to be confused with the oldDeps variable which refers to the last evaluation,
+        // (ie) a prior call to ParallelEvaluator#eval).
+        Set<SkyKey> newDepsThatWerentInTheLastEvaluation = Sets.difference(uniqueNewDeps, oldDeps);
+        Set<SkyKey> newDepsThatWereInTheLastEvaluation =
+            Sets.difference(uniqueNewDeps, newDepsThatWerentInTheLastEvaluation);
+
+        InterruptibleSupplier<Map<SkyKey, ? extends NodeEntry>>
+            newDepsThatWerentInTheLastEvaluationNodes =
+                graph.createIfAbsentBatchAsync(
+                    skyKey, Reason.RDEP_ADDITION, newDepsThatWerentInTheLastEvaluation);
+
         for (Entry<SkyKey, ? extends NodeEntry> e :
-            graph.createIfAbsentBatch(skyKey, Reason.ENQUEUING_CHILD, uniqueNewDeps).entrySet()) {
+            graph
+                .getBatch(skyKey, Reason.ENQUEUING_CHILD, newDepsThatWereInTheLastEvaluation)
+                .entrySet()) {
           SkyKey newDirectDep = e.getKey();
           NodeEntry newDirectDepEntry = e.getValue();
-          enqueueChild(
-              skyKey,
-              state,
-              newDirectDep,
-              newDirectDepEntry,
-              /*depAlreadyExists=*/ oldDeps.contains(newDirectDep));
+          enqueueChild(skyKey, state, newDirectDep, newDirectDepEntry, /*depAlreadyExists=*/ true);
+        }
+
+        for (Entry<SkyKey, ? extends NodeEntry> e :
+            newDepsThatWerentInTheLastEvaluationNodes.get().entrySet()) {
+          SkyKey newDirectDep = e.getKey();
+          NodeEntry newDirectDepEntry = e.getValue();
+          enqueueChild(skyKey, state, newDirectDep, newDirectDepEntry, /*depAlreadyExists=*/ false);
         }
         // It is critical that there is no code below this point in the try block.
       } catch (InterruptedException ie) {
@@ -585,7 +607,6 @@
     }
   }
 
-
   /**
    * Add any additional deps that were registered during the run of a builder that finished by
    * creating a node or throwing an error. Builders may throw errors even if all their deps were not
@@ -596,21 +617,22 @@
    *
    * @throws InterruptedException
    */
-  private static void registerNewlyDiscoveredDepsForDoneEntry(
+  private void registerNewlyDiscoveredDepsForDoneEntry(
       SkyKey skyKey,
       NodeEntry entry,
-      Map<SkyKey, ? extends NodeEntry> newlyRequestedDepMap,
       Set<SkyKey> oldDeps,
       SkyFunctionEnvironment env,
       boolean keepGoing)
       throws InterruptedException {
 
-    Iterator<SkyKey> it = env.getNewlyRequestedDeps().iterator();
-    if (!it.hasNext()) {
-      return;
-    }
     // We don't expect any unfinished deps in a keep-going build.
     if (!keepGoing) {
+      Map<SkyKey, ? extends NodeEntry> newlyRequestedDepMap =
+          graph.getBatch(skyKey, Reason.DONE_CHECKING, env.getNewlyRequestedDeps());
+      Iterator<SkyKey> it = env.getNewlyRequestedDeps().iterator();
+      if (!it.hasNext()) {
+        return;
+      }
       Set<SkyKey> unfinishedDeps = new HashSet<>();
       while (it.hasNext()) {
         SkyKey dep = it.next();
@@ -622,18 +644,45 @@
     }
 
     Set<SkyKey> uniqueNewDeps = entry.addTemporaryDirectDeps(env.getNewlyRequestedDeps());
-    for (SkyKey newDep : uniqueNewDeps) {
-      // Note that this depEntry can't be null. If env.newlyRequestedDeps contained a key with a
+    Set<SkyKey> newlyAddedNewDeps = Sets.difference(uniqueNewDeps, oldDeps);
+    Set<SkyKey> previouslyRegisteredNewDeps = Sets.difference(uniqueNewDeps, newlyAddedNewDeps);
+
+    InterruptibleSupplier<Map<SkyKey, ? extends NodeEntry>> newlyAddedNewDepNodes =
+        graph.getBatchAsync(skyKey, Reason.RDEP_ADDITION, newlyAddedNewDeps);
+
+    for (Entry<SkyKey, ? extends NodeEntry> newDep :
+        graph.getBatch(skyKey, Reason.SIGNAL_DEP, previouslyRegisteredNewDeps).entrySet()) {
+      // Note that this depEntry can't be null. In a keep-going build, we expect all deps to be
+      // done. In a non-keep-going build, If env.newlyRequestedDeps contained a key with a
       // null entry, then it would have been added to unfinishedDeps and then removed from
       // env.newlyRequestedDeps just above this loop.
-      NodeEntry depEntry = Preconditions.checkNotNull(newlyRequestedDepMap.get(newDep), newDep);
-      DependencyState triState =
-          oldDeps.contains(newDep)
-              ? depEntry.checkIfDoneForDirtyReverseDep(skyKey)
-              : depEntry.addReverseDepAndCheckIfDone(skyKey);
-      Preconditions.checkState(DependencyState.DONE == triState,
-          "new dep %s was not already done for %s. ValueEntry: %s. DepValueEntry: %s",
-          newDep, skyKey, entry, depEntry);
+      NodeEntry depEntry = newDep.getValue();
+      DependencyState triState = depEntry.checkIfDoneForDirtyReverseDep(skyKey);
+      Preconditions.checkState(
+          DependencyState.DONE == triState,
+          "new dep %s was not already done for %s. NodeEntry: %s. DepNodeEntry: %s",
+          newDep,
+          skyKey,
+          entry,
+          depEntry);
+      entry.signalDep();
+    }
+
+    for (SkyKey newDep : newlyAddedNewDeps) {
+      // Note that this depEntry can't be null. In a keep-going build, we expect all deps to be
+      // done. In a non-keep-going build, If env.newlyRequestedDeps contained a key with a
+      // null entry, then it would have been added to unfinishedDeps and then removed from
+      // env.newlyRequestedDeps just above this loop.
+      NodeEntry depEntry =
+          Preconditions.checkNotNull(newlyAddedNewDepNodes.get().get(newDep), newDep);
+      DependencyState triState = depEntry.addReverseDepAndCheckIfDone(skyKey);
+      Preconditions.checkState(
+          DependencyState.DONE == triState,
+          "new dep %s was not already done for %s. NodeEntry: %s. DepNodeEntry: %s",
+          newDep,
+          skyKey,
+          entry,
+          depEntry);
       entry.signalDep();
     }
     Preconditions.checkState(
diff --git a/src/main/java/com/google/devtools/build/skyframe/BUILD b/src/main/java/com/google/devtools/build/skyframe/BUILD
index bb24dde..34ad3a7 100644
--- a/src/main/java/com/google/devtools/build/skyframe/BUILD
+++ b/src/main/java/com/google/devtools/build/skyframe/BUILD
@@ -40,6 +40,7 @@
         "//src/main/java/com/google/devtools/build/lib/skyframe/serialization/autocodec",
         "//src/main/java/com/google/devtools/build/lib/vfs",
         "//src/main/java/com/google/devtools/common/options",
+        "//third_party:error_prone",
         "//third_party:guava",
         "//third_party:jsr305",
     ],
diff --git a/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java b/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java
index 1ac8620..d711fc1 100644
--- a/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java
@@ -14,6 +14,7 @@
 package com.google.devtools.build.skyframe;
 
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
+import com.google.errorprone.annotations.CanIgnoreReturnValue;
 import java.util.Collection;
 import java.util.Map;
 import javax.annotation.Nullable;
@@ -40,6 +41,21 @@
       @Nullable SkyKey requestor, Reason reason, Iterable<SkyKey> keys) throws InterruptedException;
 
   /**
+   * Like {@link QueryableGraph#getBatchAsync}, except it creates a new node for each key not
+   * already present in the graph. Thus, the returned map will have an entry for each key in {@code
+   * keys}.
+   *
+   * @param requestor if non-{@code null}, the node on behalf of which the given {@code keys} are
+   *     being requested.
+   * @param reason the reason the nodes are being requested.
+   */
+  @CanIgnoreReturnValue
+  default InterruptibleSupplier<Map<SkyKey, ? extends NodeEntry>> createIfAbsentBatchAsync(
+      @Nullable SkyKey requestor, Reason reason, Iterable<SkyKey> keys) {
+    return InterruptibleSupplier.Memoize.of(() -> createIfAbsentBatch(requestor, reason, keys));
+  }
+
+  /**
    * Optional optimization: graph may use internal knowledge to filter out keys in {@code deps} that
    * have not been recomputed since the last computation of {@code parent}. When determining if
    * {@code parent} needs to be re-evaluated, this may be used to avoid unnecessary graph accesses.
diff --git a/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
index a50284b..0279639 100644
--- a/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
@@ -356,7 +356,8 @@
 
   /**
    * Adds the temporary direct deps given in {@code helper} and returns the set of unique deps
-   * added.
+   * added. It is the users responsibility to ensure that there are no elements in common between
+   * helper and the already existing temporary direct deps.
    */
   @ThreadSafe
   Set<SkyKey> addTemporaryDirectDeps(GroupedListHelper<SkyKey> helper);
diff --git a/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java b/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
index 8c5af09..4a1a31a 100644
--- a/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
@@ -14,6 +14,7 @@
 package com.google.devtools.build.skyframe;
 
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
+import com.google.errorprone.annotations.CanIgnoreReturnValue;
 import java.util.Map;
 import javax.annotation.Nullable;
 
@@ -50,12 +51,13 @@
           throws InterruptedException;
 
   /**
-   * A prefetch call may be used to hint to the graph that we may call {@link #getBatch} on the
-   * specified keys later.
+   * A version of {@link #getBatch} that returns an {@link InterruptibleSupplier} to possibly
+   * retrieve the results later.
    */
-  default void prefetchBatch(
+  @CanIgnoreReturnValue
+  default InterruptibleSupplier<Map<SkyKey, ? extends NodeEntry>> getBatchAsync(
       @Nullable SkyKey requestor, Reason reason, Iterable<? extends SkyKey> keys) {
-    // Do nothing.
+    return InterruptibleSupplier.Memoize.of(() -> getBatch(requestor, reason, keys));
   }
 
   /**
@@ -128,6 +130,9 @@
     /** The node is being looked up merely for an existence check. */
     EXISTENCE_CHECKING,
 
+    /** The node is being looked up merely to see if it is done or not. */
+    DONE_CHECKING,
+
     /**
      * The node is being looked up to service {@link WalkableGraph#getValue},
      * {@link WalkableGraph#getException}, {@link WalkableGraph#getMissingAndExceptions}, or
diff --git a/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java
index d8c8f7a..cff9f46 100644
--- a/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java
@@ -158,7 +158,7 @@
         depKeysAsSet = depKeys.toSet();
         evaluatorContext
             .getGraph()
-            .prefetchBatch(
+            .getBatchAsync(
                 requestor,
                 Reason.PREFETCH,
                 Iterables.filter(oldDeps, Predicates.not(Predicates.in(depKeysAsSet))));