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);
+  }
 }
diff --git a/src/test/java/com/google/devtools/build/lib/analysis/BuildViewTest.java b/src/test/java/com/google/devtools/build/lib/analysis/BuildViewTest.java
index 34eba92..ba4ba4e 100644
--- a/src/test/java/com/google/devtools/build/lib/analysis/BuildViewTest.java
+++ b/src/test/java/com/google/devtools/build/lib/analysis/BuildViewTest.java
@@ -53,11 +53,9 @@
 import com.google.devtools.build.lib.util.Pair;
 import com.google.devtools.build.lib.vfs.Path;
 import com.google.devtools.build.lib.vfs.PathFragment;
-import com.google.devtools.build.skyframe.DeterministicInMemoryGraph;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.EventType;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.Listener;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.Order;
+import com.google.devtools.build.skyframe.NotifyingGraph.EventType;
+import com.google.devtools.build.skyframe.NotifyingGraph.Listener;
+import com.google.devtools.build.skyframe.NotifyingGraph.Order;
 import com.google.devtools.build.skyframe.SkyKey;
 import com.google.devtools.build.skyframe.TrackingAwaiter;
 
@@ -826,8 +824,7 @@
             }
           }
         };
-    NotifyingInMemoryGraph graph = new DeterministicInMemoryGraph(listener);
-    setGraphForTesting(graph);
+    injectGraphListenerForTesting(listener, /*deterministic=*/ true);
     reporter.removeHandler(failFastHandler);
     try {
       update("//foo:query", "//foo:zquery");
@@ -837,7 +834,6 @@
           .contains("Analysis of target '//foo:query' failed; build aborted");
     }
     TrackingAwaiter.INSTANCE.assertNoErrors();
-    graph.assertNoExceptions();
   }
 
   /**
diff --git a/src/test/java/com/google/devtools/build/lib/analysis/util/BuildViewTestBase.java b/src/test/java/com/google/devtools/build/lib/analysis/util/BuildViewTestBase.java
index 46b5164..be2f423 100644
--- a/src/test/java/com/google/devtools/build/lib/analysis/util/BuildViewTestBase.java
+++ b/src/test/java/com/google/devtools/build/lib/analysis/util/BuildViewTestBase.java
@@ -38,8 +38,9 @@
 import com.google.devtools.build.lib.util.Pair;
 import com.google.devtools.build.lib.vfs.Path;
 import com.google.devtools.build.lib.vfs.PathFragment;
+import com.google.devtools.build.skyframe.DeterministicGraph;
 import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph;
+import com.google.devtools.build.skyframe.NotifyingGraph.Listener;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -126,10 +127,11 @@
     assertContainsEvent("circular symlinks detected");
   }
 
-  protected void setGraphForTesting(NotifyingInMemoryGraph notifyingInMemoryGraph) {
+  protected void injectGraphListenerForTesting(Listener listener, boolean deterministic) {
     InMemoryMemoizingEvaluator memoizingEvaluator =
         (InMemoryMemoizingEvaluator) skyframeExecutor.getEvaluatorForTesting();
-    memoizingEvaluator.setGraphForTesting(notifyingInMemoryGraph);
+    memoizingEvaluator.injectGraphTransformerForTesting(
+        DeterministicGraph.makeTransformer(listener, deterministic));
   }
 
   protected void runTestForMultiCpuAnalysisFailure(String badCpu, String goodCpu) throws Exception {
diff --git a/src/test/java/com/google/devtools/build/skyframe/BUILD b/src/test/java/com/google/devtools/build/skyframe/BUILD
index 3eb83d8..fccede9 100644
--- a/src/test/java/com/google/devtools/build/skyframe/BUILD
+++ b/src/test/java/com/google/devtools/build/skyframe/BUILD
@@ -1,17 +1,12 @@
 TESTUTIL_FILES = [
-    "DeterministicInMemoryGraph.java",
-    "ErrorInfoSubject.java",
-    "ErrorInfoSubjectFactory.java",
-    "EvaluationResultSubject.java",
-    "EvaluationResultSubjectFactory.java",
-    "NotifyingInMemoryGraph.java",
     "TrackingAwaiter.java",
     "GraphTester.java",
     "GenericFunctionException.java",
     "SomeErrorException.java",
     "TrackingInvalidationReceiver.java",
     "WalkableGraphUtils.java",
-]
+    # Truth Subject, SubjectFactory, and Graph files.
+] + glob(["*Subject.java"]) + glob(["*SubjectFactory.java"]) + glob(["*Graph.java"])
 
 java_library(
     name = "testutil",
diff --git a/src/test/java/com/google/devtools/build/skyframe/DeterministicGraph.java b/src/test/java/com/google/devtools/build/skyframe/DeterministicGraph.java
new file mode 100644
index 0000000..8114977
--- /dev/null
+++ b/src/test/java/com/google/devtools/build/skyframe/DeterministicGraph.java
@@ -0,0 +1,132 @@
+// 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.common.base.Function;
+import com.google.common.collect.Iterables;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import javax.annotation.Nullable;
+
+/**
+ * {@link NotifyingGraph} that returns reverse deps, temporary direct deps, and the results of
+ * batch requests ordered alphabetically by sky key string representation.
+ */
+public class DeterministicGraph<TGraph extends ThinNodeQueryableGraph>
+    extends NotifyingGraph<TGraph> {
+  public static final Function<ThinNodeQueryableGraph, ProcessableGraph> MAKE_DETERMINISTIC =
+      new Function<ThinNodeQueryableGraph, ProcessableGraph>() {
+        @Override
+        public ProcessableGraph apply(ThinNodeQueryableGraph queryableGraph) {
+          if (queryableGraph instanceof InMemoryGraph) {
+            return new DeterministicInMemoryGraph((InMemoryGraph) queryableGraph);
+          } else {
+            return new DeterministicGraph<>(queryableGraph);
+          }
+        }
+      };
+
+  public static Function<ThinNodeQueryableGraph, ProcessableGraph> makeTransformer(
+      final Listener listener, boolean deterministic) {
+    if (deterministic) {
+      return new Function<ThinNodeQueryableGraph, ProcessableGraph>() {
+        @Override
+        public ProcessableGraph apply(ThinNodeQueryableGraph queryableGraph) {
+          if (queryableGraph instanceof InMemoryGraph) {
+            return new DeterministicInMemoryGraph((InMemoryGraph) queryableGraph, listener);
+          } else {
+            return new DeterministicGraph<>(queryableGraph, listener);
+          }
+        }
+      };
+    } else {
+      return NotifyingGraph.makeNotifyingTransformer(listener);
+    }
+  }
+
+  private static final Comparator<SkyKey> ALPHABETICAL_SKYKEY_COMPARATOR =
+      new Comparator<SkyKey>() {
+        @Override
+        public int compare(SkyKey o1, SkyKey o2) {
+          return o1.toString().compareTo(o2.toString());
+        }
+      };
+
+  DeterministicGraph(TGraph delegate, Listener listener) {
+    super(delegate, listener);
+  }
+
+  DeterministicGraph(TGraph delegate) {
+    super(delegate, NotifyingGraph.Listener.NULL_LISTENER);
+  }
+
+  @Nullable
+  @Override
+  protected DeterministicValueEntry wrapEntry(SkyKey key, @Nullable ThinNodeEntry entry) {
+    return entry == null ? null : new DeterministicValueEntry(key, entry);
+  }
+
+  private static Map<SkyKey, NodeEntry> makeDeterministic(Map<SkyKey, NodeEntry> map) {
+    Map<SkyKey, NodeEntry> result = new TreeMap<>(ALPHABETICAL_SKYKEY_COMPARATOR);
+    result.putAll(map);
+    return result;
+  }
+
+  @Override
+  public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
+    return makeDeterministic(super.getBatch(keys));
+  }
+
+  @Override
+  public Map<SkyKey, NodeEntry> createIfAbsentBatch(Iterable<SkyKey> keys) {
+    return makeDeterministic(super.createIfAbsentBatch(keys));
+  }
+
+  /**
+   * This class uses TreeSet to store reverse dependencies of NodeEntry. As a result all values are
+   * lexicographically sorted.
+   */
+  private class DeterministicValueEntry extends NotifyingNodeEntry {
+    private DeterministicValueEntry(SkyKey myKey, ThinNodeEntry delegate) {
+      super(myKey, delegate);
+    }
+
+    @Override
+    public synchronized Collection<SkyKey> getReverseDeps() {
+      TreeSet<SkyKey> result = new TreeSet<>(ALPHABETICAL_SKYKEY_COMPARATOR);
+      Iterables.addAll(result, super.getReverseDeps());
+      return result;
+    }
+
+    @Override
+    public synchronized Set<SkyKey> getInProgressReverseDeps() {
+      TreeSet<SkyKey> result = new TreeSet<>(ALPHABETICAL_SKYKEY_COMPARATOR);
+      result.addAll(super.getInProgressReverseDeps());
+      return result;
+    }
+
+    @Override
+    public synchronized Set<SkyKey> getTemporaryDirectDeps() {
+      TreeSet<SkyKey> result = new TreeSet<>(ALPHABETICAL_SKYKEY_COMPARATOR);
+      result.addAll(super.getTemporaryDirectDeps());
+      return result;
+    }
+  }
+}
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 f427b2c..9e16284 100644
--- a/src/test/java/com/google/devtools/build/skyframe/DeterministicInMemoryGraph.java
+++ b/src/test/java/com/google/devtools/build/skyframe/DeterministicInMemoryGraph.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,74 +13,35 @@
 // limitations under the License.
 package com.google.devtools.build.skyframe;
 
-import com.google.common.collect.Iterables;
-
-import java.util.Collection;
-import java.util.Comparator;
 import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
 
