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