Allow Skyframe tests to use the NotifyingGraph and DeterministicGraph helpers to enforce concurrency synchronization points and determinism even if they are not using an InMemoryGraph-backed evaluator.

--
MOS_MIGRATED_REVID=121977783
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 6fe62ec..3172b4a 100644
--- a/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java
+++ b/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java
@@ -20,9 +20,7 @@
 
 import javax.annotation.Nullable;
 
-/**
- * A BuildDriver wraps a MemoizingEvaluator, passing along the proper Version.
- */
+/** A BuildDriver wraps a MemoizingEvaluator, passing along the proper Version. */
 public interface BuildDriver {
   /**
    * See {@link MemoizingEvaluator#evaluate}, which has the same semantics except for the
@@ -48,4 +46,6 @@
   @Nullable
   ErrorInfo getExistingErrorForTesting(SkyKey key);
 
+  @Nullable
+  NodeEntry getEntryForTesting(SkyKey key);
 }
diff --git a/src/main/java/com/google/devtools/build/skyframe/DelegatingNodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/DelegatingNodeEntry.java
new file mode 100644
index 0000000..52b2ffb
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/DelegatingNodeEntry.java
@@ -0,0 +1,192 @@
+// 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 com.google.devtools.build.lib.util.GroupedList.GroupedListHelper;
+
+import java.util.Collection;
+import java.util.Set;
+
+import javax.annotation.Nullable;
+
+/** Convenience class for {@link NodeEntry} implementations that delegate many operations. */
+public abstract class DelegatingNodeEntry implements NodeEntry {
+  protected abstract NodeEntry getDelegate();
+
+  protected ThinNodeEntry getThinDelegate() {
+    return getDelegate();
+  }
+
+  @Override
+  public boolean keepEdges() {
+    return getDelegate().keepEdges();
+  }
+
+  @Override
+  public SkyValue getValue() {
+    return getDelegate().getValue();
+  }
+
+  @Override
+  public SkyValue getValueMaybeWithMetadata() {
+    return getDelegate().getValueMaybeWithMetadata();
+  }
+
+  @Override
+  public SkyValue toValue() {
+    return getDelegate().toValue();
+  }
+
+  @Nullable
+  @Override
+  public ErrorInfo getErrorInfo() {
+    return getDelegate().getErrorInfo();
+  }
+
+  @Override
+  public Set<SkyKey> getInProgressReverseDeps() {
+    return getDelegate().getInProgressReverseDeps();
+  }
+
+  @Override
+  public Set<SkyKey> setValue(SkyValue value, Version version) {
+    return getDelegate().setValue(value, version);
+  }
+
+  @Override
+  public DependencyState addReverseDepAndCheckIfDone(@Nullable SkyKey reverseDep) {
+    return getDelegate().addReverseDepAndCheckIfDone(reverseDep);
+  }
+
+  @Override
+  public DependencyState checkIfDoneForDirtyReverseDep(SkyKey reverseDep) {
+    return getDelegate().checkIfDoneForDirtyReverseDep(reverseDep);
+  }
+
+  @Override
+  public boolean signalDep() {
+    return getDelegate().signalDep();
+  }
+
+  @Override
+  public boolean signalDep(Version childVersion) {
+    return getDelegate().signalDep(childVersion);
+  }
+
+  @Override
+  public Set<SkyKey> markClean() {
+    return getDelegate().markClean();
+  }
+
+  @Override
+  public void forceRebuild() {
+    getDelegate().forceRebuild();
+  }
+
+  @Override
+  public Version getVersion() {
+    return getDelegate().getVersion();
+  }
+
+  @Override
+  public DirtyState getDirtyState() {
+    return getDelegate().getDirtyState();
+  }
+
+  @Override
+  public Collection<SkyKey> getNextDirtyDirectDeps() {
+    return getDelegate().getNextDirtyDirectDeps();
+  }
+
+  @Override
+  public Iterable<SkyKey> getAllDirectDepsForIncompleteNode() {
+    return getDelegate().getAllDirectDepsForIncompleteNode();
+  }
+
+  @Override
+  public Collection<SkyKey> markRebuildingAndGetAllRemainingDirtyDirectDeps() {
+    return getDelegate().markRebuildingAndGetAllRemainingDirtyDirectDeps();
+  }
+
+  @Override
+  public Set<SkyKey> getTemporaryDirectDeps() {
+    return getDelegate().getTemporaryDirectDeps();
+  }
+
+  @Override
+  public boolean noDepsLastBuild() {
+    return getDelegate().noDepsLastBuild();
+  }
+
+  @Override
+  public void removeUnfinishedDeps(Set<SkyKey> unfinishedDeps) {
+    getDelegate().removeUnfinishedDeps(unfinishedDeps);
+  }
+
+  @Override
+  public void addTemporaryDirectDeps(GroupedListHelper<SkyKey> helper) {
+    getDelegate().addTemporaryDirectDeps(helper);
+  }
+
+  @Override
+  public boolean isReady() {
+    return getDelegate().isReady();
+  }
+
+  @Override
+  public boolean isDone() {
+    return getThinDelegate().isDone();
+  }
+
+  @Override
+  public Iterable<SkyKey> getDirectDeps() {
+    return getThinDelegate().getDirectDeps();
+  }
+
+  @Override
+  public void removeReverseDep(SkyKey reverseDep) {
+    getThinDelegate().removeReverseDep(reverseDep);
+  }
+
+  @Override
+  public void removeInProgressReverseDep(SkyKey reverseDep) {
+    getThinDelegate().removeInProgressReverseDep(reverseDep);
+  }
+
+  @Override
+  public Iterable<SkyKey> getReverseDeps() {
+    return getThinDelegate().getReverseDeps();
+  }
+
+  @Override
+  public boolean isDirty() {
+    return getThinDelegate().isDirty();
+  }
+
+  @Override
+  public boolean isChanged() {
+    return getThinDelegate().isChanged();
+  }
+
+  @Override
+  @Nullable
+  public MarkedDirtyResult markDirty(boolean isChanged) {
+    return getThinDelegate().markDirty(isChanged);
+  }
+
+  @Override
+  public void addTemporaryDirectDepsGroupToDirtyEntry(Collection<SkyKey> group) {
+    getDelegate().addTemporaryDirectDepsGroupToDirtyEntry(group);
+  }
+}
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 5a589ba..02735c0 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java
@@ -1,4 +1,4 @@
-// Copyright 2014 The Bazel Authors. All rights reserved.
+// 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.
@@ -13,135 +13,22 @@
 // limitations under the License.
 package com.google.devtools.build.skyframe;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.MapMaker;
-import com.google.common.collect.Maps;
-
-import java.util.Collections;
 import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
 
-import javax.annotation.Nullable;
-
-/**
- * An in-memory graph implementation. All operations are thread-safe with ConcurrentMap semantics.
- * Also see {@link NodeEntry}.
- *
- * <p>This class is public only for use in alternative graph implementations.
- */
-public class InMemoryGraph implements ProcessableGraph {
-
-  protected final ConcurrentMap<SkyKey, NodeEntry> nodeMap =
-      new MapMaker().initialCapacity(1024).concurrencyLevel(200).makeMap();
-  private final boolean keepEdges;
-
-  InMemoryGraph() {
-    this(/*keepEdges=*/true);
-  }
-
-  public InMemoryGraph(boolean keepEdges) {
-    this.keepEdges = keepEdges;
-  }
-
-  @Override
-  public void remove(SkyKey skyKey) {
-    nodeMap.remove(skyKey);
-  }
-
-  @Override
-  public NodeEntry get(SkyKey skyKey) {
-    return nodeMap.get(skyKey);
-  }
-
-  @Override
-  public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
-    ImmutableMap.Builder<SkyKey, NodeEntry> builder = ImmutableMap.builder();
-    for (SkyKey key : keys) {
-      NodeEntry entry = get(key);
-      if (entry != null) {
-        builder.put(key, entry);
-      }
-    }
-    return builder.build();
-  }
-
-  protected NodeEntry createIfAbsent(SkyKey key) {
-    NodeEntry newval = keepEdges ? new InMemoryNodeEntry() : new EdgelessInMemoryNodeEntry();
-    NodeEntry oldval = nodeMap.putIfAbsent(key, newval);
-    return oldval == null ? newval : oldval;
-  }
-
-  @Override
-  public Map<SkyKey, NodeEntry> createIfAbsentBatch(Iterable<SkyKey> keys) {
-    ImmutableMap.Builder<SkyKey, NodeEntry> builder = ImmutableMap.builder();
-    for (SkyKey key : keys) {
-      builder.put(key, createIfAbsent(key));
-    }
-    return builder.build();
-  }
-
-  /** Only done nodes exist to the outside world. */
-  private static final Predicate<NodeEntry> NODE_DONE_PREDICATE =
-      new Predicate<NodeEntry>() {
-        @Override
-        public boolean apply(NodeEntry entry) {
-          return entry != null && entry.isDone();
-        }
-      };
-
-  /**
-   * Returns a value, if it exists. If not, returns null.
-   */
-  @Nullable public SkyValue getValue(SkyKey key) {
-    NodeEntry entry = get(key);
-    return NODE_DONE_PREDICATE.apply(entry) ? entry.getValue() : null;
-  }
-
+/** {@link ProcessableGraph} that exposes the contents of the entire graph. */
+interface InMemoryGraph extends ProcessableGraph {
   /**
    * 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.
    */
-  Map<SkyKey, SkyValue> getValues() {
-    return Collections.unmodifiableMap(Maps.transformValues(
-        nodeMap,
-        new Function<NodeEntry, SkyValue>() {
-          @Override
-          public SkyValue apply(NodeEntry entry) {
-            return entry.toValue();
-          }
-        }));
-  }
+  Map<SkyKey, SkyValue> getValues();
 
   /**
    * Returns a read-only live view of the done values in the graph. Dirty, changed, and error values
    * are not present in the returned map
    */
-  Map<SkyKey, SkyValue> getDoneValues() {
-    return Collections.unmodifiableMap(Maps.filterValues(Maps.transformValues(
-        nodeMap,
-        new Function<NodeEntry, SkyValue>() {
-          @Override
-          public SkyValue apply(NodeEntry entry) {
-            return entry.isDone() ? entry.getValue() : null;
-          }
-        }), Predicates.notNull()));
-  }
+  Map<SkyKey, SkyValue> getDoneValues();
 
   // Only for use by MemoizingEvaluator#delete
-  Map<SkyKey, NodeEntry> getAllValues() {
-    return Collections.unmodifiableMap(nodeMap);
-  }
-
-  @VisibleForTesting
-  protected ConcurrentMap<SkyKey, NodeEntry> getNodeMap() {
-    return nodeMap;
-  }
-
-  boolean keepsEdges() {
-    return keepEdges;
-  }
+  Map<SkyKey, NodeEntry> getAllValues();
 }
diff --git a/src/main/java/com/google/devtools/build/skyframe/InMemoryGraphImpl.java b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraphImpl.java
new file mode 100644
index 0000000..3679d62
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraphImpl.java
@@ -0,0 +1,125 @@
+// Copyright 2014 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.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.MapMaker;
+import com.google.common.collect.Maps;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * An in-memory graph implementation. All operations are thread-safe with ConcurrentMap semantics.
+ * Also see {@link NodeEntry}.
+ *
+ * <p>This class is public only for use in alternative graph implementations.
+ */
+public class InMemoryGraphImpl implements InMemoryGraph {
+
+  protected final ConcurrentMap<SkyKey, NodeEntry> nodeMap =
+      new MapMaker().initialCapacity(1024).concurrencyLevel(200).makeMap();
+  private final boolean keepEdges;
+
+  InMemoryGraphImpl() {
+    this(/*keepEdges=*/ true);
+  }
+
+  public InMemoryGraphImpl(boolean keepEdges) {
+    this.keepEdges = keepEdges;
+  }
+
+  @Override
+  public void remove(SkyKey skyKey) {
+    nodeMap.remove(skyKey);
+  }
+
+  @Override
+  public NodeEntry get(SkyKey skyKey) {
+    return nodeMap.get(skyKey);
+  }
+
+  @Override
+  public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
+    ImmutableMap.Builder<SkyKey, NodeEntry> builder = ImmutableMap.builder();
+    for (SkyKey key : keys) {
+      NodeEntry entry = get(key);
+      if (entry != null) {
+        builder.put(key, entry);
+      }
+    }
+    return builder.build();
+  }
+
+  protected NodeEntry createIfAbsent(SkyKey key) {
+    NodeEntry newval = keepEdges ? new InMemoryNodeEntry() : new EdgelessInMemoryNodeEntry();
+    NodeEntry oldval = nodeMap.putIfAbsent(key, newval);
+    return oldval == null ? newval : oldval;
+  }
+
+  @Override
+  public Map<SkyKey, NodeEntry> createIfAbsentBatch(Iterable<SkyKey> keys) {
+    ImmutableMap.Builder<SkyKey, NodeEntry> builder = ImmutableMap.builder();
+    for (SkyKey key : keys) {
+      builder.put(key, createIfAbsent(key));
+    }
+    return builder.build();
+  }
+
+  @Override
+  public Map<SkyKey, SkyValue> getValues() {
+    return Collections.unmodifiableMap(
+        Maps.transformValues(
+            nodeMap,
+            new Function<NodeEntry, SkyValue>() {
+              @Override
+              public SkyValue apply(NodeEntry entry) {
+                return entry.toValue();
+              }
+            }));
+  }
+
+  @Override
+  public Map<SkyKey, SkyValue> getDoneValues() {
+    return Collections.unmodifiableMap(
+        Maps.filterValues(
+            Maps.transformValues(
+                nodeMap,
+                new Function<NodeEntry, SkyValue>() {
+                  @Override
+                  public SkyValue apply(NodeEntry entry) {
+                    return entry.isDone() ? entry.getValue() : null;
+                  }
+                }),
+            Predicates.notNull()));
+  }
+
+  @Override
+  public Map<SkyKey, NodeEntry> getAllValues() {
+    return Collections.unmodifiableMap(nodeMap);
+  }
+
+  @VisibleForTesting
+  protected ConcurrentMap<SkyKey, NodeEntry> getNodeMap() {
+    return nodeMap;
+  }
+
+  boolean keepsEdges() {
+    return keepEdges;
+  }
+}
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 5b55a63..2d571c5 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
@@ -100,7 +100,7 @@
     this.skyFunctions = ImmutableMap.copyOf(skyFunctions);
     this.differencer = Preconditions.checkNotNull(differencer);
     this.progressReceiver = invalidationReceiver;