-/** {@link NotifyingInMemoryGraph} that returns reverse deps ordered alphabetically. */
-public class DeterministicInMemoryGraph extends NotifyingInMemoryGraph {
-  private static final Comparator<SkyKey> ALPHABETICAL_SKYKEY_COMPARATOR =
-      new Comparator<SkyKey>() {
-    @Override
-    public int compare(SkyKey o1, SkyKey o2) {
-      return o1.toString().compareTo(o2.toString());
-    }
-  };
+/**
+ * {@link DeterministicGraph} that implements the {@link InMemoryGraph} interface. Sadly, cannot be
+ * a {@link NotifyingInMemoryGraph} due to Java's forbidding multiple inheritance.
+ */
+class DeterministicInMemoryGraph extends DeterministicGraph<InMemoryGraph>
+    implements InMemoryGraph {
 
-  public DeterministicInMemoryGraph(Listener listener) {
-    super(listener);
+  DeterministicInMemoryGraph(InMemoryGraph delegate, Listener graphListener) {
+    super(delegate, graphListener);
   }
 
-  public DeterministicInMemoryGraph() {
-    super(Listener.NULL_LISTENER);
+  DeterministicInMemoryGraph(InMemoryGraph delegate) {
+    super(delegate);
   }
 
   @Override
-  protected DeterministicValueEntry getEntry(SkyKey key) {
-    return new DeterministicValueEntry(key);
+  public Map<SkyKey, SkyValue> getValues() {
+    return delegate.getValues();
   }
 
   @Override
-  public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
-    Map<SkyKey, NodeEntry> result = new TreeMap<>(ALPHABETICAL_SKYKEY_COMPARATOR);
-    result.putAll(super.getBatch(keys));
-    return result;
+  public Map<SkyKey, SkyValue> getDoneValues() {
+    return delegate.getDoneValues();
   }
 
-  /**
-   * This class uses TreeSet to store reverse dependencies of NodeEntry. As a result all values are
-   * lexicographically sorted.
-   */
-  private class DeterministicValueEntry extends NotifyingNodeEntry {
-    private DeterministicValueEntry(SkyKey myKey) {
-      super(myKey);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public synchronized Collection<SkyKey> getReverseDeps() {
-      TreeSet<SkyKey> result = new TreeSet<>(ALPHABETICAL_SKYKEY_COMPARATOR);
-      Iterables.addAll(result, super.getReverseDeps());
-      return result;
-    }
-
-    @Override
-    public synchronized Set<SkyKey> getInProgressReverseDeps() {
-      TreeSet<SkyKey> result = new TreeSet<>(ALPHABETICAL_SKYKEY_COMPARATOR);
-      result.addAll(buildingState.getReverseDepsToSignal());
-      return result;
-    }
-
-    @Override
-    public synchronized Set<SkyKey> getTemporaryDirectDeps() {
-      TreeSet<SkyKey> result = new TreeSet<>(ALPHABETICAL_SKYKEY_COMPARATOR);
-      result.addAll(super.getTemporaryDirectDeps());
-      return result;
-    }
+  @Override
+  public Map<SkyKey, NodeEntry> getAllValues() {
+    return delegate.getAllValues();
   }
 }
diff --git a/src/test/java/com/google/devtools/build/skyframe/EagerInvalidatorTest.java b/src/test/java/com/google/devtools/build/skyframe/EagerInvalidatorTest.java
index 49706c8..1976988 100644
--- a/src/test/java/com/google/devtools/build/skyframe/EagerInvalidatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/EagerInvalidatorTest.java
@@ -63,7 +63,7 @@
  */
 @RunWith(Enclosed.class)
 public class EagerInvalidatorTest {
-  protected InMemoryGraph graph;
+  protected InMemoryGraphImpl graph;
   protected GraphTester tester = new GraphTester();
   protected InvalidationState state = newInvalidationState();
   protected AtomicReference<InvalidatingNodeVisitor<?>> visitor = new AtomicReference<>();
@@ -211,7 +211,7 @@
         throw new UnsupportedOperationException();
       }
     };
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a");
     set("b", "b");
     tester.getOrCreate("ab").addDependency("a").addDependency("b")
@@ -256,7 +256,7 @@
 
     // Given a graph consisting of two nodes, "a" and "ab" such that "ab" depends on "a",
     // And given "ab" is in error,
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a");
     tester.getOrCreate("ab").addDependency("a").setHasError(true);
     eval(false, skyKey("ab"));
@@ -297,7 +297,7 @@
         throw new UnsupportedOperationException();
       }
     };
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     invalidateWithoutError(receiver, skyKey("a"));
     assertThat(invalidated).isEmpty();
     set("a", "a");
@@ -313,7 +313,7 @@
     WeakReference<HeavyValue> weakRef = new WeakReference<>(heavyValue);
     tester.set("a", heavyValue);
 
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     eval(false, key);
     invalidate(graph, null, key);
 
@@ -331,7 +331,7 @@
 
   @Test
   public void reverseDepsConsistent() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a");
     set("b", "b");
     set("c", "c");
@@ -371,7 +371,7 @@
 
   @Test
   public void interruptChild() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     int numValues = 50; // More values than the invalidator has threads.
     final SkyKey[] family = new SkyKey[numValues];
     final SkyKey child = GraphTester.skyKey("child");
@@ -435,8 +435,7 @@
     assertFalse(state.isEmpty());
     final Set<SkyKey> invalidated = Sets.newConcurrentHashSet();
     assertFalse(isInvalidated(parent));
-    SkyValue parentValue = graph.getValue(parent);
-    assertNotNull(parentValue);
+    assertNotNull(graph.get(parent).getValue());
     receiver = new EvaluationProgressReceiver() {
       @Override
       public void invalidated(SkyKey skyKey, InvalidationState state) {
@@ -510,7 +509,7 @@
     Random random = new Random(TestUtils.getRandomSeed());
     int graphSize = 1000;
     int tries = 5;
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey[] values = constructLargeGraph(graphSize);
     eval(/*keepGoing=*/false, values);
     final Thread mainThread = Thread.currentThread();
@@ -579,7 +578,7 @@
   }
 
   protected void setupInvalidatableGraph() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a");
     set("b", "b");
     tester.getOrCreate("ab").addDependency("a").addDependency("b").setComputedValue(CONCATENATE);
diff --git a/src/test/java/com/google/devtools/build/skyframe/InMemoryGraphConcurrencyTest.java b/src/test/java/com/google/devtools/build/skyframe/InMemoryGraphConcurrencyTest.java
index 28f0da3..18d6875 100644
--- a/src/test/java/com/google/devtools/build/skyframe/InMemoryGraphConcurrencyTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/InMemoryGraphConcurrencyTest.java
@@ -18,7 +18,7 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Concurrency tests for {@link InMemoryGraph}. */
+/** Concurrency tests for {@link InMemoryGraphImpl}. */
 @RunWith(JUnit4.class)
 public class InMemoryGraphConcurrencyTest extends GraphConcurrencyTest {
   private ProcessableGraph graph;
@@ -36,7 +36,7 @@
 
   @Override
   protected void makeGraph() {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
   }
 
   @Override
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 6257555..8ca696e 100644
--- a/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
@@ -51,9 +51,9 @@
 import com.google.devtools.build.skyframe.GraphTester.StringValue;
 import com.google.devtools.build.skyframe.GraphTester.TestFunction;
 import com.google.devtools.build.skyframe.GraphTester.ValueComputer;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.EventType;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.Listener;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.Order;
+import com.google.devtools.build.skyframe.NotifyingGraph.EventType;
+import com.google.devtools.build.skyframe.NotifyingGraph.Listener;
+import com.google.devtools.build.skyframe.NotifyingGraph.Order;
 import com.google.devtools.build.skyframe.SkyFunction.Environment;
 import com.google.devtools.build.skyframe.SkyFunctionException.Transience;
 
@@ -78,9 +78,7 @@
 
 import javax.annotation.Nullable;
 
-/**
- * Tests for {@link MemoizingEvaluator}.
- */
+/** Tests for {@link MemoizingEvaluator}.*/
 @RunWith(JUnit4.class)
 public class MemoizingEvaluatorTest {
 
@@ -701,33 +699,32 @@
   // should be notified that mid has been built.
   @Test
   public void alreadyAnalyzedBadTarget() throws Exception {
-    final SkyKey mid = GraphTester.toSkyKey("mid");
+    final SkyKey mid = GraphTester.toSkyKey("zzmid");
     final CountDownLatch valueSet = new CountDownLatch(1);
-    setGraphForTesting(
-        new NotifyingInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (!key.equals(mid)) {
-                  return;
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (!key.equals(mid)) {
+              return;
+            }
+            switch (type) {
+              case ADD_REVERSE_DEP:
+                if (context == null) {
+                  // Context is null when we are enqueuing this value as a top-level job.
+                  TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(valueSet, "value not set");
                 }
-                switch (type) {
-                  case ADD_REVERSE_DEP:
-                    if (context == null) {
-                      // Context is null when we are enqueuing this value as a top-level job.
-                      TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                          valueSet, "value not set");
-                    }
-                    break;
-                  case SET_VALUE:
-                    valueSet.countDown();
-                    break;
-                  default:
-                    break;
-                }
-              }
-            }));
-    SkyKey top = GraphTester.skyKey("top");
+                break;
+              case SET_VALUE:
+                valueSet.countDown();
+                break;
+              default:
+                break;
+            }
+          }
+        },
+        /*deterministic=*/ true);
+    SkyKey top = GraphTester.skyKey("aatop");
     tester.getOrCreate(top).addDependency(mid).setComputedValue(CONCATENATE);
     tester.getOrCreate(mid).setHasError(true);
     tester.eval(/*keepGoing=*/false, top, mid);
@@ -1050,7 +1047,7 @@
 
   @Test
   public void cycleAndSelfEdgeWithDirtyValueInSameGroup() throws Exception {
-    setGraphForTesting(new DeterministicInMemoryGraph());
+    makeGraphDeterministic();
     final SkyKey cycleKey1 = GraphTester.toSkyKey("zcycleKey1");
     final SkyKey cycleKey2 = GraphTester.toSkyKey("acycleKey2");
     tester.getOrCreate(cycleKey2).addDependency(cycleKey2).setComputedValue(CONCATENATE);
@@ -1187,25 +1184,25 @@
     final CountDownLatch errorThrown = new CountDownLatch(1);
     // We don't do anything on the first build.
     final AtomicBoolean secondBuild = new AtomicBoolean(false);
-    setGraphForTesting(
-        new DeterministicInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (!secondBuild.get()) {
-                  return;
-                }
-                if (key.equals(otherTop) && type == EventType.SIGNAL) {
-                  // otherTop is being signaled that dep1 is done. Tell the error value that it is
-                  // ready, then wait until the error is thrown, so that otherTop's builder is not
-                  // re-entered.
-                  valuesReady.countDown();
-                  TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                      errorThrown, "error not thrown");
-                  return;
-                }
-              }
-            }));
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (!secondBuild.get()) {
+              return;
+            }
+            if (key.equals(otherTop) && type == EventType.SIGNAL) {
+              // otherTop is being signaled that dep1 is done. Tell the error value that it is
+              // ready, then wait until the error is thrown, so that otherTop's builder is not
+              // re-entered.
+              valuesReady.countDown();
+              TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
+                  errorThrown, "error not thrown");
+              return;
+            }
+          }
+        },
+        /*deterministic=*/ true);
     final SkyKey dep1 = GraphTester.toSkyKey("dep1");
     tester.set(dep1, new StringValue("dep1"));
     final SkyKey dep2 = GraphTester.toSkyKey("dep2");
