Replace QueryableGraph#getBatch with #getBatchWithFieldHints. This allows alternate graph implementations to optimize how they construct node entries.

--
MOS_MIGRATED_REVID=126932020
diff --git a/src/main/java/com/google/devtools/build/skyframe/DelegatingWalkableGraph.java b/src/main/java/com/google/devtools/build/skyframe/DelegatingWalkableGraph.java
index c99bc32..7a584ec 100644
--- a/src/main/java/com/google/devtools/build/skyframe/DelegatingWalkableGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/DelegatingWalkableGraph.java
@@ -15,9 +15,11 @@
 
 import com.google.common.base.Function;
 import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
 import com.google.devtools.build.lib.util.Preconditions;
 
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -28,46 +30,25 @@
  * {@link WalkableGraph} that looks nodes up in a {@link QueryableGraph}.
  */
 public class DelegatingWalkableGraph implements WalkableGraph {
-
-  private final QueryableGraph fullGraph;
-  private final QueryableGraph thinGraph;
+  private final QueryableGraph graph;
 
   public DelegatingWalkableGraph(QueryableGraph graph) {
-    this(graph, graph);
-  }
-
-  /**
-   * Use this constructor when you want to differentiate reads that require the node value vs reads
-   * that only traverse dependencies.
-   */
-  public DelegatingWalkableGraph(QueryableGraph fullGraph, QueryableGraph thinGraph) {
-    this.fullGraph = fullGraph;
-    this.thinGraph = thinGraph;
+    this.graph = graph;
   }
 
   private NodeEntry getEntry(SkyKey key) {
-    NodeEntry entry = Preconditions.checkNotNull(fullGraph.get(key), key);
+    NodeEntry entry =
+        Preconditions.checkNotNull(
+            graph.getBatchWithFieldHints(ImmutableList.of(key), NodeEntryField.VALUE_ONLY).get(key),
+            key);
     Preconditions.checkState(entry.isDone(), "%s %s", key, entry);
     return entry;
   }
 
-  /**
-   * Returns a map giving the {@link NodeEntry} corresponding to the given {@code keys}. If there is
-   * no node in the graph corresponding to a {@link SkyKey} in {@code keys}, it is silently ignored
-   * and will not be present in the returned map. This tolerance allows callers to avoid
-   * pre-filtering their keys by checking for existence, which can be expensive.
-   */
-  private static Map<SkyKey, NodeEntry> getEntries(Iterable<SkyKey> keys, QueryableGraph graph) {
-    Map<SkyKey, NodeEntry> result = graph.getBatch(keys);
-    for (Map.Entry<SkyKey, NodeEntry> entry : result.entrySet()) {
-      Preconditions.checkState(entry.getValue().isDone(), entry);
-    }
-    return result;
-  }
-
   @Override
   public boolean exists(SkyKey key) {
-    NodeEntry entry = thinGraph.get(key);
+    NodeEntry entry =
+        graph.getBatchWithFieldHints(ImmutableList.of(key), NodeEntryField.NO_FIELDS).get(key);
     return entry != null && entry.isDone();
   }
 
@@ -88,14 +69,17 @@
 
   @Override
   public Map<SkyKey, SkyValue> getSuccessfulValues(Iterable<SkyKey> keys) {
-    return Maps.filterValues(Maps.transformValues(fullGraph.getBatch(keys), GET_SKY_VALUE_FUNCTION),
+    return Maps.filterValues(
+        Maps.transformValues(
+            graph.getBatchWithFieldHints(keys, NodeEntryField.VALUE_ONLY), GET_SKY_VALUE_FUNCTION),
         Predicates.notNull());
   }
 
   @Override
   public Map<SkyKey, Exception> getMissingAndExceptions(Iterable<SkyKey> keys) {
     Map<SkyKey, Exception> result = new HashMap<>();
-    Map<SkyKey, NodeEntry> graphResult = fullGraph.getBatch(keys);
+    Map<SkyKey, NodeEntry> graphResult =
+        graph.getBatchWithFieldHints(keys, NodeEntryField.VALUE_ONLY);
     for (SkyKey key : keys) {
       NodeEntry nodeEntry = graphResult.get(key);
       if (nodeEntry == null || !nodeEntry.isDone()) {
@@ -119,9 +103,11 @@
 
   @Override
   public Map<SkyKey, Iterable<SkyKey>> getDirectDeps(Iterable<SkyKey> keys) {
-    Map<SkyKey, NodeEntry> entries = getEntries(keys, thinGraph);
+    Map<SkyKey, NodeEntry> entries =
+        graph.getBatchWithFieldHints(keys, EnumSet.of(NodeEntryField.DIRECT_DEPS));
     Map<SkyKey, Iterable<SkyKey>> result = new HashMap<>(entries.size());
     for (Entry<SkyKey, NodeEntry> entry : entries.entrySet()) {
+      Preconditions.checkState(entry.getValue().isDone(), entry);
       result.put(entry.getKey(), entry.getValue().getDirectDeps());
     }
     return result;
@@ -129,9 +115,11 @@
 
   @Override
   public Map<SkyKey, Iterable<SkyKey>> getReverseDeps(Iterable<SkyKey> keys) {
-    Map<SkyKey, NodeEntry> entries = getEntries(keys, thinGraph);
+    Map<SkyKey, NodeEntry> entries =
+        graph.getBatchWithFieldHints(keys, EnumSet.of(NodeEntryField.REVERSE_DEPS));
     Map<SkyKey, Iterable<SkyKey>> result = new HashMap<>(entries.size());
     for (Entry<SkyKey, NodeEntry> entry : entries.entrySet()) {
+      Preconditions.checkState(entry.getValue().isDone(), entry);
       result.put(entry.getKey(), entry.getValue().getReverseDeps());
     }
     return result;
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 b3d8bd0..a9c067d 100644
--- a/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java
@@ -24,8 +24,9 @@
 @ThreadSafe
 interface EvaluableGraph extends QueryableGraph, DeletableGraph {
   /**
-   * Like {@link QueryableGraph#getBatch}, 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}.
+   * Like {@link QueryableGraph#getBatchWithFieldHints}, 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}.
    */
   Map<SkyKey, NodeEntry> createIfAbsentBatch(Iterable<SkyKey> keys);
 }
diff --git a/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java
index 5d508a9..6f25093 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java
@@ -17,6 +17,9 @@
 
 /** {@link ProcessableGraph} that exposes the contents of the entire graph. */
 interface InMemoryGraph extends ProcessableGraph, InvalidatableGraph {
+  @Override
+  Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys);
+
   /**
    * Returns a read-only live view of the nodes in the graph. All node are included. Dirty values
    * include their Node value. Values in error have a null value.
diff --git a/src/main/java/com/google/devtools/build/skyframe/InMemoryGraphImpl.java b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraphImpl.java
index 3679d62..ef6e24b 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryGraphImpl.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraphImpl.java
@@ -21,6 +21,7 @@
 import com.google.common.collect.Maps;
 
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 
@@ -66,6 +67,12 @@
     return builder.build();
   }
 
+  @Override
+  public Map<SkyKey, NodeEntry> getBatchWithFieldHints(
+      Iterable<SkyKey> keys, EnumSet<NodeEntryField> fields) {
+    return getBatch(keys);
+  }
+
   protected NodeEntry createIfAbsent(SkyKey key) {
     NodeEntry newval = keepEdges ? new InMemoryNodeEntry() : new EdgelessInMemoryNodeEntry();
     NodeEntry oldval = nodeMap.putIfAbsent(key, newval);
diff --git a/src/main/java/com/google/devtools/build/skyframe/NodeEntryField.java b/src/main/java/com/google/devtools/build/skyframe/NodeEntryField.java
new file mode 100644
index 0000000..97c1dfd
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/NodeEntryField.java
@@ -0,0 +1,41 @@
+// Copyright 2016 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 java.util.EnumSet;
+
+/**
+ * Fields of a {@link NodeEntry} that clients of a {@link QueryableGraph} may need. Clients may
+ * specify these fields in {@link QueryableGraph#getBatchWithFieldHints} to help particular {@link
+ * QueryableGraph} implementations decide how lazily to construct the returned node entries.
+ */
+public enum NodeEntryField {
+  /** The value ({@link NodeEntry#getValueMaybeWithMetadata}) will be needed. */
+  VALUE,
+  /** The direct deps ({@link NodeEntry#getDirectDeps}) will be needed. */
+  DIRECT_DEPS,
+  /** The reverse deps ({@link NodeEntry#getReverseDeps}) will be needed. */
+  REVERSE_DEPS,
+  /**
+   * The reverse deps as a whole will not be needed, but we may need to check the presence of a
+   * reverse dep or add/delete one.
+   */
+  INDIVIDUAL_REVERSE_DEPS;
+
+  public static final EnumSet<NodeEntryField> NO_FIELDS = EnumSet.noneOf(NodeEntryField.class);
+  public static final EnumSet<NodeEntryField> VALUE_ONLY = EnumSet.of(VALUE);
+  public static final EnumSet<NodeEntryField> NO_VALUE = EnumSet.of(DIRECT_DEPS, REVERSE_DEPS);
+  public static final EnumSet<NodeEntryField> ALL_FIELDS =
+      EnumSet.of(VALUE, DIRECT_DEPS, REVERSE_DEPS);
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
index 1254dbc..e7674cd 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -56,6 +56,7 @@
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Deque;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -221,9 +222,11 @@
     this.errorHandler = errorHandler;
   }
 
-  /**
-   * Receives the events from the NestedSet and delegates to the reporter.
-   */
+  private Map<SkyKey, NodeEntry> getBatchValues(Iterable<SkyKey> keys) {
+    return graph.getBatchWithFieldHints(keys, NodeEntryField.VALUE_ONLY);
+  }
+
+  /** Receives the events from the NestedSet and delegates to the reporter. */
   private static class NestedSetEventReceiver implements NestedSetVisitor.Receiver<TaggedEvents> {
 
     private final EventHandler reporter;
@@ -329,7 +332,7 @@
         keysToPrefetchBuilder.addAll(depKeysAsIterable).addAll(oldDeps);
         keysToPrefetch = keysToPrefetchBuilder.build();
       }
-      Map<SkyKey, NodeEntry> batchMap = graph.getBatch(keysToPrefetch);
+      Map<SkyKey, NodeEntry> batchMap = getBatchValues(keysToPrefetch);
       if (PREFETCH_OLD_DEPS) {
         batchMap = ImmutableMap.copyOf(
             Maps.filterKeys(batchMap, Predicates.in(ImmutableSet.copyOf(depKeysAsIterable))));
@@ -464,7 +467,7 @@
           missingKeys.add(key);
         }
       }
-      Map<SkyKey, NodeEntry> missingEntries = graph.getBatch(missingKeys);
+      Map<SkyKey, NodeEntry> missingEntries = getBatchValues(missingKeys);
       for (SkyKey key : missingKeys) {
         builder.put(key, maybeGetValueFromError(key, missingEntries.get(key), bubbleErrorInfo));
       }
@@ -660,7 +663,11 @@
         // Remove the rdep on this entry for each of its old deps that is no longer a direct dep.
         Set<SkyKey> depsToRemove =
             Sets.difference(oldDeps, primaryEntry.getTemporaryDirectDeps().toSet());
-        for (NodeEntry oldDepEntry : graph.getBatch(depsToRemove).values()) {
+        for (NodeEntry oldDepEntry :
+            graph
+                .getBatchWithFieldHints(
+                    depsToRemove, EnumSet.of(NodeEntryField.INDIVIDUAL_REVERSE_DEPS))
+                .values()) {
           oldDepEntry.removeReverseDep(skyKey);
         }
       }
@@ -916,7 +923,10 @@
             // is done, then it is the parent's responsibility to notice that, which we do here.
             // We check the deps for errors so that we don't continue building this node if it has
             // a child error.
-            Map<SkyKey, NodeEntry> entriesToCheck = graph.getBatch(directDepsToCheck);
+            Map<SkyKey, NodeEntry> entriesToCheck =
+                graph.getBatchWithFieldHints(
+                    directDepsToCheck,
+                    EnumSet.of(NodeEntryField.VALUE, NodeEntryField.INDIVIDUAL_REVERSE_DEPS));
             for (Map.Entry<SkyKey, NodeEntry> entry : entriesToCheck.entrySet()) {
               if (entry.getValue().isDone() && entry.getValue().getErrorInfo() != null) {
                 // If any child has an error, we arbitrarily add a dep on the first one (needed
@@ -1021,7 +1031,7 @@
             }
           }
 
-          Map<SkyKey, NodeEntry> newlyRequestedDeps = graph.getBatch(env.newlyRequestedDeps);
+          Map<SkyKey, NodeEntry> newlyRequestedDeps = getBatchValues(env.newlyRequestedDeps);
           boolean isTransitivelyTransient = reifiedBuilderException.isTransient();
           for (NodeEntry depEntry
               : Iterables.concat(env.directDeps.values(), newlyRequestedDeps.values())) {
@@ -1075,7 +1085,12 @@
             skyKey, newDirectDeps, state);
         env.setValue(value);
         registerNewlyDiscoveredDepsForDoneEntry(
-            skyKey, state, graph.getBatch(env.newlyRequestedDeps), oldDeps, env);
+            skyKey,
+            state,
+            graph.getBatchWithFieldHints(
+                env.newlyRequestedDeps, EnumSet.of(NodeEntryField.INDIVIDUAL_REVERSE_DEPS)),
+            oldDeps,
+            env);
         env.commit(/*enqueueParents=*/true);
         return;
       }
@@ -1190,7 +1205,10 @@
    */
   private void signalValuesAndEnqueueIfReady(
       @Nullable ValueVisitor visitor, Iterable<SkyKey> keys, Version version) {
-    Map<SkyKey, NodeEntry> batch = graph.getBatch(keys);
+    // No fields of the entry are needed here, since we're just enqueuing for evaluation, but more
+    // importantly, these hints are not respected for not-done nodes. If they are, we may need to
+    // alter this hint.
+    Map<SkyKey, NodeEntry> batch = graph.getBatchWithFieldHints(keys, NodeEntryField.NO_FIELDS);
     if (visitor != null) {
       for (SkyKey key : keys) {
         NodeEntry entry = Preconditions.checkNotNull(batch.get(key), key);
@@ -1224,13 +1242,13 @@
 
   /**
    * 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 provided -- we trust that a SkyFunction may be know it should throw an error even if not
-   * all of its requested deps are done. However, that means we're assuming the SkyFunction would
-   * throw that same error if all of its requested deps were done. Unfortunately, there is no way to
+   * creating a node or throwing an error. Builders may throw errors even if all their deps were not
+   * provided -- we trust that a SkyFunction may be know it should throw an error even if not all of
+   * its requested deps are done. However, that means we're assuming the SkyFunction would throw
+   * that same error if all of its requested deps were done. Unfortunately, there is no way to
    * enforce that condition.
    */
-  private void registerNewlyDiscoveredDepsForDoneEntry(
+  private static void registerNewlyDiscoveredDepsForDoneEntry(
       SkyKey skyKey,
       NodeEntry entry,
       Map<SkyKey, NodeEntry> newlyRequestedDepMap,
@@ -1288,7 +1306,7 @@
     // directly without launching the heavy machinery, spawning threads, etc.
     // Inform progressReceiver that these nodes are done to be consistent with the main code path.
     boolean allAreDone = true;
-    Map<SkyKey, NodeEntry> batch = graph.getBatch(skyKeySet);
+    Map<SkyKey, NodeEntry> batch = getBatchValues(skyKeySet);
     for (SkyKey key : skyKeySet) {
       if (!isDoneForBuild(batch.get(key))) {
         allAreDone = false;
@@ -1871,8 +1889,12 @@
         continue;
       }
       // Prefetch all children, in case our graph performs better with a primed cache. No need to
-      // recurse into done nodes.
-      Map<SkyKey, NodeEntry> childrenNodes = graph.getBatch(children);
+      // recurse into done nodes. The fields of done nodes aren't necessary, since we'll filter them
+      // out.
+      // TODO(janakr): If graph implementations start using these hints for not-done nodes, we may
+      // have to change this.
+      Map<SkyKey, NodeEntry> childrenNodes =
+          graph.getBatchWithFieldHints(children, NodeEntryField.NO_FIELDS);
       Preconditions.checkState(childrenNodes.size() == Iterables.size(children), childrenNodes);
       children = Maps.filterValues(childrenNodes, new Predicate<NodeEntry>() {
         @Override
@@ -1930,7 +1952,7 @@
    */
   private List<ErrorInfo> getChildrenErrors(Iterable<SkyKey> children, SkyKey unfinishedChild) {
     List<ErrorInfo> allErrors = new ArrayList<>();
-    for (Entry<SkyKey, NodeEntry> childMapEntry : graph.getBatch(children).entrySet()) {
+    for (Entry<SkyKey, NodeEntry> childMapEntry : getBatchValues(children).entrySet()) {
       SkyKey childKey = childMapEntry.getKey();
       NodeEntry childNodeEntry = childMapEntry.getValue();
       ErrorInfo errorInfo = getErrorMaybe(childKey, childNodeEntry,
@@ -2033,7 +2055,7 @@
   }
 
   private Map<SkyKey, NodeEntry> getAndCheckDoneBatch(Iterable<SkyKey> keys) {
-    Map<SkyKey, NodeEntry> nodes = graph.getBatch(keys);
+    Map<SkyKey, NodeEntry> nodes = getBatchValues(keys);
     for (Map.Entry<SkyKey, NodeEntry> nodeEntryMapEntry : nodes.entrySet()) {
       checkDone(nodeEntryMapEntry.getKey(), nodeEntryMapEntry.getValue());
     }
@@ -2042,7 +2064,7 @@
 
   private static final SkyValue NULL_MARKER = new SkyValue() {};
 
-  private SkyValue maybeGetValueFromError(
+  private static SkyValue maybeGetValueFromError(
       SkyKey key,
       @Nullable NodeEntry entry,
       @Nullable Map<SkyKey, ValueWithMetadata> bubbleErrorInfo) {
@@ -2056,11 +2078,11 @@
   }
 
   /**
-   * Return true if the entry does not need to be re-evaluated this build. The entry will need to
-   * be re-evaluated if it is not done, but also if it was not completely evaluated last build and
-   * this build is keepGoing.
+   * Return true if the entry does not need to be re-evaluated this build. The entry will need to be
+   * re-evaluated if it is not done, but also if it was not completely evaluated last build and this
+   * build is keepGoing.
    */
-  private boolean isDoneForBuild(@Nullable NodeEntry entry) {
+  private static boolean isDoneForBuild(@Nullable NodeEntry entry) {
     return entry != null && entry.isDone();
   }
 
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 0419828..2703802 100644
--- a/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
@@ -15,6 +15,7 @@
 
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
 
+import java.util.EnumSet;
 import java.util.Map;
 
 import javax.annotation.Nullable;
@@ -27,9 +28,13 @@
   NodeEntry get(SkyKey key);
 
   /**
-   * Fetches all the given nodes. Returns a map {@code m} such that, for all {@code k} in
-   * {@code keys}, {@code m.get(k).equals(e)} iff {@code get(k) == e} and {@code e != null}, and
-   * {@code !m.containsKey(k)} iff {@code get(k) == null}.
+   * Fetches all the given nodes. Returns a map {@code m} such that, for all {@code k} in {@code
+   * keys}, {@code m.get(k).equals(e)} iff {@code get(k) == e} and {@code e != null}, and {@code
+   * !m.containsKey(k)} iff {@code get(k) == null}. The {@code fields} parameter is a hint to the
+   * QueryableGraph implementation that allows it to possibly construct certain fields of the
+   * returned node entries more lazily. Hints may only be applied to nodes in a certain state, like
+   * done nodes.
    */
-  Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys);
+  Map<SkyKey, NodeEntry> getBatchWithFieldHints(
+      Iterable<SkyKey> keys, EnumSet<NodeEntryField> fields);
 }
diff --git a/src/main/java/com/google/devtools/build/skyframe/QueryableGraphBackedSkyFunctionEnvironment.java b/src/main/java/com/google/devtools/build/skyframe/QueryableGraphBackedSkyFunctionEnvironment.java
index c9d33a4..ed38be3 100644
--- a/src/main/java/com/google/devtools/build/skyframe/QueryableGraphBackedSkyFunctionEnvironment.java
+++ b/src/main/java/com/google/devtools/build/skyframe/QueryableGraphBackedSkyFunctionEnvironment.java
@@ -70,7 +70,8 @@
 
   @Override
   protected Map<SkyKey, ValueOrUntypedException> getValueOrUntypedExceptions(Set<SkyKey> depKeys) {
-    Map<SkyKey, NodeEntry> resultMap = queryableGraph.getBatch(depKeys);
+    Map<SkyKey, NodeEntry> resultMap =
+        queryableGraph.getBatchWithFieldHints(depKeys, NodeEntryField.VALUE_ONLY);
     Map<SkyKey, NodeEntry> resultWithMissingKeys = new HashMap<>(resultMap);
     for (SkyKey missingDep : Sets.difference(depKeys, resultMap.keySet())) {
       resultWithMissingKeys.put(missingDep, null);
diff --git a/src/test/java/com/google/devtools/build/skyframe/DeterministicHelper.java b/src/test/java/com/google/devtools/build/skyframe/DeterministicHelper.java
index 655333a..50bd380 100644
--- a/src/test/java/com/google/devtools/build/skyframe/DeterministicHelper.java
+++ b/src/test/java/com/google/devtools/build/skyframe/DeterministicHelper.java
@@ -17,6 +17,7 @@
 
 import java.util.Collection;
 import java.util.Comparator;
+import java.util.EnumSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
@@ -115,8 +116,9 @@
     }
 
     @Override
-    public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
-      return makeDeterministic(super.getBatch(keys));
+    public Map<SkyKey, NodeEntry> getBatchWithFieldHints(
+        Iterable<SkyKey> keys, EnumSet<NodeEntryField> fields) {
+      return makeDeterministic(super.getBatchWithFieldHints(keys, fields));
     }
   }
 
diff --git a/src/test/java/com/google/devtools/build/skyframe/DeterministicInMemoryGraph.java b/src/test/java/com/google/devtools/build/skyframe/DeterministicInMemoryGraph.java
index a023383..bb61d0f 100644
--- a/src/test/java/com/google/devtools/build/skyframe/DeterministicInMemoryGraph.java
+++ b/src/test/java/com/google/devtools/build/skyframe/DeterministicInMemoryGraph.java
@@ -27,6 +27,11 @@
   }
 
   @Override
+  public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
+    return getBatchWithFieldHints(keys, NodeEntryField.ALL_FIELDS);
+  }
+
+  @Override
   public Map<SkyKey, SkyValue> getValues() {
     return ((InMemoryGraph) delegate).getValues();
   }
diff --git a/src/test/java/com/google/devtools/build/skyframe/GraphConcurrencyTest.java b/src/test/java/com/google/devtools/build/skyframe/GraphConcurrencyTest.java
index a0580e2..73e42ea 100644
--- a/src/test/java/com/google/devtools/build/skyframe/GraphConcurrencyTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/GraphConcurrencyTest.java
@@ -275,8 +275,9 @@
   }
 
   /**
-   * Initially calling {@link NodeEntry#setValue} and then making sure concurrent calls to
-   * {@link QueryableGraph#get} and {@link QueryableGraph#getBatch} do not interfere with the node.
+   * Initially calling {@link NodeEntry#setValue} and then making sure concurrent calls to {@link
+   * QueryableGraph#get} and {@link QueryableGraph#getBatchWithFieldHints} do not interfere with the
+   * node.
    */
   @Test
   public void testDoneToDirty() throws Exception {
@@ -377,15 +378,16 @@
               } catch (InterruptedException e) {
                 throw new AssertionError(e);
               }
-              Map<SkyKey, NodeEntry> batchMap = graph.getBatch(batch);
+              Map<SkyKey, NodeEntry> batchMap =
+                  graph.getBatchWithFieldHints(batch, NodeEntryField.NO_FIELDS);
               getBatchCountDownLatch.countDown();
               assertThat(batchMap).hasSize(batch.size());
               for (NodeEntry entry : batchMap.values()) {
                 // Batch requests are made at the same time that the version increments from the
                 // base. Check that there is no problem in requesting the version and that the
                 // number is sane.
-                assertThat(entry.getVersion()).isAnyOf(startingVersion,
-                    getNextVersion(startingVersion));
+                assertThat(entry.getVersion())
+                    .isAnyOf(startingVersion, getNextVersion(startingVersion));
               }
             }
           };
diff --git a/src/test/java/com/google/devtools/build/skyframe/NotifyingHelper.java b/src/test/java/com/google/devtools/build/skyframe/NotifyingHelper.java
index 2f973b5..711e9e7 100644
--- a/src/test/java/com/google/devtools/build/skyframe/NotifyingHelper.java
+++ b/src/test/java/com/google/devtools/build/skyframe/NotifyingHelper.java
@@ -20,6 +20,7 @@
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
 import com.google.devtools.build.lib.util.GroupedList;
 
+import java.util.EnumSet;
 import java.util.Map;
 import java.util.Set;
 
@@ -121,8 +122,10 @@
     }
 
     @Override
-    public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
-      return Maps.transformEntries(delegate.getBatch(keys), notifyingHelper.wrapEntry);
+    public Map<SkyKey, NodeEntry> getBatchWithFieldHints(
+        Iterable<SkyKey> keys, EnumSet<NodeEntryField> fields) {
+      return Maps.transformEntries(
+          delegate.getBatchWithFieldHints(keys, fields), notifyingHelper.wrapEntry);
     }
 
     @Nullable
diff --git a/src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.java b/src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.java
index 752dac7..1689440 100644
--- a/src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.java
+++ b/src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.java
@@ -23,6 +23,11 @@
   }
 
   @Override
+  public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
+    return getBatchWithFieldHints(keys, NodeEntryField.ALL_FIELDS);
+  }
+
+  @Override
   public Map<SkyKey, SkyValue> getValues() {
     return ((InMemoryGraph) delegate).getValues();
   }