-    this.graph = new InMemoryGraph(keepEdges);
+    this.graph = new InMemoryGraphImpl(keepEdges);
     this.emittedEventState = emittedEventState;
     this.keepEdges = keepEdges;
   }
@@ -261,22 +261,35 @@
     return graph.getDoneValues();
   }
 
+  private static boolean isDone(@Nullable NodeEntry entry) {
+    return entry != null && entry.isDone();
+  }
+
   @Override
   @Nullable public SkyValue getExistingValueForTesting(SkyKey key) {
-    return graph.getValue(key);
+    NodeEntry entry = getExistingEntryForTesting(key);
+    return isDone(entry) ? entry.getValue() : null;
   }
 
   @Override
   @Nullable public ErrorInfo getExistingErrorForTesting(SkyKey key) {
-    NodeEntry entry = graph.get(key);
-    return (entry == null || !entry.isDone()) ? null : entry.getErrorInfo();
+    NodeEntry entry = getExistingEntryForTesting(key);
+    return isDone(entry) ? entry.getErrorInfo() : null;
   }
 
-  public void setGraphForTesting(InMemoryGraph graph) {
-    this.graph = graph;
+  @Nullable
+  @Override
+  public NodeEntry getExistingEntryForTesting(SkyKey key) {
+    return graph.get(key);
   }
 
-  public InMemoryGraph getGraphForTesting() {
+  @Override
+  public void injectGraphTransformerForTesting(
+      Function<ThinNodeQueryableGraph, ProcessableGraph> transformer) {
+    this.graph = (InMemoryGraph) transformer.apply(this.graph);
+  }
+
+  public ProcessableGraph getGraphForTesting() {
     return graph;
   }
 
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 f00d88f..fa16ae8 100644
--- a/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
@@ -14,6 +14,7 @@
 package com.google.devtools.build.skyframe;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.devtools.build.lib.collect.nestedset.NestedSetVisitor;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadHostile;
@@ -119,6 +120,25 @@
   @Nullable
   ErrorInfo getExistingErrorForTesting(SkyKey key);
 
+  @Nullable
+  NodeEntry getExistingEntryForTesting(SkyKey key);
+
+  /**
+   * Tests that want finer control over the graph being used may provide a {@code transformer} here.
+   * This {@code transformer} will be applied to the graph for each invalidation/evaluation. While
+   * the graph returned by {@code transformer#apply} must technically be a {@link ProcessableGraph},
+   * if a {@link ThinNodeQueryableGraph} was given as the argument to {@code transformer#apply},
+   * then only the methods in {@link ThinNodeQueryableGraph} will be called on the returned graph,
+   * in other words it will be treated as a {@link ThinNodeQueryableGraph}. Thus, the returned graph
+   * is free not to actually implement the remaining methods in {@link ProcessableGraph} in that
+   * case.
+   *
+   * <p>Similarly, if the argument to {@code transformer#apply} is an {@link InMemoryGraph}, then
+   * the resulting graph must be an {@link InMemoryGraph}.
+   * */
+  void injectGraphTransformerForTesting(
+      Function<ThinNodeQueryableGraph, ProcessableGraph> transformer);
+
   /**
    * Write the graph to the output stream. Not necessarily thread-safe. Use only for debugging
    * purposes.
@@ -126,10 +146,8 @@
   @ThreadHostile
   void dump(boolean summarize, PrintStream out);
 
-  /**
-   * A supplier for creating instances of a particular evaluator implementation.
-   */
-  public static interface EvaluatorSupplier {
+  /** A supplier for creating instances of a particular evaluator implementation. */
+  interface EvaluatorSupplier {
     MemoizingEvaluator create(
         Map<SkyFunctionName, ? extends SkyFunction> skyFunctions,
         Differencer differencer,
@@ -143,5 +161,5 @@
    * {@code EmittedEventState} first and pass it to the graph during creation. This allows them to
    * determine whether or not to replay events.
    */
-  public static class EmittedEventState extends NestedSetVisitor.VisitedState<TaggedEvents> {}
+  class EmittedEventState extends NestedSetVisitor.VisitedState<TaggedEvents> {}
 }
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 da39237..51c393a 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -1261,9 +1261,12 @@
     // We delay this check until we know that some kind of evaluation is necessary, since !keepGoing
     // and !keepsEdges are incompatible only in the case of a failed evaluation -- there is no
     // need to be overly harsh to callers who are just trying to retrieve a cached result.
-    Preconditions.checkState(keepGoing || !(graph instanceof InMemoryGraph)
-        || ((InMemoryGraph) graph).keepsEdges(),
-        "nokeep_going evaluations are not allowed if graph edges are not kept: %s", skyKeys);
+    Preconditions.checkState(
+        keepGoing
+            || !(graph instanceof InMemoryGraphImpl)
+            || ((InMemoryGraphImpl) graph).keepsEdges(),
+        "nokeep_going evaluations are not allowed if graph edges are not kept: %s",
+        skyKeys);
 
     Profiler.instance().startTask(ProfilerTask.SKYFRAME_EVAL, skyKeySet);
     try {
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 886c40a..c65f681 100644
--- a/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java
+++ b/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java
@@ -63,4 +63,10 @@
   public ErrorInfo getExistingErrorForTesting(SkyKey key) {
     return memoizingEvaluator.getExistingErrorForTesting(key);
   }
+
+  @Nullable
+  @Override
+  public NodeEntry getEntryForTesting(SkyKey key) {
+    return memoizingEvaluator.getExistingEntryForTesting(key);
+  }
 }