@@ -1228,7 +1225,7 @@
     // Mark dep1 changed, so otherTop will be dirty and request re-evaluation of dep1.
     tester.getOrCreate(dep1, /*markAsModified=*/true);
     SkyKey topKey = GraphTester.toSkyKey("top");
-    // Note that since DeterministicInMemoryGraph alphabetizes reverse deps, it is important that
+    // Note that since DeterministicGraph alphabetizes reverse deps, it is important that
     // "cycle2" comes before "top".
     final SkyKey cycle1Key = GraphTester.toSkyKey("cycle1");
     final SkyKey cycle2Key = GraphTester.toSkyKey("cycle2");
@@ -1773,12 +1770,10 @@
    * We are checking here that we are resilient to a race condition in which a value that is
    * checking its children for dirtiness is signaled by all of its children, putting it in a ready
    * state, before the thread has terminated. Optionally, one of its children may throw an error,
-   * shutting down the threadpool. This is similar to
-   * {@link ParallelEvaluatorTest#slowChildCleanup}: a child about to throw signals its parent and
-   * the parent's builder restarts itself before the exception is thrown. Here, the signaling
-   * happens while dirty dependencies are being checked, as opposed to during actual evaluation, but
-   * the principle is the same. We control the timing by blocking "top"'s registering itself on its
-   * deps.
+   * shutting down the threadpool. The essential race is that a child about to throw signals its
+   * parent and the parent's builder restarts itself before the exception is thrown. Here, the
+   * signaling happens while dirty dependencies are being checked. We control the timing by blocking
+   * "top"'s registering itself on its deps.
    */
   private void dirtyChildEnqueuesParentDuringCheckDependencies(final boolean throwError)
       throws Exception {
@@ -1791,33 +1786,32 @@
     final AtomicBoolean delayTopSignaling = new AtomicBoolean(false);
     final CountDownLatch topSignaled = new CountDownLatch(1);
     final CountDownLatch topRestartedBuild = new CountDownLatch(1);
-    setGraphForTesting(
-        new DeterministicInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(
-                  SkyKey key, EventType type, Order order, @Nullable Object context) {
-                if (!delayTopSignaling.get()) {
-                  return;
-                }
-                if (key.equals(top) && type == EventType.SIGNAL && order == Order.AFTER) {
-                  // top is signaled by firstKey (since slowAddingDep is blocking), so slowAddingDep
-                  // is now free to acknowledge top as a parent.
-                  topSignaled.countDown();
-                  return;
-                }
-                if (key.equals(slowAddingDep)
-                    && type == EventType.ADD_REVERSE_DEP
-                    && top.equals(context)
-                    && order == Order.BEFORE) {
-                  // If top is trying to declare a dep on slowAddingDep, wait until firstKey has
-                  // signaled top. Then this add dep will return DONE and top will be signaled,
-                  // making it ready, so it will be enqueued.
-                  TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                      topSignaled, "first key didn't signal top in time");
-                }
-              }
-            }));
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, @Nullable Object context) {
+            if (!delayTopSignaling.get()) {
+              return;
+            }
+            if (key.equals(top) && type == EventType.SIGNAL && order == Order.AFTER) {
+              // top is signaled by firstKey (since slowAddingDep is blocking), so slowAddingDep
+              // is now free to acknowledge top as a parent.
+              topSignaled.countDown();
+              return;
+            }
+            if (key.equals(slowAddingDep)
+                && type == EventType.ADD_REVERSE_DEP
+                && top.equals(context)
+                && order == Order.BEFORE) {
+              // If top is trying to declare a dep on slowAddingDep, wait until firstKey has
+              // signaled top. Then this add dep will return DONE and top will be signaled,
+              // making it ready, so it will be enqueued.
+              TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
+                  topSignaled, "first key didn't signal top in time");
+            }
+          }
+        },
+        /*deterministic=*/ true);
     // Value that is modified on the second build. Its thread won't finish until it signals top,
     // which will wait for the signal before it enqueues its next dep. We prevent the thread from
     // finishing by having the listener to which it reports its warning block until top's builder
@@ -1907,27 +1901,25 @@
     final CountDownLatch changedKeyStarted = new CountDownLatch(1);
     final CountDownLatch changedKeyCanFinish = new CountDownLatch(1);
     final AtomicBoolean controlTiming = new AtomicBoolean(false);
-    setGraphForTesting(
-        new NotifyingInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (!controlTiming.get()) {
-                  return;
-                }
-                if (key.equals(midKey)
-                    && type == EventType.CHECK_IF_DONE
-                    && order == Order.BEFORE) {
-                  TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                      changedKeyStarted, "changed key didn't start");
-                } else if (key.equals(changedKey)
-                    && type == EventType.REMOVE_REVERSE_DEP
-                    && order == Order.AFTER
-                    && midKey.equals(context)) {
-                  changedKeyCanFinish.countDown();
-                }
-              }
-            }));
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (!controlTiming.get()) {
+              return;
+            }
+            if (key.equals(midKey) && type == EventType.CHECK_IF_DONE && order == Order.BEFORE) {
+              TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
+                  changedKeyStarted, "changed key didn't start");
+            } else if (key.equals(changedKey)
+                && type == EventType.REMOVE_REVERSE_DEP
+                && order == Order.AFTER
+                && midKey.equals(context)) {
+              changedKeyCanFinish.countDown();
+            }
+          }
+        },
+        /*deterministic=*/ false);
     // Then top builds as expected.
     assertThat(tester.evalAndGet(/*keepGoing=*/ false, topKey)).isEqualTo(new StringValue("first"));
     // When changed is modified,
@@ -1988,24 +1980,24 @@
     // leaf4 should not built in the second build.
     final SkyKey leaf4 = GraphTester.toSkyKey("leaf4");
     final AtomicBoolean shouldNotBuildLeaf4 = new AtomicBoolean(false);
-    setGraphForTesting(
-        new NotifyingInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (shouldNotBuildLeaf4.get()
-                    && key.equals(leaf4)
-                    && type != EventType.REMOVE_REVERSE_DEP) {
-                  throw new IllegalStateException(
-                      "leaf4 should not have been considered this build: "
-                          + type
-                          + ", "
-                          + order
-                          + ", "
-                          + context);
-                }
-              }
-            }));
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (shouldNotBuildLeaf4.get()
+                && key.equals(leaf4)
+                && type != EventType.REMOVE_REVERSE_DEP) {
+              throw new IllegalStateException(
+                  "leaf4 should not have been considered this build: "
+                      + type
+                      + ", "
+                      + order
+                      + ", "
+                      + context);
+            }
+          }
+        },
+        /*deterministic=*/ false);
     tester.set(leaf4, new StringValue("leaf4"));
 
     // Create leaf0, leaf1 and leaf2 values with values "leaf2", "leaf3", "leaf4" respectively.
@@ -2097,39 +2089,39 @@
     // changed thread checks value entry once (to see if it is changed). dirty thread checks twice,
     // to see if it is changed, and if it is dirty.
     final CountDownLatch threadsStarted = new CountDownLatch(3);
-    setGraphForTesting(
-        new NotifyingInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (!blockingEnabled.get()) {
-                  return;
-                }
-                if (!key.equals(parent)) {
-                  return;
-                }
-                if (type == EventType.IS_CHANGED && order == Order.BEFORE) {
-                  threadsStarted.countDown();
-                }
-                // Dirtiness only checked by dirty thread.
-                if (type == EventType.IS_DIRTY && order == Order.BEFORE) {
-                  threadsStarted.countDown();
-                }
-                if (type == EventType.MARK_DIRTY) {
-                  TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                      threadsStarted, "Both threads did not query if value isChanged in time");
-                  boolean isChanged = (Boolean) context;
-                  if (order == Order.BEFORE && !isChanged) {
-                    TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                        waitForChanged, "'changed' thread did not mark value changed in time");
-                    return;
-                  }
-                  if (order == Order.AFTER && isChanged) {
-                    waitForChanged.countDown();
-                  }
-                }
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (!blockingEnabled.get()) {
+              return;
+            }
+            if (!key.equals(parent)) {
+              return;
+            }
+            if (type == EventType.IS_CHANGED && order == Order.BEFORE) {
+              threadsStarted.countDown();
+            }
+            // Dirtiness only checked by dirty thread.
+            if (type == EventType.IS_DIRTY && order == Order.BEFORE) {
+              threadsStarted.countDown();
+            }
+            if (type == EventType.MARK_DIRTY) {
+              TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
+                  threadsStarted, "Both threads did not query if value isChanged in time");
+              boolean isChanged = (Boolean) context;
+              if (order == Order.BEFORE && !isChanged) {
+                TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
+                    waitForChanged, "'changed' thread did not mark value changed in time");
+                return;
               }
-            }));
+              if (order == Order.AFTER && isChanged) {
+                waitForChanged.countDown();
+              }
+            }
+          }
+        },
+        /*deterministic=*/ false);
     SkyKey leaf = GraphTester.toSkyKey("leaf");
     tester.set(leaf, new StringValue("leaf"));
     tester.getOrCreate(parent).addDependency(leaf).setComputedValue(CONCATENATE);
