aboutsummaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
authorGravatar Chloe Calvarin <ccalvarin@google.com>2016-10-05 22:52:55 +0000
committerGravatar Damien Martin-Guillerez <dmarting@google.com>2016-10-06 07:41:32 +0000
commit860b8d27724f6beeb67e19f102ddca00b143bee6 (patch)
tree139421c989295111ac452e159ec084f4a3397937
parentcddad444da9748f26b4326f8e2de13ad7f26239e (diff)
EvaluationProgressReceiver absorbs other progress trackers.
Remove DirtyKeyTracker and inflight node tracking as independent progress tracking bits of code, and instead use an internal-facing DirtyTrackingProgressReceiver wrapper of the user's EvaluationProgressReceiver. -- MOS_MIGRATED_REVID=135291924
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/DirtyKeyTracker.java43
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/DirtyKeyTrackerImpl.java40
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/DirtyTrackingProgressReceiver.java143
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/EagerInvalidator.java43
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java34
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/InvalidatingNodeVisitor.java55
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/NodeEntryVisitor.java42
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/NullDirtyKeyTrackerImpl.java37
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java55
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java19
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/SimpleCycleDetector.java11
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java34
-rw-r--r--src/test/java/com/google/devtools/build/skyframe/EagerInvalidatorTest.java93
-rw-r--r--src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java17
14 files changed, 287 insertions, 379 deletions
diff --git a/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTracker.java b/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTracker.java
deleted file mode 100644
index 9ff18d13a8..0000000000
--- a/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTracker.java
+++ /dev/null
@@ -1,43 +0,0 @@
-// 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.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
-
-import java.util.Set;
-
-/**
- * Interface for implementations that need to keep track of dirty SkyKeys.
- */
-public interface DirtyKeyTracker {
-
- /**
- * Marks the {@code skyKey} as dirty.
- */
- @ThreadSafe
- void dirty(SkyKey skyKey);
-
- /**
- * Marks the {@code skyKey} as not dirty.
- */
- @ThreadSafe
- void notDirty(SkyKey skyKey);
-
- /**
- * Returns the set of keys k for which there was a call to dirty(k) but not a subsequent call
- * to notDirty(k).
- */
- @ThreadSafe
- Set<SkyKey> getDirtyKeys();
-}
diff --git a/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTrackerImpl.java b/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTrackerImpl.java
deleted file mode 100644
index 4d4f76ab44..0000000000
--- a/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTrackerImpl.java
+++ /dev/null
@@ -1,40 +0,0 @@
-// 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.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-
-import java.util.Set;
-
-/** Encapsulates a thread-safe set of SkyKeys. */
-public class DirtyKeyTrackerImpl implements DirtyKeyTracker {
-
- private final Set<SkyKey> dirtyKeys = Sets.newConcurrentHashSet();
-
- @Override
- public void dirty(SkyKey skyKey) {
- dirtyKeys.add(skyKey);
- }
-
- @Override
- public void notDirty(SkyKey skyKey) {
- dirtyKeys.remove(skyKey);
- }
-
- @Override
- public Set<SkyKey> getDirtyKeys() {
- return ImmutableSet.copyOf(dirtyKeys);
- }
-}
diff --git a/src/main/java/com/google/devtools/build/skyframe/DirtyTrackingProgressReceiver.java b/src/main/java/com/google/devtools/build/skyframe/DirtyTrackingProgressReceiver.java
new file mode 100644
index 0000000000..a4899d62fb
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/DirtyTrackingProgressReceiver.java
@@ -0,0 +1,143 @@
+// 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.Supplier;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+/**
+ * A delegating {@link EvaluationProgressReceiver} that tracks inflight nodes, nodes which
+ * are being evaluated or scheduled for evaluation, and dirty nodes.
+ */
+public class DirtyTrackingProgressReceiver implements EvaluationProgressReceiver {
+
+ @Nullable private final EvaluationProgressReceiver progressReceiver;
+ private final Set<SkyKey> dirtyKeys = Sets.newConcurrentHashSet();
+ private Set<SkyKey> inflightKeys = Sets.newConcurrentHashSet();
+
+ public DirtyTrackingProgressReceiver(@Nullable EvaluationProgressReceiver progressReceiver) {
+ this.progressReceiver = progressReceiver;
+ }
+
+ /** Called when a node is injected into the graph, and not evaluated. */
+ protected void injected(SkyKey skyKey) {
+ // This node was never evaluated, but is now clean and need not be re-evaluated
+ inflightKeys.remove(skyKey);
+ removeFromDirtySet(skyKey);
+ }
+
+ @Override
+ public void invalidated(SkyKey skyKey, InvalidationState state) {
+ if (progressReceiver != null) {
+ progressReceiver.invalidated(skyKey, state);
+ }
+
+ switch (state) {
+ case DELETED:
+ // This key was removed from the graph, so no longer needs to be marked as dirty.
+ removeFromDirtySet(skyKey);
+ break;
+ case DIRTY:
+ addToDirtySet(skyKey);
+ break;
+ default:
+ throw new IllegalStateException(state.toString());
+ }
+ }
+
+ @Override
+ public void enqueueing(SkyKey skyKey) {
+ enqueueing(skyKey, false);
+ }
+
+ /**
+ * Called when a node was requested to be enqueued but wasn't because either an interrupt or
+ * an error (in nokeep_going mode) had occurred.
+ */
+ protected void enqueueAfterError(SkyKey skyKey) {
+ enqueueing(skyKey, true);
+ }
+
+ private void enqueueing(SkyKey skyKey, boolean afterError) {
+ // We unconditionally add the key to the set of in-flight nodes even if evaluation is never
+ // scheduled, because we still want to remove the previously created NodeEntry from the graph.
+ // Otherwise we would leave the graph in a weird state (wasteful garbage in the best case and
+ // inconsistent in the worst case).
+ boolean newlyEnqueued = inflightKeys.add(skyKey);
+ if (newlyEnqueued) {
+ // All nodes enqueued for evaluation will be either verified clean, re-evaluated, or cleaned
+ // up after being in-flight when an error happens in nokeep_going mode or in the event of an
+ // interrupt. In any of these cases, they won't be dirty anymore.
+ removeFromDirtySet(skyKey);
+ if (progressReceiver != null && !afterError) {
+ // Only tell the external listener the node was enqueued if no there was neither an error
+ // or interrupt.
+ progressReceiver.enqueueing(skyKey);
+ }
+ }
+ }
+
+ @Override
+ public void computed(SkyKey skyKey, long elapsedTimeNanos) {
+ if (progressReceiver != null) {
+ progressReceiver.computed(skyKey, elapsedTimeNanos);
+ }
+ }
+
+ @Override
+ public void evaluated(SkyKey skyKey, Supplier<SkyValue> valueSupplier,
+ EvaluationState state) {
+ if (progressReceiver != null) {
+ progressReceiver.evaluated(skyKey, valueSupplier, state);
+ }
+
+ // This key was either built or marked clean, so we can remove it from both the dirty and
+ // inflight nodes.
+ inflightKeys.remove(skyKey);
+ removeFromDirtySet(skyKey);
+ }
+
+ /** Returns if the key is enqueued for evaluation. */
+ protected boolean isInflight(SkyKey skyKey) {
+ return inflightKeys.contains(skyKey);
+ }
+
+ /** Returns the set of all keys that are enqueued for evaluation, and resets the set to empty. */
+ protected Set<SkyKey> getAndClearInflightKeys() {
+ Set<SkyKey> keys = inflightKeys;
+ inflightKeys = Sets.newConcurrentHashSet();
+ return keys;
+ }
+
+ /**
+ * Returns the set of all dirty keys that have not been enqueued.
+ * This is useful for garbage collection, where we would not want to remove dirty nodes that are
+ * needed for evaluation (in the downward transitive closure of the set of the evaluation's
+ * top level nodes).
+ */
+ protected Set<SkyKey> getUnenqueuedDirtyKeys(){
+ return ImmutableSet.copyOf(dirtyKeys);
+ }
+
+ protected void addToDirtySet(SkyKey skyKey) {
+ dirtyKeys.add(skyKey);
+ }
+
+ protected void removeFromDirtySet(SkyKey skyKey) {
+ dirtyKeys.remove(skyKey);
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/EagerInvalidator.java b/src/main/java/com/google/devtools/build/skyframe/EagerInvalidator.java
index bff685097d..3577230e3a 100644
--- a/src/main/java/com/google/devtools/build/skyframe/EagerInvalidator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/EagerInvalidator.java
@@ -44,14 +44,13 @@ public final class EagerInvalidator {
public static void delete(
InMemoryGraph graph,
Iterable<SkyKey> diff,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- boolean traverseGraph,
- DirtyKeyTracker dirtyKeyTracker)
+ boolean traverseGraph)
throws InterruptedException {
DeletingNodeVisitor visitor =
createDeletingVisitorIfNeeded(
- graph, diff, progressReceiver, state, traverseGraph, dirtyKeyTracker);
+ graph, diff, progressReceiver, state, traverseGraph);
if (visitor != null) {
visitor.run();
}
@@ -61,37 +60,32 @@ public final class EagerInvalidator {
static DeletingNodeVisitor createDeletingVisitorIfNeeded(
InMemoryGraph graph,
Iterable<SkyKey> diff,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- boolean traverseGraph,
- DirtyKeyTracker dirtyKeyTracker) {
+ boolean traverseGraph) {
state.update(diff);
return state.isEmpty() ? null
- : new DeletingNodeVisitor(graph, progressReceiver, state, traverseGraph,
- dirtyKeyTracker);
+ : new DeletingNodeVisitor(graph, progressReceiver, state, traverseGraph);
}
@Nullable
static DirtyingNodeVisitor createInvalidatingVisitorIfNeeded(
QueryableGraph graph,
Iterable<SkyKey> diff,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker,
Function<ExecutorParams, ? extends ExecutorService> executorFactory) {
state.update(diff);
return state.isEmpty() ? null
- : new DirtyingNodeVisitor(graph, progressReceiver, state, dirtyKeyTracker,
- executorFactory);
+ : new DirtyingNodeVisitor(graph, progressReceiver, state, executorFactory);
}
@Nullable
private static DirtyingNodeVisitor createInvalidatingVisitorIfNeeded(
QueryableGraph graph,
Iterable<SkyKey> diff,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker,
ForkJoinPool forkJoinPool,
boolean supportInterruptions,
ErrorHandler errorHandler) {
@@ -102,11 +96,9 @@ public final class EagerInvalidator {
graph,
progressReceiver,
state,
- dirtyKeyTracker,
forkJoinPool,
supportInterruptions);
}
-
/**
* Invalidates given values and their upward transitive closure in the graph if necessary, using
* an executor constructed with the provided factory.
@@ -114,14 +106,13 @@ public final class EagerInvalidator {
public static void invalidate(
QueryableGraph graph,
Iterable<SkyKey> diff,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker,
Function<ExecutorParams, ? extends ExecutorService> executorFactory)
throws InterruptedException {
DirtyingNodeVisitor visitor =
createInvalidatingVisitorIfNeeded(
- graph, diff, progressReceiver, state, dirtyKeyTracker, executorFactory);
+ graph, diff, progressReceiver, state, executorFactory);
if (visitor != null) {
visitor.run();
}
@@ -134,9 +125,8 @@ public final class EagerInvalidator {
public static void invalidate(
QueryableGraph graph,
Iterable<SkyKey> diff,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker,
ForkJoinPool forkJoinPool,
boolean supportInterruptions)
throws InterruptedException {
@@ -146,7 +136,6 @@ public final class EagerInvalidator {
diff,
progressReceiver,
state,
- dirtyKeyTracker,
forkJoinPool,
supportInterruptions,
ErrorHandler.NullHandler.INSTANCE);
@@ -159,12 +148,10 @@ public final class EagerInvalidator {
public static void invalidate(
QueryableGraph graph,
Iterable<SkyKey> diff,
- EvaluationProgressReceiver progressReceiver,
- InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker)
+ DirtyTrackingProgressReceiver progressReceiver,
+ InvalidationState state)
throws InterruptedException {
- invalidate(graph, diff, progressReceiver, state, dirtyKeyTracker,
- AbstractQueueVisitor.EXECUTOR_FACTORY);
+ invalidate(graph, diff, progressReceiver, state, AbstractQueueVisitor.EXECUTOR_FACTORY);
}
}
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 317c631581..c0f0ed478f 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
@@ -27,10 +27,8 @@ import com.google.devtools.build.skyframe.Differencer.Diff;
import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.DeletingInvalidationState;
import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.DirtyingInvalidationState;
import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.InvalidationState;
-import com.google.devtools.build.skyframe.ParallelEvaluator.Receiver;
import com.google.devtools.build.skyframe.QueryableGraph.Reason;
import java.io.PrintStream;
-import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashSet;
@@ -52,7 +50,7 @@ import javax.annotation.Nullable;
public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
private final ImmutableMap<SkyFunctionName, ? extends SkyFunction> skyFunctions;
- @Nullable private final EvaluationProgressReceiver progressReceiver;
+ private final DirtyTrackingProgressReceiver progressReceiver;
// Not final only for testing.
private InMemoryGraph graph;
private IntVersion lastGraphVersion = null;
@@ -61,7 +59,6 @@ public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
private Set<SkyKey> valuesToDelete = new LinkedHashSet<>();
private Set<SkyKey> valuesToDirty = new LinkedHashSet<>();
private Map<SkyKey, SkyValue> valuesToInject = new HashMap<>();
- private final DirtyKeyTracker dirtyKeyTracker = new DirtyKeyTrackerImpl();
private final InvalidationState deleterState = new DeletingInvalidationState();
private final Differencer differencer;
@@ -97,7 +94,7 @@ public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
boolean keepEdges) {
this.skyFunctions = ImmutableMap.copyOf(skyFunctions);
this.differencer = Preconditions.checkNotNull(differencer);
- this.progressReceiver = progressReceiver;
+ this.progressReceiver = new DirtyTrackingProgressReceiver(progressReceiver);
this.graph = new InMemoryGraphImpl(keepEdges);
this.emittedEventState = emittedEventState;
this.keepEdges = keepEdges;
@@ -127,7 +124,7 @@ public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
Preconditions.checkArgument(versionAgeLimit >= 0);
final Version threshold = IntVersion.of(lastGraphVersion.getVal() - versionAgeLimit);
valuesToDelete.addAll(
- Sets.filter(dirtyKeyTracker.getDirtyKeys(), new Predicate<SkyKey>() {
+ Sets.filter(progressReceiver.getUnenqueuedDirtyKeys(), new Predicate<SkyKey>() {
@Override
public boolean apply(SkyKey skyKey) {
NodeEntry entry = graph.get(null, Reason.OTHER, skyKey);
@@ -150,6 +147,9 @@ public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
lastGraphVersion, version);
setAndCheckEvaluateState(true, roots);
try {
+ // Mark for removal any inflight nodes from the previous evaluation.
+ valuesToDelete.addAll(progressReceiver.getAndClearInflightKeys());
+
// The RecordingDifferencer implementation is not quite working as it should be at this point.
// It clears the internal data structures after getDiff is called and will not return
// diffs for historical versions. This makes the following code sensitive to interrupts.
@@ -164,16 +164,6 @@ public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
performInvalidation();
injectValues(intVersion);
- // We must delete all nodes that are still in-flight at the end of the evaluation (in case the
- // evaluation is aborted for some reason). In order to quickly return control to the caller,
- // we store the set of such nodes for deletion at the start of the next evaluation.
- Receiver<Collection<SkyKey>> lazyDeletingReceiver =
- new Receiver<Collection<SkyKey>>() {
- @Override
- public void accept(Collection<SkyKey> skyKeys) {
- valuesToDelete.addAll(skyKeys);
- }
- };
ParallelEvaluator evaluator =
new ParallelEvaluator(
graph,
@@ -184,9 +174,7 @@ public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
DEFAULT_STORED_EVENT_FILTER,
keepGoing,
numThreads,
- progressReceiver,
- dirtyKeyTracker,
- lazyDeletingReceiver);
+ progressReceiver);
EvaluationResult<T> result = evaluator.eval(roots);
return EvaluationResult.<T>builder()
.mergeFrom(result)
@@ -235,7 +223,7 @@ public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
return;
}
try {
- ParallelEvaluator.injectValues(valuesToInject, version, graph, dirtyKeyTracker);
+ ParallelEvaluator.injectValues(valuesToInject, version, graph, progressReceiver);
} catch (InterruptedException e) {
throw new IllegalStateException("InMemoryGraph doesn't throw interrupts", e);
}
@@ -244,15 +232,13 @@ public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
}
private void performInvalidation() throws InterruptedException {
- EagerInvalidator.delete(graph, valuesToDelete, progressReceiver, deleterState, keepEdges,
- dirtyKeyTracker);
+ EagerInvalidator.delete(graph, valuesToDelete, progressReceiver, deleterState, keepEdges);
// Note that clearing the valuesToDelete would not do an internal resizing. Therefore, if any
// build has a large set of dirty values, subsequent operations (even clearing) will be slower.
// Instead, just start afresh with a new LinkedHashSet.
valuesToDelete = new LinkedHashSet<>();
- EagerInvalidator.invalidate(graph, valuesToDirty, progressReceiver, invalidatorState,
- dirtyKeyTracker);
+ EagerInvalidator.invalidate(graph, valuesToDirty, progressReceiver, invalidatorState);
// Ditto.
valuesToDirty = new LinkedHashSet<>();
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/InvalidatingNodeVisitor.java b/src/main/java/com/google/devtools/build/skyframe/InvalidatingNodeVisitor.java
index 5a7f989f5b..577639a618 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InvalidatingNodeVisitor.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InvalidatingNodeVisitor.java
@@ -80,26 +80,23 @@ public abstract class InvalidatingNodeVisitor<TGraph extends QueryableGraph> {
};
protected final TGraph graph;
- @Nullable protected final EvaluationProgressReceiver progressReceiver;
- protected final DirtyKeyTracker dirtyKeyTracker;
+ protected final DirtyTrackingProgressReceiver progressReceiver;
// Aliased to InvalidationState.pendingVisitations.
protected final Set<Pair<SkyKey, InvalidationType>> pendingVisitations;
protected final QuiescingExecutor executor;
protected InvalidatingNodeVisitor(
TGraph graph,
- @Nullable EvaluationProgressReceiver progressReceiver,
- InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker) {
+ DirtyTrackingProgressReceiver progressReceiver,
+ InvalidationState state) {
this(
- graph, progressReceiver, state, dirtyKeyTracker, AbstractQueueVisitor.EXECUTOR_FACTORY);
+ graph, progressReceiver, state, AbstractQueueVisitor.EXECUTOR_FACTORY);
}
protected InvalidatingNodeVisitor(
TGraph graph,
- @Nullable EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker,
Function<ExecutorParams, ? extends ExecutorService> executorFactory) {
this.executor =
new AbstractQueueVisitor(
@@ -112,24 +109,21 @@ public abstract class InvalidatingNodeVisitor<TGraph extends QueryableGraph> {
executorFactory,
errorClassifier);
this.graph = Preconditions.checkNotNull(graph);
- this.progressReceiver = progressReceiver;
- this.dirtyKeyTracker = Preconditions.checkNotNull(dirtyKeyTracker);
+ this.progressReceiver = Preconditions.checkNotNull(progressReceiver);
this.pendingVisitations = state.pendingValues;
}
protected InvalidatingNodeVisitor(
TGraph graph,
- @Nullable EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker,
ForkJoinPool forkJoinPool) {
this.executor = ForkJoinQuiescingExecutor.newBuilder()
.withOwnershipOf(forkJoinPool)
.setErrorClassifier(errorClassifier)
.build();
this.graph = Preconditions.checkNotNull(graph);
- this.progressReceiver = progressReceiver;
- this.dirtyKeyTracker = Preconditions.checkNotNull(dirtyKeyTracker);
+ this.progressReceiver = Preconditions.checkNotNull(progressReceiver);
this.pendingVisitations = state.pendingValues;
}
@@ -162,13 +156,6 @@ public abstract class InvalidatingNodeVisitor<TGraph extends QueryableGraph> {
return executor.getInterruptionLatchForTestingOnly();
}
- protected void informProgressReceiver(SkyKey key,
- EvaluationProgressReceiver.InvalidationState state) {
- if (progressReceiver != null) {
- progressReceiver.invalidated(key, state);
- }
- }
-
/** Enqueues nodes for invalidation. Elements of {@code keys} may not exist in the graph. */
@ThreadSafe
abstract void visit(Iterable<SkyKey> keys, InvalidationType invalidationType);
@@ -242,11 +229,10 @@ public abstract class InvalidatingNodeVisitor<TGraph extends QueryableGraph> {
DeletingNodeVisitor(
InMemoryGraph graph,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- boolean traverseGraph,
- DirtyKeyTracker dirtyKeyTracker) {
- super(graph, progressReceiver, state, dirtyKeyTracker);
+ boolean traverseGraph) {
+ super(graph, progressReceiver, state);
this.traverseGraph = traverseGraph;
}
@@ -353,11 +339,10 @@ public abstract class InvalidatingNodeVisitor<TGraph extends QueryableGraph> {
}
// Allow custom key-specific logic to update dirtiness status.
- informProgressReceiver(
- key, EvaluationProgressReceiver.InvalidationState.DELETED);
+ progressReceiver.invalidated(key,
+ EvaluationProgressReceiver.InvalidationState.DELETED);
// Actually remove the node.
graph.remove(key);
- dirtyKeyTracker.notDirty(key);
// Remove the node from the set as the last operation.
pendingVisitations.remove(invalidationPair);
@@ -382,11 +367,10 @@ public abstract class InvalidatingNodeVisitor<TGraph extends QueryableGraph> {
protected DirtyingNodeVisitor(
QueryableGraph graph,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker,
Function<ExecutorParams, ? extends ExecutorService> executorFactory) {
- super(graph, progressReceiver, state, dirtyKeyTracker, executorFactory);
+ super(graph, progressReceiver, state, executorFactory);
this.supportInterruptions = true;
}
@@ -396,12 +380,11 @@ public abstract class InvalidatingNodeVisitor<TGraph extends QueryableGraph> {
*/
protected DirtyingNodeVisitor(
QueryableGraph graph,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
InvalidationState state,
- DirtyKeyTracker dirtyKeyTracker,
ForkJoinPool forkJoinPool,
boolean supportInterruptions) {
- super(graph, progressReceiver, state, dirtyKeyTracker, forkJoinPool);
+ super(graph, progressReceiver, state, forkJoinPool);
this.supportInterruptions = supportInterruptions;
}
@@ -526,8 +509,8 @@ public abstract class InvalidatingNodeVisitor<TGraph extends QueryableGraph> {
// only be marked dirty (because only a dependency of theirs has changed).
visit(markedDirtyResult.getReverseDepsUnsafe(), InvalidationType.DIRTIED, key);
- informProgressReceiver(key, EvaluationProgressReceiver.InvalidationState.DIRTY);
- dirtyKeyTracker.dirty(key);
+ progressReceiver.invalidated(key,
+ EvaluationProgressReceiver.InvalidationState.DIRTY);
// Remove the node from the set as the last operation.
if (supportInterruptions) {
pendingVisitations.remove(Pair.of(key, invalidationType));
diff --git a/src/main/java/com/google/devtools/build/skyframe/NodeEntryVisitor.java b/src/main/java/com/google/devtools/build/skyframe/NodeEntryVisitor.java
index c4cb73045c..722410e77b 100644
--- a/src/main/java/com/google/devtools/build/skyframe/NodeEntryVisitor.java
+++ b/src/main/java/com/google/devtools/build/skyframe/NodeEntryVisitor.java
@@ -52,10 +52,8 @@ class NodeEntryVisitor {
private final QuiescingExecutor quiescingExecutor;
private final AtomicBoolean preventNewEvaluations = new AtomicBoolean(false);
- private final Set<SkyKey> inflightNodes = Sets.newConcurrentHashSet();
private final Set<RuntimeException> crashes = Sets.newConcurrentHashSet();
- private final DirtyKeyTracker dirtyKeyTracker;
- private final EvaluationProgressReceiver progressReceiver;
+ private final DirtyTrackingProgressReceiver progressReceiver;
/**
* Function that allows this visitor to execute the appropriate {@link Runnable} when given a
* {@link SkyKey} to evaluate.
@@ -64,22 +62,19 @@ class NodeEntryVisitor {
NodeEntryVisitor(
ForkJoinPool forkJoinPool,
- DirtyKeyTracker dirtyKeyTracker,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
Function<SkyKey, Runnable> runnableMaker) {
this.quiescingExecutor = ForkJoinQuiescingExecutor.newBuilder()
.withOwnershipOf(forkJoinPool)
.setErrorClassifier(NODE_ENTRY_VISITOR_ERROR_CLASSIFIER)
.build();
- this.dirtyKeyTracker = dirtyKeyTracker;
this.progressReceiver = progressReceiver;
this.runnableMaker = runnableMaker;
}
NodeEntryVisitor(
int threadCount,
- DirtyKeyTracker dirtyKeyTracker,
- EvaluationProgressReceiver progressReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
Function<SkyKey, Runnable> runnableMaker) {
quiescingExecutor =
new AbstractQueueVisitor(
@@ -90,7 +85,6 @@ class NodeEntryVisitor {
/*failFastOnException*/ true,
"skyframe-evaluator",
NODE_ENTRY_VISITOR_ERROR_CLASSIFIER);
- this.dirtyKeyTracker = dirtyKeyTracker;
this.progressReceiver = progressReceiver;
this.runnableMaker = runnableMaker;
}
@@ -100,23 +94,13 @@ class NodeEntryVisitor {
}
void enqueueEvaluation(SkyKey key) {
- // We unconditionally add the key to the set of in-flight nodes because even if evaluation is
- // never scheduled we still want to remove the previously created NodeEntry from the graph.
- // Otherwise we would leave the graph in a weird state (wasteful garbage in the best case and
- // inconsistent in the worst case).
- boolean newlyEnqueued = inflightNodes.add(key);
- // All nodes enqueued for evaluation will be either verified clean, re-evaluated, or cleaned
- // up after being in-flight when an error happens in nokeep_going mode or in the event of an
- // interrupt. In any of these cases, they won't be dirty anymore.
- if (newlyEnqueued) {
- dirtyKeyTracker.notDirty(key);
- }
if (preventNewEvaluations.get()) {
+ // If an error happens in nokeep_going mode, we still want to mark these nodes as inflight,
+ // otherwise cleanup will not happen properly.
+ progressReceiver.enqueueAfterError(key);
return;
}
- if (newlyEnqueued && progressReceiver != null) {
- progressReceiver.enqueueing(key);
- }
+ progressReceiver.enqueueing(key);
quiescingExecutor.execute(runnableMaker.apply(key));
}
@@ -137,18 +121,6 @@ class NodeEntryVisitor {
return crashes;
}
- void notifyDone(SkyKey key) {
- inflightNodes.remove(key);
- }
-
- boolean isInflight(SkyKey key) {
- return inflightNodes.contains(key);
- }
-
- Set<SkyKey> getInflightNodes() {
- return inflightNodes;
- }
-
@VisibleForTesting
CountDownLatch getExceptionLatchForTestingOnly() {
return quiescingExecutor.getExceptionLatchForTestingOnly();
diff --git a/src/main/java/com/google/devtools/build/skyframe/NullDirtyKeyTrackerImpl.java b/src/main/java/com/google/devtools/build/skyframe/NullDirtyKeyTrackerImpl.java
deleted file mode 100644
index 1fff93c916..0000000000
--- a/src/main/java/com/google/devtools/build/skyframe/NullDirtyKeyTrackerImpl.java
+++ /dev/null
@@ -1,37 +0,0 @@
-// 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.collect.ImmutableSet;
-
-import java.util.Set;
-
-/**
- * Tracks nothing. Should be used by evaluators that don't do dirty node garbage collection.
- */
-public class NullDirtyKeyTrackerImpl implements DirtyKeyTracker {
-
- @Override
- public void dirty(SkyKey skyKey) {
- }
-
- @Override
- public void notDirty(SkyKey skyKey) {
- }
-
- @Override
- public Set<SkyKey> getDirtyKeys() {
- return ImmutableSet.of();
- }
-}
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 8cfe1f6e59..ee3ac7290b 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -90,9 +90,6 @@ public final class ParallelEvaluator implements Evaluator {
void accept(T object);
}
- private final DirtyKeyTracker dirtyKeyTracker;
- private final Receiver<Collection<SkyKey>> inflightKeysReceiver;
-
private final ParallelEvaluatorContext evaluatorContext;
private final CycleDetector cycleDetector;
@@ -105,12 +102,8 @@ public final class ParallelEvaluator implements Evaluator {
EventFilter storedEventFilter,
boolean keepGoing,
int threadCount,
- @Nullable EvaluationProgressReceiver progressReceiver,
- DirtyKeyTracker dirtyKeyTracker,
- Receiver<Collection<SkyKey>> inflightKeysReceiver) {
+ DirtyTrackingProgressReceiver progressReceiver) {
this.graph = graph;
- this.inflightKeysReceiver = inflightKeysReceiver;
- this.dirtyKeyTracker = Preconditions.checkNotNull(dirtyKeyTracker);
evaluatorContext =
new ParallelEvaluatorContext(
graph,
@@ -122,7 +115,6 @@ public final class ParallelEvaluator implements Evaluator {
/*storeErrorsAlongsideValues=*/ true,
progressReceiver,
storedEventFilter,
- dirtyKeyTracker,
createEvaluateRunnable(),
threadCount);
cycleDetector = new SimpleCycleDetector();
@@ -137,16 +129,12 @@ public final class ParallelEvaluator implements Evaluator {
EventFilter storedEventFilter,
boolean keepGoing,
boolean storeErrorsAlongsideValues,
- @Nullable EvaluationProgressReceiver progressReceiver,
- DirtyKeyTracker dirtyKeyTracker,
- Receiver<Collection<SkyKey>> inflightKeysReceiver,
+ DirtyTrackingProgressReceiver progressReceiver,
ForkJoinPool forkJoinPool,
CycleDetector cycleDetector) {
this.graph = graph;
- this.inflightKeysReceiver = inflightKeysReceiver;
this.cycleDetector = cycleDetector;
Preconditions.checkState(storeErrorsAlongsideValues || keepGoing);
- this.dirtyKeyTracker = Preconditions.checkNotNull(dirtyKeyTracker);
evaluatorContext =
new ParallelEvaluatorContext(
graph,
@@ -158,7 +146,6 @@ public final class ParallelEvaluator implements Evaluator {
storeErrorsAlongsideValues,
progressReceiver,
storedEventFilter,
- dirtyKeyTracker,
createEvaluateRunnable(),
Preconditions.checkNotNull(forkJoinPool));
}
@@ -331,14 +318,10 @@ public final class ParallelEvaluator implements Evaluator {
case VERIFIED_CLEAN:
// No child has a changed value. This node can be marked done and its parents signaled
// without any re-evaluation.
- evaluatorContext.getVisitor().notifyDone(skyKey);
Set<SkyKey> reverseDeps = state.markClean();
- if (evaluatorContext.getProgressReceiver() != null) {
- // Tell the receiver that the value was not actually changed this run.
- evaluatorContext
- .getProgressReceiver()
+ // Tell the receiver that the value was not actually changed this run.
+ evaluatorContext.getProgressReceiver()
.evaluated(skyKey, new SkyValueSupplier(state), EvaluationState.CLEAN);
- }
if (!evaluatorContext.keepGoing() && state.getErrorInfo() != null) {
if (!evaluatorContext.getVisitor().preventNewEvaluations()) {
return DirtyOutcome.ALREADY_PROCESSED;
@@ -370,9 +353,9 @@ public final class ParallelEvaluator implements Evaluator {
}
Set<SkyKey> oldDeps = state.getAllRemainingDirtyDirectDeps();
- SkyFunctionEnvironment env =
- new SkyFunctionEnvironment(
- skyKey, state.getTemporaryDirectDeps(), oldDeps, evaluatorContext);
+ SkyFunctionEnvironment env =
+ new SkyFunctionEnvironment(
+ skyKey, state.getTemporaryDirectDeps(), oldDeps, evaluatorContext);
SkyFunctionName functionName = skyKey.functionName();
SkyFunction factory =
Preconditions.checkNotNull(
@@ -451,9 +434,7 @@ public final class ParallelEvaluator implements Evaluator {
env.doneBuilding();
long elapsedTimeNanos = BlazeClock.instance().nanoTime() - startTime;
if (elapsedTimeNanos > 0) {
- if (evaluatorContext.getProgressReceiver() != null) {
- evaluatorContext.getProgressReceiver().computed(skyKey, elapsedTimeNanos);
- }
+ evaluatorContext.getProgressReceiver().computed(skyKey, elapsedTimeNanos);
Profiler.instance().logSimpleTaskDuration(startTime, elapsedTimeNanos,
ProfilerTask.SKYFUNCTION, skyKey);
}
@@ -546,8 +527,8 @@ public final class ParallelEvaluator implements Evaluator {
return;
}
- for (Entry<SkyKey, ? extends NodeEntry> e :
- graph.createIfAbsentBatch(skyKey, Reason.ENQUEUING_CHILD, newDirectDeps).entrySet()) {
+ for (Entry<SkyKey, ? extends NodeEntry> e :
+ graph.createIfAbsentBatch(skyKey, Reason.ENQUEUING_CHILD, newDirectDeps).entrySet()) {
SkyKey newDirectDep = e.getKey();
NodeEntry newDirectDepEntry = e.getValue();
enqueueChild(
@@ -592,7 +573,7 @@ public final class ParallelEvaluator implements Evaluator {
/**
* Add any additional deps that were registered during the run of a builder that finished by
* creating a node or throwing an error. Builders may throw errors even if all their deps were not
- * provided -- we trust that a SkyFunction may be know it should throw an error even if not all of
+ * provided -- we trust that a SkyFunction might know it should throw an error even if not all of
* its requested deps are done. However, that means we're assuming the SkyFunction would throw
* that same error if all of its requested deps were done. Unfortunately, there is no way to
* enforce that condition.
@@ -739,7 +720,7 @@ public final class ParallelEvaluator implements Evaluator {
ImmutableMap.of(ErrorTransienceValue.KEY, (SkyValue) ErrorTransienceValue.INSTANCE),
evaluatorContext.getGraphVersion(),
graph,
- dirtyKeyTracker);
+ evaluatorContext.getProgressReceiver());
}
for (Entry<SkyKey, ? extends NodeEntry> e :
graph.createIfAbsentBatch(null, Reason.PRE_OR_POST_EVALUATION, skyKeys).entrySet()) {
@@ -759,11 +740,7 @@ public final class ParallelEvaluator implements Evaluator {
throw new IllegalStateException(entry + " for " + skyKey + " in unknown state");
}
}
- try {
- return waitForCompletionAndConstructResult(skyKeys);
- } finally {
- inflightKeysReceiver.accept(evaluatorContext.getVisitor().getInflightNodes());
- }
+ return waitForCompletionAndConstructResult(skyKeys);
}
private <T extends SkyValue> EvaluationResult<T> waitForCompletionAndConstructResult(
@@ -926,7 +903,7 @@ public final class ParallelEvaluator implements Evaluator {
evaluatorContext.getGraphVersion());
continue;
}
- if (evaluatorContext.getVisitor().isInflight(bubbleParent)
+ if (evaluatorContext.getProgressReceiver().isInflight(bubbleParent)
&& bubbleParentEntry.getTemporaryDirectDeps().expensiveContains(errorKey)) {
// Only bubble up to parent if it's part of this build. If this node was dirtied and
// re-evaluated, but in a build without this parent, we may try to bubble up to that
@@ -1130,7 +1107,7 @@ public final class ParallelEvaluator implements Evaluator {
Map<SkyKey, SkyValue> injectionMap,
Version version,
EvaluableGraph graph,
- DirtyKeyTracker dirtyKeyTracker)
+ DirtyTrackingProgressReceiver progressReceiver)
throws InterruptedException {
Map<SkyKey, ? extends NodeEntry> prevNodeEntries =
graph.createIfAbsentBatch(null, Reason.OTHER, injectionMap.keySet());
@@ -1157,7 +1134,7 @@ public final class ParallelEvaluator implements Evaluator {
}
prevEntry.setValue(value, version);
// Now that this key's injected value is set, it is no longer dirty.
- dirtyKeyTracker.notDirty(key);
+ progressReceiver.injected(key);
}
}
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java
index f364a71f4b..0b6a07bbc8 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java
@@ -49,7 +49,7 @@ class ParallelEvaluatorContext {
private final NestedSetVisitor<TaggedEvents> replayingNestedSetEventVisitor;
private final boolean keepGoing;
private final boolean storeErrorsAlongsideValues;
- @Nullable private final EvaluationProgressReceiver progressReceiver;
+ private final DirtyTrackingProgressReceiver progressReceiver;
private final EventFilter storedEventFilter;
/**
* The visitor managing the thread pool. Used to enqueue parents when an entry is finished, and,
@@ -67,9 +67,8 @@ class ParallelEvaluatorContext {
EmittedEventState emittedEventState,
boolean keepGoing,
boolean storeErrorsAlongsideValues,
- final EvaluationProgressReceiver progressReceiver,
+ final DirtyTrackingProgressReceiver progressReceiver,
EventFilter storedEventFilter,
- final DirtyKeyTracker dirtyKeyTracker,
final Function<SkyKey, Runnable> runnableMaker,
final int threadCount) {
this.graph = graph;
@@ -80,7 +79,7 @@ class ParallelEvaluatorContext {
new NestedSetVisitor<>(new NestedSetEventReceiver(reporter), emittedEventState);
this.keepGoing = keepGoing;
this.storeErrorsAlongsideValues = storeErrorsAlongsideValues;
- this.progressReceiver = progressReceiver;
+ this.progressReceiver = Preconditions.checkNotNull(progressReceiver);
this.storedEventFilter = storedEventFilter;
visitorSupplier =
Suppliers.memoize(
@@ -88,7 +87,7 @@ class ParallelEvaluatorContext {
@Override
public NodeEntryVisitor get() {
return new NodeEntryVisitor(
- threadCount, dirtyKeyTracker, progressReceiver, runnableMaker);
+ threadCount, progressReceiver, runnableMaker);
}
});
}
@@ -101,9 +100,8 @@ class ParallelEvaluatorContext {
EmittedEventState emittedEventState,
boolean keepGoing,
boolean storeErrorsAlongsideValues,
- final EvaluationProgressReceiver progressReceiver,
+ final DirtyTrackingProgressReceiver progressReceiver,
EventFilter storedEventFilter,
- final DirtyKeyTracker dirtyKeyTracker,
final Function<SkyKey, Runnable> runnableMaker,
final ForkJoinPool forkJoinPool) {
this.graph = graph;
@@ -114,7 +112,7 @@ class ParallelEvaluatorContext {
new NestedSetVisitor<>(new NestedSetEventReceiver(reporter), emittedEventState);
this.keepGoing = keepGoing;
this.storeErrorsAlongsideValues = storeErrorsAlongsideValues;
- this.progressReceiver = progressReceiver;
+ this.progressReceiver = Preconditions.checkNotNull(progressReceiver);
this.storedEventFilter = storedEventFilter;
visitorSupplier =
Suppliers.memoize(
@@ -122,7 +120,7 @@ class ParallelEvaluatorContext {
@Override
public NodeEntryVisitor get() {
return new NodeEntryVisitor(
- forkJoinPool, dirtyKeyTracker, progressReceiver, runnableMaker);
+ forkJoinPool, progressReceiver, runnableMaker);
}
});
}
@@ -186,8 +184,7 @@ class ParallelEvaluatorContext {
return visitorSupplier.get();
}
- @Nullable
- EvaluationProgressReceiver getProgressReceiver() {
+ DirtyTrackingProgressReceiver getProgressReceiver() {
return progressReceiver;
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/SimpleCycleDetector.java b/src/main/java/com/google/devtools/build/skyframe/SimpleCycleDetector.java
index 1c41fa56c8..59fbc077e5 100644
--- a/src/main/java/com/google/devtools/build/skyframe/SimpleCycleDetector.java
+++ b/src/main/java/com/google/devtools/build/skyframe/SimpleCycleDetector.java
@@ -107,19 +107,19 @@ class SimpleCycleDetector implements CycleDetector {
toVisit.push(root);
- // The procedure for this check is as follows: we visit a node, push it onto the graph stack,
+ // The procedure for this check is as follows: we visit a node, push it onto the graph path,
// push a marker value onto the toVisit stack, and then push all of its children onto the
// toVisit stack. Thus, when the marker node comes to the top of the toVisit stack, we have
// visited the downward transitive closure of the value. At that point, all of its children must
// be finished, and so we can build the definitive error info for the node, popping it off the
- // graph stack.
+ // graph path.
while (!toVisit.isEmpty()) {
SkyKey key = toVisit.pop();
NodeEntry entry;
if (key == CHILDREN_FINISHED) {
- // A marker node means we are done with all children of a node. Since all nodes have
- // errors, we must have found errors in the children when that happens.
+ // We have reached the marker node - that means all children of a node have been visited.
+ // Since all nodes have errors, we must have found errors in the children at this point.
key = graphPath.remove(graphPath.size() - 1);
entry =
Preconditions.checkNotNull(
@@ -131,8 +131,7 @@ class SimpleCycleDetector implements CycleDetector {
}
if (!evaluatorContext.keepGoing()) {
// in the --nokeep_going mode, we would have already returned if we'd found a cycle below
- // this node. The fact that we haven't means that there were no cycles below this node
- // -- it just hadn't finished evaluating. So skip it.
+ // this node. We haven't, so there are no cycles below this node; skip further evaluation
continue;
}
Set<SkyKey> removedDeps = ImmutableSet.of();
diff --git a/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java
index a15522025b..8c3126f38b 100644
--- a/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java
@@ -565,27 +565,25 @@ class SkyFunctionEnvironment extends AbstractSkyFunctionEnvironment {
"%s should be at most %s in the version partial ordering",
valueVersion,
evaluatorContext.getGraphVersion());
- if (evaluatorContext.getProgressReceiver() != null) {
- // Tell the receiver that this value was built. If valueVersion.equals(graphVersion), it
- // was evaluated this run, and so was changed. Otherwise, it is less than graphVersion,
- // by the Preconditions check above, and was not actually changed this run -- when it was
- // written above, its version stayed below this update's version, so its value remains the
- // same as before.
- // We use a SkyValueSupplier here because it keeps a reference to the entry, allowing for
- // the receiver to be confident that the entry is readily accessible in memory.
- evaluatorContext
- .getProgressReceiver()
- .evaluated(
- skyKey,
- new SkyValueSupplier(primaryEntry),
- valueVersion.equals(evaluatorContext.getGraphVersion())
- ? EvaluationState.BUILT
- : EvaluationState.CLEAN);
- }
+
+ // Tell the receiver that this value was built. If valueVersion.equals(graphVersion), it was
+ // evaluated this run, and so was changed. Otherwise, it is less than graphVersion, by the
+ // Preconditions check above, and was not actually changed this run -- when it was written
+ // above, its version stayed below this update's version, so its value remains the same.
+ // We use a SkyValueSupplier here because it keeps a reference to the entry, allowing for
+ // the receiver to be confident that the entry is readily accessible in memory.
+ evaluatorContext
+ .getProgressReceiver()
+ .evaluated(
+ skyKey,
+ new SkyValueSupplier(primaryEntry),
+ valueVersion.equals(evaluatorContext.getGraphVersion())
+ ? EvaluationState.BUILT
+ : EvaluationState.CLEAN);
+
evaluatorContext.signalValuesAndEnqueueIfReady(
skyKey, reverseDeps, valueVersion, enqueueParents);
- evaluatorContext.getVisitor().notifyDone(skyKey);
evaluatorContext.getReplayingNestedSetEventVisitor().visit(events);
}
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 63fbee7f01..428bff5090 100644
--- a/src/test/java/com/google/devtools/build/skyframe/EagerInvalidatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/EagerInvalidatorTest.java
@@ -40,23 +40,20 @@ import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.DirtyingNodeVi
import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.InvalidationState;
import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.InvalidationType;
import com.google.devtools.build.skyframe.QueryableGraph.Reason;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.runners.Enclosed;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
import java.lang.ref.WeakReference;
-import java.util.Collection;
import java.util.HashSet;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.runners.Enclosed;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
-import javax.annotation.Nullable;
/**
* Tests for {@link InvalidatingNodeVisitor}.
@@ -67,8 +64,7 @@ public class EagerInvalidatorTest {
protected GraphTester tester = new GraphTester();
protected InvalidationState state = newInvalidationState();
protected AtomicReference<InvalidatingNodeVisitor<?>> visitor = new AtomicReference<>();
- protected DirtyKeyTrackerImpl dirtyKeyTracker;
-
+ protected DirtyTrackingProgressReceiver progressReceiver;
private IntVersion graphVersion = IntVersion.of(0);
@After
@@ -85,7 +81,7 @@ public class EagerInvalidatorTest {
@SuppressWarnings("unused") // Overridden by subclasses.
void invalidate(
- InMemoryGraph graph, EvaluationProgressReceiver progressReceiver, SkyKey... keys)
+ InMemoryGraph graph, DirtyTrackingProgressReceiver progressReceiver, SkyKey... keys)
throws InterruptedException {
throw new UnsupportedOperationException();
}
@@ -152,19 +148,12 @@ public class EagerInvalidatorTest {
InMemoryMemoizingEvaluator.DEFAULT_STORED_EVENT_FILTER,
keepGoing,
200,
- null,
- new DirtyKeyTrackerImpl(),
- new ParallelEvaluator.Receiver<Collection<SkyKey>>() {
- @Override
- public void accept(Collection<SkyKey> object) {
- // ignore
- }
- });
+ new DirtyTrackingProgressReceiver(null));
graphVersion = graphVersion.next();
return evaluator.eval(ImmutableList.copyOf(keys));
}
- protected void invalidateWithoutError(@Nullable EvaluationProgressReceiver progressReceiver,
+ protected void invalidateWithoutError(DirtyTrackingProgressReceiver progressReceiver,
SkyKey... keys) throws InterruptedException {
invalidate(graph, progressReceiver, keys);
assertTrue(state.isEmpty());
@@ -185,13 +174,14 @@ public class EagerInvalidatorTest {
@Before
public void setUp() throws Exception {
- dirtyKeyTracker = new DirtyKeyTrackerImpl();
+ progressReceiver = new DirtyTrackingProgressReceiver(null);
}
@Test
public void receiverWorks() throws Exception {
final Set<SkyKey> invalidated = Sets.newConcurrentHashSet();
- EvaluationProgressReceiver receiver = new EvaluationProgressReceiver() {
+ DirtyTrackingProgressReceiver receiver = new DirtyTrackingProgressReceiver(
+ new EvaluationProgressReceiver() {
@Override
public void invalidated(SkyKey skyKey, InvalidationState state) {
Preconditions.checkState(state == expectedState());
@@ -213,7 +203,7 @@ public class EagerInvalidatorTest {
EvaluationState state) {
throw new UnsupportedOperationException();
}
- };
+ });
graph = new InMemoryGraphImpl();
set("a", "a");
set("b", "b");
@@ -233,7 +223,8 @@ public class EagerInvalidatorTest {
@Test
public void receiverIsNotifiedAboutNodesInError() throws Exception {
final Set<SkyKey> invalidated = Sets.newConcurrentHashSet();
- EvaluationProgressReceiver receiver = new EvaluationProgressReceiver() {
+ DirtyTrackingProgressReceiver receiver = new DirtyTrackingProgressReceiver(
+ new EvaluationProgressReceiver() {
@Override
public void invalidated(SkyKey skyKey, InvalidationState state) {
Preconditions.checkState(state == expectedState());
@@ -255,7 +246,7 @@ public class EagerInvalidatorTest {
EvaluationState state) {
throw new UnsupportedOperationException();
}
- };
+ });
// Given a graph consisting of two nodes, "a" and "ab" such that "ab" depends on "a",
// And given "ab" is in error,
@@ -277,7 +268,8 @@ public class EagerInvalidatorTest {
@Test
public void invalidateValuesNotInGraph() throws Exception {
final Set<SkyKey> invalidated = Sets.newConcurrentHashSet();
- EvaluationProgressReceiver receiver = new EvaluationProgressReceiver() {
+ DirtyTrackingProgressReceiver receiver = new DirtyTrackingProgressReceiver(
+ new EvaluationProgressReceiver() {
@Override
public void invalidated(SkyKey skyKey, InvalidationState state) {
Preconditions.checkState(state == InvalidationState.DIRTY);
@@ -299,7 +291,7 @@ public class EagerInvalidatorTest {
EvaluationState state) {
throw new UnsupportedOperationException();
}
- };
+ });
graph = new InMemoryGraphImpl();
invalidateWithoutError(receiver, skyKey("a"));
assertThat(invalidated).isEmpty();
@@ -318,7 +310,7 @@ public class EagerInvalidatorTest {
graph = new InMemoryGraphImpl();
eval(false, key);
- invalidate(graph, null, key);
+ invalidate(graph, new DirtyTrackingProgressReceiver(null), key);
tester = null;
heavyValue = null;
@@ -351,7 +343,7 @@ public class EagerInvalidatorTest {
assertThat(graph.get(null, Reason.OTHER, skyKey("c"))
.getReverseDeps()).containsExactly(skyKey("ab_c"), skyKey("bc"));
- invalidateWithoutError(null, skyKey("ab"));
+ invalidateWithoutError(new DirtyTrackingProgressReceiver(null), skyKey("ab"));
eval(false);
// The graph values should be gone.
@@ -396,7 +388,7 @@ public class EagerInvalidatorTest {
eval(/*keepGoing=*/false, parent);
final Thread mainThread = Thread.currentThread();
final AtomicReference<SkyKey> badKey = new AtomicReference<>();
- EvaluationProgressReceiver receiver =
+ DirtyTrackingProgressReceiver receiver = new DirtyTrackingProgressReceiver(
new EvaluationProgressReceiver() {
@Override
public void invalidated(SkyKey skyKey, InvalidationState state) {
@@ -432,7 +424,7 @@ public class EagerInvalidatorTest {
SkyKey skyKey, Supplier<SkyValue> skyValueSupplier, EvaluationState state) {
throw new UnsupportedOperationException();
}
- };
+ });
try {
invalidateWithoutError(receiver, child);
fail();
@@ -444,7 +436,8 @@ public class EagerInvalidatorTest {
final Set<SkyKey> invalidated = Sets.newConcurrentHashSet();
assertFalse(isInvalidated(parent));
assertNotNull(graph.get(null, Reason.OTHER, parent).getValue());
- receiver = new EvaluationProgressReceiver() {
+ receiver = new DirtyTrackingProgressReceiver(
+ new EvaluationProgressReceiver() {
@Override
public void invalidated(SkyKey skyKey, InvalidationState state) {
invalidated.add(skyKey);
@@ -465,7 +458,7 @@ public class EagerInvalidatorTest {
EvaluationState state) {
throw new UnsupportedOperationException();
}
- };
+ });
invalidateWithoutError(receiver);
assertTrue(invalidated.contains(parent));
assertThat(state.getInvalidationsForTesting()).isEmpty();
@@ -534,8 +527,8 @@ public class EagerInvalidatorTest {
}
int countDownStart = validValuesToDo > 0 ? random.nextInt(validValuesToDo) : 0;
final CountDownLatch countDownToInterrupt = new CountDownLatch(countDownStart);
- final EvaluationProgressReceiver receiver =
- new EvaluationProgressReceiver() {
+ final DirtyTrackingProgressReceiver receiver =
+ new DirtyTrackingProgressReceiver(new EvaluationProgressReceiver() {
@Override
public void invalidated(SkyKey skyKey, InvalidationState state) {
countDownToInterrupt.countDown();
@@ -566,7 +559,7 @@ public class EagerInvalidatorTest {
SkyKey skyKey, Supplier<SkyValue> skyValueSupplier, EvaluationState state) {
throw new UnsupportedOperationException();
}
- };
+ });
try {
invalidate(graph, receiver,
Sets.newHashSet(
@@ -604,12 +597,12 @@ public class EagerInvalidatorTest {
public static class DeletingInvalidatorTest extends EagerInvalidatorTest {
@Override
protected void invalidate(
- InMemoryGraph graph, EvaluationProgressReceiver progressReceiver, SkyKey... keys)
+ InMemoryGraph graph, DirtyTrackingProgressReceiver progressReceiver, SkyKey... keys)
throws InterruptedException {
Iterable<SkyKey> diff = ImmutableList.copyOf(keys);
DeletingNodeVisitor deletingNodeVisitor =
EagerInvalidator.createDeletingVisitorIfNeeded(
- graph, diff, progressReceiver, state, true, dirtyKeyTracker);
+ graph, diff, new DirtyTrackingProgressReceiver(progressReceiver), state, true);
if (deletingNodeVisitor != null) {
visitor.set(deletingNodeVisitor);
deletingNodeVisitor.run();
@@ -642,9 +635,10 @@ public class EagerInvalidatorTest {
}
@Test
- public void dirtyKeyTrackerWorksWithDeletingInvalidator() throws Exception {
+ public void dirtyTrackingProgressReceiverWorksWithDeletingInvalidator() throws Exception {
setupInvalidatableGraph();
- TrackingProgressReceiver receiver = new TrackingProgressReceiver();
+ DirtyTrackingProgressReceiver receiver = new DirtyTrackingProgressReceiver(
+ new TrackingProgressReceiver());
// Dirty the node, and ensure that the tracker is aware of it:
Iterable<SkyKey> diff1 = ImmutableList.of(skyKey("a"));
@@ -655,16 +649,15 @@ public class EagerInvalidatorTest {
diff1,
receiver,
state1,
- dirtyKeyTracker,
AbstractQueueVisitor.EXECUTOR_FACTORY))
.run();
- assertThat(dirtyKeyTracker.getDirtyKeys()).containsExactly(skyKey("a"), skyKey("ab"));
+ assertThat(receiver.getUnenqueuedDirtyKeys()).containsExactly(skyKey("a"), skyKey("ab"));
// Delete the node, and ensure that the tracker is no longer tracking it:
Iterable<SkyKey> diff = ImmutableList.of(skyKey("a"));
Preconditions.checkNotNull(EagerInvalidator.createDeletingVisitorIfNeeded(graph, diff,
- receiver, state, true, dirtyKeyTracker)).run();
- assertThat(dirtyKeyTracker.getDirtyKeys()).isEmpty();
+ receiver, state, true)).run();
+ assertThat(receiver.getUnenqueuedDirtyKeys()).isEmpty();
}
}
@@ -675,7 +668,7 @@ public class EagerInvalidatorTest {
public static class DirtyingInvalidatorTest extends EagerInvalidatorTest {
@Override
protected void invalidate(
- InMemoryGraph graph, EvaluationProgressReceiver progressReceiver, SkyKey... keys)
+ InMemoryGraph graph, DirtyTrackingProgressReceiver progressReceiver, SkyKey... keys)
throws InterruptedException {
Iterable<SkyKey> diff = ImmutableList.copyOf(keys);
DirtyingNodeVisitor dirtyingNodeVisitor =
@@ -684,7 +677,6 @@ public class EagerInvalidatorTest {
diff,
progressReceiver,
state,
- dirtyKeyTracker,
AbstractQueueVisitor.EXECUTOR_FACTORY);
if (dirtyingNodeVisitor != null) {
visitor.set(dirtyingNodeVisitor);
@@ -718,13 +710,14 @@ public class EagerInvalidatorTest {
}
@Test
- public void dirtyKeyTrackerWorksWithDirtyingInvalidator() throws Exception {
+ public void dirtyTrackingProgressReceiverWorksWithDirtyingInvalidator() throws Exception {
setupInvalidatableGraph();
- TrackingProgressReceiver receiver = new TrackingProgressReceiver();
+ DirtyTrackingProgressReceiver receiver =
+ new DirtyTrackingProgressReceiver(new TrackingProgressReceiver());
// Dirty the node, and ensure that the tracker is aware of it:
invalidate(graph, receiver, skyKey("a"));
- assertThat(dirtyKeyTracker.getDirtyKeys()).hasSize(2);
+ assertThat(receiver.getUnenqueuedDirtyKeys()).hasSize(2);
}
}
}
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 a901a29200..cb797acbf3 100644
--- a/src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/ParallelEvaluatorTest.java
@@ -49,7 +49,6 @@ import com.google.devtools.build.skyframe.NotifyingHelper.Order;
import com.google.devtools.build.skyframe.SkyFunctionException.Transience;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -77,7 +76,8 @@ public class ParallelEvaluatorTest {
private EventCollector eventCollector;
- private EvaluationProgressReceiver revalidationReceiver;
+ private DirtyTrackingProgressReceiver revalidationReceiver =
+ new DirtyTrackingProgressReceiver(null);
@Before
public void initializeReporter() {
@@ -104,14 +104,7 @@ public class ParallelEvaluatorTest {
storedEventFilter,
keepGoing,
150,
- revalidationReceiver,
- new DirtyKeyTrackerImpl(),
- new ParallelEvaluator.Receiver<Collection<SkyKey>>() {
- @Override
- public void accept(Collection<SkyKey> object) {
- // ignore
- }
- });
+ revalidationReceiver);
}
private ParallelEvaluator makeEvaluator(ProcessableGraph graph,
@@ -259,7 +252,7 @@ public class ParallelEvaluatorTest {
eval(/*keepGoing=*/false, fastKey);
}
final Set<SkyKey> receivedValues = Sets.newConcurrentHashSet();
- revalidationReceiver = new EvaluationProgressReceiver() {
+ revalidationReceiver = new DirtyTrackingProgressReceiver(new EvaluationProgressReceiver() {
@Override
public void invalidated(SkyKey skyKey, InvalidationState state) {}
@@ -274,7 +267,7 @@ public class ParallelEvaluatorTest {
EvaluationState state) {
receivedValues.add(skyKey);
}
- };
+ });
TestThread evalThread = new TestThread() {
@Override
public void runTest() throws Exception {