@@ -2954,7 +2946,7 @@
   @Test
   public void errorOnlyBubblesToRequestingParents() throws Exception {
     // We need control over the order of reverse deps, so use a deterministic graph.
-    setGraphForTesting(new DeterministicInMemoryGraph());
+    makeGraphDeterministic();
     SkyKey errorKey = GraphTester.toSkyKey("error");
     tester.set(errorKey, new StringValue("biding time"));
     SkyKey slowKey = GraphTester.toSkyKey("slow");
@@ -3079,42 +3071,43 @@
    */
   @Test
   public void raceClearingIncompleteValues() throws Exception {
-    SkyKey topKey = GraphTester.toSkyKey("top");
-    final SkyKey midKey = GraphTester.toSkyKey("mid");
+    // Make sure top is enqueued before mid, to avoid a deadlock.
+    SkyKey topKey = GraphTester.toSkyKey("aatop");
+    final SkyKey midKey = GraphTester.toSkyKey("zzmid");
     SkyKey badKey = GraphTester.toSkyKey("bad");
     final AtomicBoolean waitForSecondCall = new AtomicBoolean(false);
     final CountDownLatch otherThreadWinning = new CountDownLatch(1);
     final AtomicReference<Thread> firstThread = new AtomicReference<>();
-    setGraphForTesting(
-        new NotifyingInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (!waitForSecondCall.get()) {
-                  return;
-                }
-                if (key.equals(midKey)) {
-                  if (type == EventType.CREATE_IF_ABSENT) {
-                    // The first thread to create midKey will not be the first thread to add a
-                    // reverse dep to it.
-                    firstThread.compareAndSet(null, Thread.currentThread());
-                    return;
-                  }
-                  if (type == EventType.ADD_REVERSE_DEP) {
-                    if (order == Order.BEFORE && Thread.currentThread().equals(firstThread.get())) {
-                      // If this thread created midKey, block until the other thread adds a dep on
-                      // it.
-                      TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                          otherThreadWinning, "other thread didn't pass this one");
-                    } else if (order == Order.AFTER
-                        && !Thread.currentThread().equals(firstThread.get())) {
-                      // This thread has added a dep. Allow the other thread to proceed.
-                      otherThreadWinning.countDown();
-                    }
-                  }
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (!waitForSecondCall.get()) {
+              return;
+            }
+            if (key.equals(midKey)) {
+              if (type == EventType.CREATE_IF_ABSENT) {
+                // The first thread to create midKey will not be the first thread to add a
+                // reverse dep to it.
+                firstThread.compareAndSet(null, Thread.currentThread());
+                return;
+              }
+              if (type == EventType.ADD_REVERSE_DEP) {
+                if (order == Order.BEFORE && Thread.currentThread().equals(firstThread.get())) {
+                  // If this thread created midKey, block until the other thread adds a dep on
+                  // it.
+                  TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
+                      otherThreadWinning, "other thread didn't pass this one");
+                } else if (order == Order.AFTER
+                    && !Thread.currentThread().equals(firstThread.get())) {
+                  // This thread has added a dep. Allow the other thread to proceed.
+                  otherThreadWinning.countDown();
                 }
               }
-            }));
+            }
+          }
+        },
+        /*deterministic=*/ true);
     tester.getOrCreate(topKey).addDependency(midKey).setComputedValue(CONCATENATE);
     tester.getOrCreate(midKey).addDependency(badKey).setComputedValue(CONCATENATE);
     tester.getOrCreate(badKey).setHasError(true);
@@ -3124,8 +3117,16 @@
     result = tester.eval(/*keepGoing=*/ true, topKey, midKey);
     assertNotNull(firstThread.get());
     assertEquals(0, otherThreadWinning.getCount());
-    assertThat(result.getError(midKey).getRootCauses()).containsExactly(badKey);
-    assertThat(result.getError(topKey).getRootCauses()).containsExactly(badKey);
+    assertThatEvaluationResult(result).hasErrorEntryForKeyThat(midKey).isNotNull();
+    assertThatEvaluationResult(result).hasErrorEntryForKeyThat(topKey).isNotNull();
+    if (rootCausesStored()) {
+      assertThatEvaluationResult(result)
+          .hasErrorEntryForKeyThat(midKey)
+          .rootCauseOfExceptionIs(badKey);
+      assertThatEvaluationResult(result)
+          .hasErrorEntryForKeyThat(topKey)
+          .rootCauseOfExceptionIs(badKey);
+    }
   }
 
   @Test
@@ -3220,18 +3221,18 @@
                 return null;
               }
             });
-    setGraphForTesting(
-        new NotifyingInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (key.equals(errorKey) && type == EventType.SET_VALUE && order == Order.AFTER) {
-                  errorCommitted.countDown();
-                  TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                      otherDone, "otherErrorKey's SkyFunction didn't finish in time.");
-                }
-              }
-            }));
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (key.equals(errorKey) && type == EventType.SET_VALUE && order == Order.AFTER) {
+              errorCommitted.countDown();
+              TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
+                  otherDone, "otherErrorKey's SkyFunction didn't finish in time.");
+            }
+          }
+        },
+        /*deterministic=*/ false);
 
     // When the graph is evaluated in noKeepGoing mode,
     EvaluationResult<StringValue> result =
@@ -3735,39 +3736,39 @@
     final AtomicBoolean synchronizeThreads = new AtomicBoolean(false);
     // We don't expect slow-set-value to actually be built, but if it is, we wait for it.
     final CountDownLatch slowBuilt = new CountDownLatch(1);
-    setGraphForTesting(
-        new DeterministicInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (!synchronizeThreads.get()) {
-                  return;
-                }
-                if (type == EventType.IS_DIRTY && key.equals(failingKey)) {
-                  // Wait for the build to abort or for the other node to incorrectly build.
-                  try {
-                    assertTrue(slowBuilt.await(TestUtils.WAIT_TIMEOUT_SECONDS, TimeUnit.SECONDS));
-                  } catch (InterruptedException e) {
-                    // This is ok, because it indicates the build is shutting down.
-                    Thread.currentThread().interrupt();
-                  }
-                } else if (type == EventType.SET_VALUE
-                    && key.equals(fastToRequestSlowToSetValueKey)
-                    && order == Order.AFTER) {
-                  // This indicates a problem -- this parent shouldn't be built since it depends on
-                  // an error.
-                  slowBuilt.countDown();
-                  // Before this node actually sets its value (and then throws an exception) we wait
-                  // for the other node to throw an exception.
-                  try {
-                    Thread.sleep(TestUtils.WAIT_TIMEOUT_MILLISECONDS);
-                    throw new IllegalStateException("uninterrupted in " + key);
-                  } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                  }
-                }
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (!synchronizeThreads.get()) {
+              return;
+            }
+            if (type == EventType.IS_DIRTY && key.equals(failingKey)) {
+              // Wait for the build to abort or for the other node to incorrectly build.
+              try {
+                assertTrue(slowBuilt.await(TestUtils.WAIT_TIMEOUT_SECONDS, TimeUnit.SECONDS));
+              } catch (InterruptedException e) {
+                // This is ok, because it indicates the build is shutting down.
+                Thread.currentThread().interrupt();
               }
-            }));
+            } else if (type == EventType.SET_VALUE
+                && key.equals(fastToRequestSlowToSetValueKey)
+                && order == Order.AFTER) {
+              // This indicates a problem -- this parent shouldn't be built since it depends on
+              // an error.
+              slowBuilt.countDown();
+              // Before this node actually sets its value (and then throws an exception) we wait
+              // for the other node to throw an exception.
+              try {
+                Thread.sleep(TestUtils.WAIT_TIMEOUT_MILLISECONDS);
+                throw new IllegalStateException("uninterrupted in " + key);
+              } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+              }
+            }
+          }
+        },
+        /*deterministic=*/ true);
     // Initialize graph.
     tester.eval(/*keepGoing=*/true, errorKey);
     tester.getOrCreate(invalidatedKey, /*markAsModified=*/true);
@@ -3807,61 +3808,58 @@
     // particular, we don't want to force anything during error bubbling.
     final AtomicBoolean synchronizeThreads = new AtomicBoolean(false);
     final CountDownLatch shutdownAwaiterStarted = new CountDownLatch(1);
-    setGraphForTesting(
-        new DeterministicInMemoryGraph(
-            new Listener() {
-              private final CountDownLatch cachedSignaled = new CountDownLatch(1);
+    injectGraphListenerForTesting(
+        new Listener() {
+          private final CountDownLatch cachedSignaled = new CountDownLatch(1);
 
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (!synchronizeThreads.get()
-                    || order != Order.BEFORE
-                    || type != EventType.SIGNAL) {
-                  return;
-                }
-                TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
-                    shutdownAwaiterStarted, "shutdown awaiter not started");
-                if (key.equals(uncachedParentKey)) {
-                  // When the uncached parent is first signaled by its changed dep, make sure that
-                  // we wait until the cached parent is signaled too.
-                  try {
-                    assertTrue(
-                        cachedSignaled.await(TestUtils.WAIT_TIMEOUT_SECONDS, TimeUnit.SECONDS));
-                  } catch (InterruptedException e) {
-                    // Before the relevant bug was fixed, this code was not interrupted, and the
-                    // uncached parent got to build, yielding an inconsistent state at a later point
-                    // during evaluation. With the bugfix, the cached parent is never signaled
-                    // before the evaluator shuts down, and so the above code is interrupted.
-                    Thread.currentThread().interrupt();
-                  }
-                } else if (key.equals(cachedParentKey)) {
-                  // This branch should never be reached by a well-behaved evaluator, since when the
-                  // error node is reached, the evaluator should shut down. However, we don't test
-                  // for that behavior here because that would be brittle and we expect that such an
-                  // evaluator will crash hard later on in any case.
-                  cachedSignaled.countDown();
-                  try {
-                    // Sleep until we're interrupted by the evaluator, so we know it's shutting
-                    // down.
-                    Thread.sleep(TestUtils.WAIT_TIMEOUT_MILLISECONDS);
-                    Thread currentThread = Thread.currentThread();
-                    throw new IllegalStateException(
-                        "no interruption in time in "
-                            + key
-                            + " for "
-                            + (currentThread.isInterrupted() ? "" : "un")
-                            + "interrupted "
-                            + currentThread
-                            + " with hash "
-                            + System.identityHashCode(currentThread)
-                            + " at "
-                            + System.currentTimeMillis());
-                  } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                  }
-                }
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (!synchronizeThreads.get() || order != Order.BEFORE || type != EventType.SIGNAL) {
+              return;
+            }
+            TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
+                shutdownAwaiterStarted, "shutdown awaiter not started");
+            if (key.equals(uncachedParentKey)) {
+              // When the uncached parent is first signaled by its changed dep, make sure that
+              // we wait until the cached parent is signaled too.
+              try {
+                assertTrue(cachedSignaled.await(TestUtils.WAIT_TIMEOUT_SECONDS, TimeUnit.SECONDS));
+              } catch (InterruptedException e) {
+                // Before the relevant bug was fixed, this code was not interrupted, and the
+                // uncached parent got to build, yielding an inconsistent state at a later point
+                // during evaluation. With the bugfix, the cached parent is never signaled
+                // before the evaluator shuts down, and so the above code is interrupted.
+                Thread.currentThread().interrupt();
               }
-            }));
+            } else if (key.equals(cachedParentKey)) {
+              // This branch should never be reached by a well-behaved evaluator, since when the
+              // error node is reached, the evaluator should shut down. However, we don't test
+              // for that behavior here because that would be brittle and we expect that such an
+              // evaluator will crash hard later on in any case.
+              cachedSignaled.countDown();
+              try {
+                // Sleep until we're interrupted by the evaluator, so we know it's shutting
+                // down.
+                Thread.sleep(TestUtils.WAIT_TIMEOUT_MILLISECONDS);
+                Thread currentThread = Thread.currentThread();
+                throw new IllegalStateException(
+                    "no interruption in time in "
+                        + key
+                        + " for "
+                        + (currentThread.isInterrupted() ? "" : "un")
+                        + "interrupted "
+                        + currentThread
+                        + " with hash "
+                        + System.identityHashCode(currentThread)
+                        + " at "
+                        + System.currentTimeMillis());
+              } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+              }
+            }
+          }
+        },
+        /*deterministic=*/ true);
     // Initialize graph.
     tester.eval(/*keepGoing=*/true, cachedParentKey, uncachedParentKey);
     tester.getOrCreate(invalidatedKey, /*markAsModified=*/true);
@@ -3934,9 +3932,13 @@
     tester.eval(/*keepGoing=*/false, parent2Key);
   }
 
-  private void setGraphForTesting(NotifyingInMemoryGraph notifyingInMemoryGraph) {
-    InMemoryMemoizingEvaluator memoizingEvaluator = (InMemoryMemoizingEvaluator) tester.evaluator;
-    memoizingEvaluator.setGraphForTesting(notifyingInMemoryGraph);
+  private void injectGraphListenerForTesting(Listener listener, boolean deterministic) {
+    tester.evaluator.injectGraphTransformerForTesting(
+        DeterministicGraph.makeTransformer(listener, deterministic));
+  }
+
+  private void makeGraphDeterministic() {
+    tester.evaluator.injectGraphTransformerForTesting(DeterministicGraph.MAKE_DETERMINISTIC);
   }
 
   private static final class PassThroughSelected implements ValueComputer {
@@ -4070,27 +4072,26 @@
     SkyKey inactiveKey = GraphTester.skyKey("inactive");
     final Thread mainThread = Thread.currentThread();
     final AtomicBoolean shouldInterrupt = new AtomicBoolean(false);
-    NotifyingInMemoryGraph graph =
-        new NotifyingInMemoryGraph(
-            new Listener() {
-              @Override
-              public void accept(SkyKey key, EventType type, Order order, Object context) {
-                if (shouldInterrupt.get()
-                    && key.equals(topKey)
-                    && type == EventType.IS_READY
-                    && order == Order.BEFORE) {
-                  mainThread.interrupt();
-                  shouldInterrupt.set(false);
-                  try {
-                    // Make sure threadpool propagates interrupt.
-                    Thread.sleep(TestUtils.WAIT_TIMEOUT_MILLISECONDS);
-                  } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                  }
-                }
+    injectGraphListenerForTesting(
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {
+            if (shouldInterrupt.get()
+                && key.equals(topKey)
+                && type == EventType.IS_READY
+                && order == Order.BEFORE) {
+              mainThread.interrupt();
+              shouldInterrupt.set(false);
+              try {
+                // Make sure threadpool propagates interrupt.
+                Thread.sleep(TestUtils.WAIT_TIMEOUT_MILLISECONDS);
+              } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
               }
-            });
-    setGraphForTesting(graph);
+            }
+          }
+        },
+        /*deterministic=*/ false);
     // When top depends on inactive,
     tester.getOrCreate(topKey).addDependency(inactiveKey).setComputedValue(COPY);
     StringValue val = new StringValue("inactive");
@@ -4113,13 +4114,13 @@
       // Expected.
     }
     // But inactive is still present,
-    assertThat(graph.get(inactiveKey)).isNotNull();
+    assertThat(tester.driver.getEntryForTesting(inactiveKey)).isNotNull();
     // And still dirty,
-    assertThat(graph.get(inactiveKey).isDirty()).isTrue();
+    assertThat(tester.driver.getEntryForTesting(inactiveKey).isDirty()).isTrue();
     // And re-evaluates successfully,
     assertThat(tester.evalAndGet(/*keepGoing=*/ true, inactiveKey)).isEqualTo(val);
     // But top is gone from the graph,
-    assertThat(graph.get(topKey)).isNull();
+    assertThat(tester.driver.getEntryForTesting(topKey)).isNull();
     // And we can successfully invalidate and re-evaluate inactive again.
     tester.getOrCreate(inactiveKey, /*markAsModified=*/ true);
     tester.invalidate();
diff --git a/src/test/java/com/google/devtools/build/skyframe/NotifyingGraph.java b/src/test/java/com/google/devtools/build/skyframe/NotifyingGraph.java
new file mode 100644
index 0000000..7e76b8f
--- /dev/null
+++ b/src/test/java/com/google/devtools/build/skyframe/NotifyingGraph.java
@@ -0,0 +1,275 @@
+// 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.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Maps.EntryTransformer;
+import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
+
+import java.util.Map;
+import java.util.Set;
+
+import javax.annotation.Nullable;
+
+/**
+ * Class that allows clients to be notified on each access of the graph. Clients can simply track
+ * accesses, or they can block to achieve desired synchronization. Clients should call
+ * {@link TrackingAwaiter#INSTANCE#assertNoErrors} at the end of tests in case exceptions were
+ * swallowed in async threads.
+ *
+ * <p>While this class nominally always implements a {@link ProcessableGraph}, it will throw if any
+ * of the methods in {@link ProcessableGraph} that are not in {@link ThinNodeQueryableGraph} are
+ * called on it and its {@link #delegate} is not a {@link ProcessableGraph}. This lack of type
+ * safety is so that a {@code NotifyingGraph} can be returned by {@link #makeNotifyingTransformer}
+ * and used in {@link MemoizingEvaluator#injectGraphTransformerForTesting}.
+ */
+public class NotifyingGraph<TGraph extends ThinNodeQueryableGraph> implements ProcessableGraph {
+  public static Function<ThinNodeQueryableGraph, ProcessableGraph> makeNotifyingTransformer(
+      final Listener listener) {
+    return new Function<ThinNodeQueryableGraph, ProcessableGraph>() {
+      @Nullable
+      @Override
+      public ProcessableGraph apply(ThinNodeQueryableGraph queryableGraph) {
+        if (queryableGraph instanceof InMemoryGraph) {
+          return new NotifyingInMemoryGraph((InMemoryGraph) queryableGraph, listener);
+        } else {
+          return new NotifyingGraph<>(queryableGraph, listener);
+        }
+      }
+    };
+  }
+
+  protected final TGraph delegate;
+  private final Listener graphListener;
+
+  private final EntryTransformer<SkyKey, ThinNodeEntry, NodeEntry> wrapEntry =
+      new EntryTransformer<SkyKey, ThinNodeEntry, NodeEntry>() {
+        @Nullable
+        @Override
+        public NotifyingNodeEntry transformEntry(SkyKey key, @Nullable ThinNodeEntry nodeEntry) {
+          return wrapEntry(key, nodeEntry);
+        }
+      };
+
+  NotifyingGraph(TGraph delegate, Listener graphListener) {
+    this.delegate = delegate;
+    this.graphListener = new ErrorRecordingDelegatingListener(graphListener);
+  }
+
+  private ProcessableGraph getProcessableDelegate() {
+    return (ProcessableGraph) delegate;
+  }
+
+  @Override
+  public void remove(SkyKey key) {
+    getProcessableDelegate().remove(key);
+  }
+
+  @Override
+  public Map<SkyKey, NodeEntry> createIfAbsentBatch(Iterable<SkyKey> keys) {
+    for (SkyKey key : keys) {
+      graphListener.accept(key, EventType.CREATE_IF_ABSENT, Order.BEFORE, null);
+    }
+    return Maps.transformEntries(getProcessableDelegate().createIfAbsentBatch(keys), wrapEntry);
+  }
+
+  @Override
+  public Map<SkyKey, NodeEntry> getBatch(Iterable<SkyKey> keys) {
+    if (delegate instanceof ProcessableGraph) {
+      return Maps.transformEntries(getProcessableDelegate().getBatch(keys), wrapEntry);
+    } else {
+      return Maps.transformEntries(delegate.getBatch(keys), wrapEntry);
+    }
+  }
+
+  @Nullable
+  @Override
+  public NodeEntry get(SkyKey key) {
+    return wrapEntry(key, getProcessableDelegate().get(key));
+  }
+
+  /** Subclasses should override if they wish to subclass NotifyingNodeEntry. */
+  @Nullable
+  protected NotifyingNodeEntry wrapEntry(SkyKey key, @Nullable ThinNodeEntry entry) {
+    return entry == null ? null : new NotifyingNodeEntry(key, entry);
+  }
+
+  /**
+   * Graph/value entry events that the receiver can be informed of. When writing tests, feel free to
+   * add additional events here if needed.
+   */
+  public enum EventType {
+    CREATE_IF_ABSENT,
+    ADD_REVERSE_DEP,
+    REMOVE_REVERSE_DEP,
+    SIGNAL,
+    SET_VALUE,
+    MARK_DIRTY,
+    MARK_CLEAN,
+    IS_CHANGED,
+    GET_VALUE_WITH_METADATA,
+    IS_DIRTY,
+    IS_READY,
+    CHECK_IF_DONE,
+    GET_ALL_DIRECT_DEPS_FOR_INCOMPLETE_NODE
+  }
+
+  /**
+   * Whether the given event is about to happen or has just happened. For some events, both will be
+   * published, for others, only one. When writing tests, if you need an additional one to be
+   * published, feel free to add it.
+   */
+  public enum Order {
+    BEFORE,
+    AFTER
+  }
+
+  /** Receiver to be informed when an event for a given key occurs. */
+  public interface Listener {
+    @ThreadSafe
+    void accept(SkyKey key, EventType type, Order order, Object context);
+
+    Listener NULL_LISTENER =
+        new Listener() {
+          @Override
+          public void accept(SkyKey key, EventType type, Order order, Object context) {}
+        };
+  }
+
+  private static class ErrorRecordingDelegatingListener implements Listener {
+    private final Listener delegate;
+
+    private ErrorRecordingDelegatingListener(Listener delegate) {
+      this.delegate = delegate;
+    }
+
+    @Override
+    public void accept(SkyKey key, EventType type, Order order, Object context) {
+      try {
+        delegate.accept(key, type, order, context);
+      } catch (Exception e) {
+        TrackingAwaiter.INSTANCE.injectExceptionAndMessage(
+            e, "In NotifyingGraph: " + Joiner.on(", ").join(key, type, order, context));
+        throw e;
+      }
+    }
+  }
+
+  /** {@link NodeEntry} that informs a {@link Listener} of various method calls. */
+  protected class NotifyingNodeEntry extends DelegatingNodeEntry {
+    private final SkyKey myKey;
+    private final ThinNodeEntry delegate;
+
+    protected NotifyingNodeEntry(SkyKey key, ThinNodeEntry delegate) {
+      myKey = key;
+      this.delegate = delegate;
+    }
+
+    @Override
+    protected NodeEntry getDelegate() {
+      return (NodeEntry) delegate;
+    }
+
+    @Override
+    protected ThinNodeEntry getThinDelegate() {
+      return delegate;
+    }
+
+    @Override
+    public DependencyState addReverseDepAndCheckIfDone(SkyKey reverseDep) {
+      graphListener.accept(myKey, EventType.ADD_REVERSE_DEP, Order.BEFORE, reverseDep);
+      DependencyState result = super.addReverseDepAndCheckIfDone(reverseDep);
+      graphListener.accept(myKey, EventType.ADD_REVERSE_DEP, Order.AFTER, reverseDep);
+      return result;
+    }
+
+    @Override
+    public void removeReverseDep(SkyKey reverseDep) {
+      graphListener.accept(myKey, EventType.REMOVE_REVERSE_DEP, Order.BEFORE, reverseDep);
+      super.removeReverseDep(reverseDep);
+      graphListener.accept(myKey, EventType.REMOVE_REVERSE_DEP, Order.AFTER, reverseDep);
+    }
+
+    @Override
+    public boolean signalDep(Version childVersion) {
+      graphListener.accept(myKey, EventType.SIGNAL, Order.BEFORE, childVersion);
+      boolean result = super.signalDep(childVersion);
+      graphListener.accept(myKey, EventType.SIGNAL, Order.AFTER, childVersion);
+      return result;
+    }
+
+    @Override
+    public Set<SkyKey> setValue(SkyValue value, Version version) {
+      graphListener.accept(myKey, EventType.SET_VALUE, Order.BEFORE, value);
+      Set<SkyKey> result = super.setValue(value, version);
+      graphListener.accept(myKey, EventType.SET_VALUE, Order.AFTER, value);
+      return result;
+    }
+
+    @Override
+    public MarkedDirtyResult markDirty(boolean isChanged) {
+      graphListener.accept(myKey, EventType.MARK_DIRTY, Order.BEFORE, isChanged);
+      MarkedDirtyResult result = super.markDirty(isChanged);
+      graphListener.accept(myKey, EventType.MARK_DIRTY, Order.AFTER, isChanged);
+      return result;
+    }
+
+    @Override
+    public Set<SkyKey> markClean() {
+      graphListener.accept(myKey, EventType.MARK_CLEAN, Order.BEFORE, this);
+      Set<SkyKey> result = super.markClean();
+      graphListener.accept(myKey, EventType.MARK_CLEAN, Order.AFTER, this);
+      return result;
+    }
+
+    @Override
+    public boolean isChanged() {
+      graphListener.accept(myKey, EventType.IS_CHANGED, Order.BEFORE, this);
+      return super.isChanged();
+    }
+
+    @Override
+    public boolean isDirty() {
+      graphListener.accept(myKey, EventType.IS_DIRTY, Order.BEFORE, this);
+      return super.isDirty();
+    }
+
+    @Override
+    public boolean isReady() {
+      graphListener.accept(myKey, EventType.IS_READY, Order.BEFORE, this);
+      return super.isReady();
+    }
+
+    @Override
+    public SkyValue getValueMaybeWithMetadata() {
+      graphListener.accept(myKey, EventType.GET_VALUE_WITH_METADATA, Order.BEFORE, this);
+      return super.getValueMaybeWithMetadata();
+    }
+
+    @Override
+    public DependencyState checkIfDoneForDirtyReverseDep(SkyKey reverseDep) {
+      graphListener.accept(myKey, EventType.CHECK_IF_DONE, Order.BEFORE, reverseDep);
+      return super.checkIfDoneForDirtyReverseDep(reverseDep);
+    }
+
+    @Override
+    public Iterable<SkyKey> getAllDirectDepsForIncompleteNode() {
+      graphListener.accept(
+          myKey, EventType.GET_ALL_DIRECT_DEPS_FOR_INCOMPLETE_NODE, Order.BEFORE, this);
+      return super.getAllDirectDepsForIncompleteNode();
+    }
+  }
+}
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 ce3ed50..9b90491 100644
--- a/src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.java
+++ b/src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.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,211 +13,26 @@
 // limitations under the License.
 package com.google.devtools.build.skyframe;
 
-import com.google.common.base.Joiner;
-import com.google.common.truth.Truth;
-import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
+import java.util.Map;
 
-import java.util.ArrayList;
-import java.util.Set;
-
-/**
- * Class that allows clients to be notified on each access of the graph. Clients can simply track
- * accesses, or they can block to achieve desired synchronization. Clients should call
- * {@link #assertNoExceptions} at the end of tests in case exceptions were swallowed in async
- * threads.
- */
-public class NotifyingInMemoryGraph extends InMemoryGraph {
-  private final Listener graphListener;
-  private final ArrayList<Exception> unexpectedExceptions = new ArrayList<>();
-
-  public NotifyingInMemoryGraph(Listener graphListener) {
-    this.graphListener = new ErrorRecordingDelegatingListener(graphListener);
+/** {@link NotifyingGraph} that additionally implements the {@link InMemoryGraph} interface. */
+class NotifyingInMemoryGraph extends NotifyingGraph<InMemoryGraph> implements InMemoryGraph {
+  NotifyingInMemoryGraph(InMemoryGraph delegate, Listener graphListener) {
+    super(delegate, graphListener);
   }
 
-  protected NodeEntry createIfAbsent(SkyKey key) {
-    graphListener.accept(key, EventType.CREATE_IF_ABSENT, Order.BEFORE, null);
-    NodeEntry newval = getEntry(key);
-    NodeEntry oldval = getNodeMap().putIfAbsent(key, newval);
-    return oldval == null ? newval : oldval;
+  @Override
+  public Map<SkyKey, SkyValue> getValues() {
+    return delegate.getValues();
   }
 
-  /**
-   * Should be called at end of test (ideally in an {@code @After} method) to assert that no
-   * exceptions were thrown during calls to the listener.
-   */
-  public void assertNoExceptions() {
-    Truth.assertThat(unexpectedExceptions).isEmpty();
+  @Override
+  public Map<SkyKey, SkyValue> getDoneValues() {
+    return delegate.getDoneValues();
   }
 
-  // Subclasses should override if they wish to subclass NotifyingNodeEntry.
-  protected NotifyingNodeEntry getEntry(SkyKey key) {
-    return new NotifyingNodeEntry(key);
-  }
-
-  /** Receiver to be informed when an event for a given key occurs. */
-  public interface Listener {
-    @ThreadSafe
-    void accept(SkyKey key, EventType type, Order order, Object context);
-
-    public static Listener NULL_LISTENER = new Listener() {
-      @Override
-      public void accept(SkyKey key, EventType type, Order order, Object context) {}
-    };
-  }
-
-  private class ErrorRecordingDelegatingListener implements Listener {
-    private final Listener delegate;
-
-    private ErrorRecordingDelegatingListener(Listener delegate) {
-      this.delegate = delegate;
-    }
-
-    @Override
-    public void accept(SkyKey key, EventType type, Order order, Object context) {
-      try {
-        delegate.accept(key, type, order, context);
-      } catch (Exception e) {
-        TrackingAwaiter.INSTANCE.injectExceptionAndMessage(e,
-            "In NotifyingInMemoryGraph: " + Joiner.on(", ").join(key, type, order, context));
-        throw e;
-      }
-    }
-  }
-  /**
-   * Graph/value entry events that the receiver can be informed of. When writing tests, feel free to
-   * add additional events here if needed.
-   */
-  public enum EventType {
-    CREATE_IF_ABSENT,
-    ADD_REVERSE_DEP,
-    REMOVE_REVERSE_DEP,
-    SIGNAL,
-    SET_VALUE,
-    MARK_DIRTY,
-    MARK_CLEAN,
-    IS_CHANGED,
-    GET_VALUE_WITH_METADATA,
-    IS_DIRTY,
-    IS_READY,
-    CHECK_IF_DONE,
-    GET_ALL_DIRECT_DEPS_FOR_INCOMPLETE_NODE
-  }
-
-  public enum Order {
-    BEFORE,
-    AFTER
-  }
-
-  /**
-   * Note that the methods in this class intentionally do not have the {@code synchronized}
-   * keyword! Each of them invokes the synchronized method on {@link InMemoryNodeEntry} it
-   * overrides, which provides the required synchronization for state owned by that base class.
-   *
-   * <p>These methods are not synchronized because several test cases control the flow of
-   * execution by blocking until notified by the callbacks executed in these methods. If these
-   * overrides were synchronized, they wouldn't get the chance to execute these callbacks.
-   */
-  protected class NotifyingNodeEntry extends InMemoryNodeEntry {
-    private final SkyKey myKey;
-
-    protected NotifyingNodeEntry(SkyKey key) {
-      myKey = key;
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public DependencyState addReverseDepAndCheckIfDone(SkyKey reverseDep) {
-      graphListener.accept(myKey, EventType.ADD_REVERSE_DEP, Order.BEFORE, reverseDep);
-      DependencyState result = super.addReverseDepAndCheckIfDone(reverseDep);
-      graphListener.accept(myKey, EventType.ADD_REVERSE_DEP, Order.AFTER, reverseDep);
-      return result;
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public void removeReverseDep(SkyKey reverseDep) {
-      graphListener.accept(myKey, EventType.REMOVE_REVERSE_DEP, Order.BEFORE, reverseDep);
-      super.removeReverseDep(reverseDep);
-      graphListener.accept(myKey, EventType.REMOVE_REVERSE_DEP, Order.AFTER, reverseDep);
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public boolean signalDep(Version childVersion) {
-      graphListener.accept(myKey, EventType.SIGNAL, Order.BEFORE, childVersion);
-      boolean result = super.signalDep(childVersion);
-      graphListener.accept(myKey, EventType.SIGNAL, Order.AFTER, childVersion);
-      return result;
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public Set<SkyKey> setValue(SkyValue value, Version version) {
-      graphListener.accept(myKey, EventType.SET_VALUE, Order.BEFORE, value);
-      Set<SkyKey> result = super.setValue(value, version);
-      graphListener.accept(myKey, EventType.SET_VALUE, Order.AFTER, value);
-      return result;
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public MarkedDirtyResult markDirty(boolean isChanged) {
-      graphListener.accept(myKey, EventType.MARK_DIRTY, Order.BEFORE, isChanged);
-      MarkedDirtyResult result = super.markDirty(isChanged);
-      graphListener.accept(myKey, EventType.MARK_DIRTY, Order.AFTER, isChanged);
-      return result;
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public Set<SkyKey> markClean() {
-      graphListener.accept(myKey, EventType.MARK_CLEAN, Order.BEFORE, this);
-      Set<SkyKey> result = super.markClean();
-      graphListener.accept(myKey, EventType.MARK_CLEAN, Order.AFTER, this);
-      return result;
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public boolean isChanged() {
-      graphListener.accept(myKey, EventType.IS_CHANGED, Order.BEFORE, this);
-      return super.isChanged();
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public boolean isDirty() {
-      graphListener.accept(myKey, EventType.IS_DIRTY, Order.BEFORE, this);
-      return super.isDirty();
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public boolean isReady() {
-      graphListener.accept(myKey, EventType.IS_READY, Order.BEFORE, this);
-      return super.isReady();
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public SkyValue getValueMaybeWithMetadata() {
-      graphListener.accept(myKey, EventType.GET_VALUE_WITH_METADATA, Order.BEFORE, this);
-      return super.getValueMaybeWithMetadata();
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public DependencyState checkIfDoneForDirtyReverseDep(SkyKey reverseDep) {
-      graphListener.accept(myKey, EventType.CHECK_IF_DONE, Order.BEFORE, reverseDep);
-      return super.checkIfDoneForDirtyReverseDep(reverseDep);
-    }
-
-    @SuppressWarnings("UnsynchronizedOverridesSynchronized") // See the class doc for details.
-    @Override
-    public Iterable<SkyKey> getAllDirectDepsForIncompleteNode() {
-      graphListener.accept(
-          myKey, EventType.GET_ALL_DIRECT_DEPS_FOR_INCOMPLETE_NODE, Order.BEFORE, this);
-      return super.getAllDirectDepsForIncompleteNode();
-    }
+  @Override
+  public Map<SkyKey, NodeEntry> getAllValues() {
+    return delegate.getAllValues();
   }
 }
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 41d9cdc..0b3d8fc 100644
--- a/src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java
@@ -43,9 +43,9 @@
 import com.google.devtools.build.lib.testutil.TestThread;
 import com.google.devtools.build.lib.testutil.TestUtils;
 import com.google.devtools.build.skyframe.GraphTester.StringValue;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.EventType;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.Listener;
-import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.Order;
+import com.google.devtools.build.skyframe.NotifyingGraph.EventType;
+import com.google.devtools.build.skyframe.NotifyingGraph.Listener;
+import com.google.devtools.build.skyframe.NotifyingGraph.Order;
 import com.google.devtools.build.skyframe.ParallelEvaluator.EventFilter;
 import com.google.devtools.build.skyframe.SkyFunctionException.Transience;
 
@@ -91,9 +91,6 @@
   @After
   public void assertNoTrackedErrors() {
     TrackingAwaiter.INSTANCE.assertNoErrors();
-    if (graph instanceof NotifyingInMemoryGraph) {
-      ((NotifyingInMemoryGraph) graph).assertNoExceptions();
-    }
   }
 
   private ParallelEvaluator makeEvaluator(
@@ -153,7 +150,7 @@
 
   @Test
   public void smoke() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a");
     set("b", "b");
     tester.getOrCreate("ab").addDependency("a").addDependency("b").setComputedValue(CONCATENATE);
@@ -238,7 +235,7 @@
   }
 
   private void runPartialResultOnInterruption(boolean buildFastFirst) throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     // Two runs for fastKey's builder and one for the start of waitKey's builder.
     final CountDownLatch allValuesReady = new CountDownLatch(3);
     final SkyKey waitKey = GraphTester.toSkyKey("wait");
@@ -345,9 +342,12 @@
     final Semaphore threadStarted = new Semaphore(0);
     final Semaphore threadInterrupted = new Semaphore(0);
     final String[] wasError = new String[] { null };
-    final ParallelEvaluator evaluator = makeEvaluator(new InMemoryGraph(),
-        ImmutableMap.of(GraphTester.NODE_TYPE, valueBuilderFactory.create(threadStarted, wasError)),
-        false);
+    final ParallelEvaluator evaluator =
+        makeEvaluator(
+            new InMemoryGraphImpl(),
+            ImmutableMap.of(
+                GraphTester.NODE_TYPE, valueBuilderFactory.create(threadStarted, wasError)),
+            false);
 
     Thread t = new Thread(new Runnable() {
         @Override
@@ -407,9 +407,9 @@
       }
     };
 
-    final ParallelEvaluator evaluator = makeEvaluator(new InMemoryGraph(),
-        ImmutableMap.of(GraphTester.NODE_TYPE, builder),
-        false);
+    final ParallelEvaluator evaluator =
+        makeEvaluator(
+            new InMemoryGraphImpl(), ImmutableMap.of(GraphTester.NODE_TYPE, builder), false);
 
     SkyKey valueToEval = GraphTester.toSkyKey("a");
     try {
@@ -423,7 +423,7 @@
 
   @Test
   public void simpleWarning() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a").setWarning("warning on 'a'");
     StringValue value = (StringValue) eval(false, GraphTester.toSkyKey("a"));
     assertEquals("a", value.getValue());
@@ -434,7 +434,7 @@
   /** Regression test: events from already-done value not replayed. */
   @Test
   public void eventFromDoneChildRecorded() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a").setWarning("warning on 'a'");
     SkyKey a = GraphTester.toSkyKey("a");
     SkyKey top = GraphTester.toSkyKey("top");
@@ -454,7 +454,7 @@
 
   @Test
   public void storedEventFilter() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey a = GraphTester.toSkyKey("a");
     final AtomicBoolean evaluated = new AtomicBoolean(false);
     tester.getOrCreate(a).setBuilder(new SkyFunction() {
@@ -505,7 +505,7 @@
 
   @Test
   public void shouldCreateErrorValueWithRootCause() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a");
     SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
     SkyKey errorKey = GraphTester.toSkyKey("error");
@@ -518,7 +518,7 @@
 
   @Test
   public void shouldBuildOneTarget() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     set("a", "a");
     set("b", "b");
     SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
@@ -552,7 +552,7 @@
   }
 
   private void catastrophicBuild(boolean keepGoing, boolean keepEdges) throws Exception {
-    graph = new InMemoryGraph(keepEdges);
+    graph = new InMemoryGraphImpl(keepEdges);
 
     SkyKey catastropheKey = GraphTester.toSkyKey("catastrophe");
     SkyKey otherKey = GraphTester.toSkyKey("someKey");
@@ -610,7 +610,7 @@
 
   @Test
   public void parentFailureDoesntAffectChild() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey parentKey = GraphTester.toSkyKey("parent");
     tester.getOrCreate(parentKey).setHasError(true);
     SkyKey childKey = GraphTester.toSkyKey("child");
@@ -629,7 +629,7 @@
 
   @Test
   public void newParentOfErrorShouldHaveError() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("error");
     tester.getOrCreate(errorKey).setHasError(true);
     ErrorInfo error = evalValueInError(errorKey);
@@ -642,7 +642,7 @@
 
   @Test
   public void errorTwoLevelsDeep() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey parentKey = GraphTester.toSkyKey("parent");
     SkyKey errorKey = GraphTester.toSkyKey("error");
     tester.getOrCreate(errorKey).setHasError(true);
@@ -654,7 +654,7 @@
 
   @Test
   public void valueNotUsedInFailFastErrorRecovery() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey topKey = GraphTester.toSkyKey("top");
     SkyKey recoveryKey = GraphTester.toSkyKey("midRecovery");
     SkyKey badKey = GraphTester.toSkyKey("bad");
@@ -678,7 +678,7 @@
 
   @Test
   public void multipleRootCauses() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey parentKey = GraphTester.toSkyKey("parent");
     SkyKey errorKey = GraphTester.toSkyKey("error");
     SkyKey errorKey2 = GraphTester.toSkyKey("error2");
@@ -697,7 +697,7 @@
 
   @Test
   public void rootCauseWithNoKeepGoing() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey parentKey = GraphTester.toSkyKey("parent");
     SkyKey errorKey = GraphTester.toSkyKey("error");
     tester.getOrCreate(errorKey).setHasError(true);
@@ -711,7 +711,7 @@
 
   @Test
   public void errorBubblesToParentsOfTopLevelValue() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey parentKey = GraphTester.toSkyKey("parent");
     final SkyKey errorKey = GraphTester.toSkyKey("error");
     final CountDownLatch latch = new CountDownLatch(1);
@@ -726,7 +726,7 @@
 
   @Test
   public void noKeepGoingAfterKeepGoingFails() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     tester.getOrCreate(errorKey).setHasError(true);
     SkyKey parentKey = GraphTester.toSkyKey("parent");
@@ -742,7 +742,7 @@
 
   @Test
   public void twoErrors() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey firstError = GraphTester.toSkyKey("error1");
     SkyKey secondError = GraphTester.toSkyKey("error2");
     CountDownLatch firstStart = new CountDownLatch(1);
@@ -764,7 +764,7 @@
 
   @Test
   public void simpleCycle() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     tester.getOrCreate(aKey).addDependency(bKey);
@@ -778,7 +778,7 @@
 
   @Test
   public void cycleWithHead() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey topKey = GraphTester.toSkyKey("top");
@@ -796,7 +796,7 @@
 
   @Test
   public void selfEdgeWithHead() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey topKey = GraphTester.toSkyKey("top");
     SkyKey midKey = GraphTester.toSkyKey("mid");
@@ -812,7 +812,7 @@
 
   @Test
   public void cycleWithKeepGoing() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey topKey = GraphTester.toSkyKey("top");
@@ -835,7 +835,7 @@
 
   @Test
   public void twoCycles() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey cKey = GraphTester.toSkyKey("c");
@@ -858,7 +858,7 @@
 
   @Test
   public void twoCyclesKeepGoing() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey cKey = GraphTester.toSkyKey("c");
@@ -879,7 +879,7 @@
 
   @Test
   public void triangleBelowHeadCycle() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey cKey = GraphTester.toSkyKey("c");
@@ -897,7 +897,7 @@
 
   @Test
   public void longCycle() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey cKey = GraphTester.toSkyKey("c");
@@ -915,7 +915,7 @@
 
   @Test
   public void cycleWithTail() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey cKey = GraphTester.toSkyKey("c");
@@ -936,7 +936,7 @@
   /** Regression test: "value cannot be ready in a cycle". */
   @Test
   public void selfEdgeWithExtraChildrenUnderCycle() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey cKey = GraphTester.toSkyKey("c");
@@ -954,7 +954,7 @@
   /** Regression test: "value cannot be ready in a cycle". */
   @Test
   public void cycleWithExtraChildrenUnderCycle() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey cKey = GraphTester.toSkyKey("c");
@@ -974,7 +974,7 @@
   /** Regression test: "value cannot be ready in a cycle". */
   @Test
   public void cycleAboveIndependentCycle() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey bKey = GraphTester.toSkyKey("b");
     SkyKey cKey = GraphTester.toSkyKey("c");
@@ -989,7 +989,7 @@
   }
 
   public void valueAboveCycleAndExceptionReportsException() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey aKey = GraphTester.toSkyKey("a");
     SkyKey errorKey = GraphTester.toSkyKey("error");
     SkyKey bKey = GraphTester.toSkyKey("b");
@@ -1006,7 +1006,7 @@
 
   @Test
   public void errorValueStored() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     tester.getOrCreate(errorKey).setHasError(true);
     EvaluationResult<StringValue> result = eval(false, ImmutableList.of(errorKey));
@@ -1030,7 +1030,7 @@
    */
   @Test
   public void manyCycles() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey topKey = GraphTester.toSkyKey("top");
     for (int i = 0; i < 100; i++) {
       SkyKey dep = GraphTester.toSkyKey(Integer.toString(i));
@@ -1048,7 +1048,7 @@
    */
   @Test
   public void manyPathsToCycle() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey topKey = GraphTester.toSkyKey("top");
     SkyKey midKey = GraphTester.toSkyKey("mid");
     SkyKey cycleKey = GraphTester.toSkyKey("cycle");
@@ -1089,7 +1089,7 @@
 
   @Test
   public void manyUnprocessedValuesInCycle() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey lastSelfKey = GraphTester.toSkyKey("lastSelf");
     SkyKey firstSelfKey = GraphTester.toSkyKey("firstSelf");
     SkyKey midSelfKey = GraphTester.toSkyKey("midSelf");
@@ -1135,7 +1135,7 @@
 
   @Test
   public void errorValueStoredWithKeepGoing() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     tester.getOrCreate(errorKey).setHasError(true);
     EvaluationResult<StringValue> result = eval(true, ImmutableList.of(errorKey));
@@ -1155,7 +1155,7 @@
 
   @Test
   public void continueWithErrorDep() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     tester.getOrCreate(errorKey).setHasError(true);
     tester.set("after", new StringValue("after"));
@@ -1174,7 +1174,7 @@
 
   @Test
   public void transformErrorDep() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     tester.getOrCreate(errorKey).setHasError(true);
     SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
@@ -1190,7 +1190,7 @@
 
   @Test
   public void transformErrorDepKeepGoing() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     tester.getOrCreate(errorKey).setHasError(true);
     SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
@@ -1206,7 +1206,7 @@
 
   @Test
   public void transformErrorDepOneLevelDownKeepGoing() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     tester.getOrCreate(errorKey).setHasError(true);
     tester.set("after", new StringValue("after"));
@@ -1224,7 +1224,7 @@
 
   @Test
   public void transformErrorDepOneLevelDownNoKeepGoing() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     tester.getOrCreate(errorKey).setHasError(true);
     tester.set("after", new StringValue("after"));
@@ -1246,7 +1246,7 @@
    */
   @Test
   public void cycleWithMultipleUnfinishedChildren() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     tester = new GraphTester();
     SkyKey cycleKey = GraphTester.toSkyKey("cycle");
     SkyKey midKey = GraphTester.toSkyKey("mid");
@@ -1276,7 +1276,7 @@
    * we should detect cycle.
    */
   private void cycleAndErrorInBubbleUp(boolean keepGoing) throws Exception {
-    graph = new DeterministicInMemoryGraph();
+    graph = new DeterministicGraph<>(new InMemoryGraphImpl());
     tester = new GraphTester();
     SkyKey errorKey = GraphTester.toSkyKey("error");
     SkyKey cycleKey = GraphTester.toSkyKey("cycle");
@@ -1323,7 +1323,7 @@
    */
   @Test
   public void cycleAndErrorAndOtherInBubbleUp() throws Exception {
-    graph = new DeterministicInMemoryGraph();
+    graph = new DeterministicGraph<>(new InMemoryGraphImpl());
     tester = new GraphTester();
     SkyKey errorKey = GraphTester.toSkyKey("error");
     SkyKey cycleKey = GraphTester.toSkyKey("cycle");
@@ -1366,7 +1366,7 @@
    * Here, we add an additional top-level key in error, just to mix it up.
    */
   private void cycleAndErrorAndError(boolean keepGoing) throws Exception {
-    graph = new DeterministicInMemoryGraph();
+    graph = new DeterministicGraph<>(new InMemoryGraphImpl());
     tester = new GraphTester();
     SkyKey errorKey = GraphTester.toSkyKey("error");
     SkyKey cycleKey = GraphTester.toSkyKey("cycle");
@@ -1447,8 +1447,7 @@
     ImmutableMap<SkyFunctionName, SkyFunction> skyFunctions = ImmutableMap.of(
         childType, new ChildFunction(),
         parentType, new ParentFunction());
-    ParallelEvaluator evaluator = makeEvaluator(new InMemoryGraph(),
-        skyFunctions, false);
+    ParallelEvaluator evaluator = makeEvaluator(new InMemoryGraphImpl(), skyFunctions, false);
 
     try {
       evaluator.eval(ImmutableList.of(SkyKey.create(parentType, "octodad")));
@@ -1467,7 +1466,7 @@
   }
 
   private void unexpectedErrorDep(boolean keepGoing) throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     final SomeOtherErrorException exception = new SomeOtherErrorException("error exception");
     tester.getOrCreate(errorKey).setBuilder(new SkyFunction() {
@@ -1506,7 +1505,7 @@
   }
 
   private void unexpectedErrorDepOneLevelDown(final boolean keepGoing) throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
     final SomeErrorException exception = new SomeErrorException("error exception");
     final SomeErrorException topException = new SomeErrorException("top exception");
@@ -1580,7 +1579,7 @@
    */
   private void sameDepInTwoGroups(final boolean sameFirst, final boolean twoCalls,
       final boolean valuesOrThrow) throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey topKey = GraphTester.toSkyKey("top");
     final List<SkyKey> leaves = new ArrayList<>();
     for (int i = 1; i <= 3; i++) {
@@ -1666,7 +1665,7 @@
   }
 
   private void getValuesOrThrowWithErrors(boolean keepGoing) throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey parentKey = GraphTester.toSkyKey("parent");
     final SkyKey errorDep = GraphTester.toSkyKey("errorChild");
     final SomeErrorException childExn = new SomeErrorException("child error");
@@ -1728,7 +1727,7 @@
 
   @Test
   public void duplicateCycles() throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey grandparentKey = GraphTester.toSkyKey("grandparent");
     SkyKey parentKey1 = GraphTester.toSkyKey("parent1");
     SkyKey parentKey2 = GraphTester.toSkyKey("parent2");
@@ -1830,7 +1829,7 @@
 
   public void runDepOnErrorHaltsNoKeepGoingBuildEagerly(boolean childErrorCached,
       final boolean handleChildError) throws Exception {
-    graph = new InMemoryGraph();
+    graph = new InMemoryGraphImpl();
     SkyKey parentKey = GraphTester.toSkyKey("parent");
     final SkyKey childKey = GraphTester.toSkyKey("child");
     tester.getOrCreate(childKey).setHasError(/*hasError=*/true);
@@ -2000,7 +1999,8 @@
       }
     });
     graph =
-        new NotifyingInMemoryGraph(
+        new NotifyingGraph<>(
+            new InMemoryGraphImpl(),
             new Listener() {
               @Override
               public void accept(SkyKey key, EventType type, Order order, Object context) {
@@ -2032,7 +2032,7 @@
 
   @Test
   public void cachedErrorsFromKeepGoingUsedOnNoKeepGoing() throws Exception {
-    graph = new DeterministicInMemoryGraph();
+    graph = new DeterministicGraph<>(new InMemoryGraphImpl());
     tester = new GraphTester();
     SkyKey errorKey = GraphTester.toSkyKey("error");
     SkyKey parent1Key = GraphTester.toSkyKey("parent1");
@@ -2052,7 +2052,7 @@
 
   @Test
   public void cachedTopLevelErrorsShouldHaltNoKeepGoingBuildEarly() throws Exception {
-    graph = new DeterministicInMemoryGraph();
+    graph = new DeterministicGraph<>(new InMemoryGraphImpl());
     tester = new GraphTester();
     SkyKey errorKey = GraphTester.toSkyKey("error");
     tester.getOrCreate(errorKey).setHasError(true);
@@ -2082,7 +2082,7 @@
 
   private void runUnhandledTransitiveErrors(boolean keepGoing,
       final boolean explicitlyPropagateError) throws Exception {
-    graph = new DeterministicInMemoryGraph();
+    graph = new DeterministicGraph<>(new InMemoryGraphImpl());
     tester = new GraphTester();
     SkyKey grandparentKey = GraphTester.toSkyKey("grandparent");
     final SkyKey parentKey = GraphTester.toSkyKey("parent");