aboutsummaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java75
-rw-r--r--src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java186
-rw-r--r--src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.java9
3 files changed, 250 insertions, 20 deletions
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 cf769495ee..6749a1e5cd 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -113,7 +113,6 @@ public final class ParallelEvaluator implements Evaluator {
private final int threadCount;
@Nullable private final EvaluationProgressReceiver progressReceiver;
private final DirtyKeyTracker dirtyKeyTracker;
- private final AtomicBoolean errorEncountered = new AtomicBoolean(false);
private static final Interner<SkyKey> KEY_CANONICALIZER = Interners.newWeakInterner();
@@ -537,11 +536,17 @@ public final class ParallelEvaluator implements Evaluator {
enqueue(new Evaluate(this, key));
}
- public void preventNewEvaluations() {
- preventNewEvaluations.set(true);
+ /**
+ * Stop any new evaluations from being enqueued. Returns whether this was the first thread to
+ * request a halt. If true, this thread should proceed to throw an exception. If false, another
+ * thread already requested a halt and will throw an exception, and so this thread can simply
+ * end.
+ */
+ boolean preventNewEvaluations() {
+ return preventNewEvaluations.compareAndSet(false, true);
}
- public void notifyDone(SkyKey key) {
+ void notifyDone(SkyKey key) {
inflightNodes.remove(key);
}
@@ -624,14 +629,40 @@ public final class ParallelEvaluator implements Evaluator {
// which would be incorrect if, for instance, the value re-evaluated to a non-error.
state.forceRebuild();
break; // Fall through to re-evaluation.
- } else {
- // If this isn't the error transience value, it is safe to add these deps back to the
- // node -- even if one of them has changed, the contract of pruning is that the node
- // will request these deps again when it rebuilds. We must add these deps before
- // enqueuing them, so that the node knows that it depends on them.
- state.addTemporaryDirectDeps(GroupedListHelper.create(directDepsToCheck));
}
-
+ if (!keepGoing) {
+ // This check ensures that we maintain the invariant that if a node with an error is
+ // reached during a no-keep-going build, none of its currently building parents
+ // finishes building. If the child isn't done building yet, it will detect on its own
+ // that it has an error (see the VERIFIED_CLEAN case below). On the other hand, if it
+ // is done, then it is the parent's responsibility to notice that, which we do here.
+ // We check the deps for errors so that we don't continue building this node if it has
+ // a child error.
+ for (Map.Entry<SkyKey, NodeEntry> entry :
+ graph.getBatch(directDepsToCheck).entrySet()) {
+ if (entry.getValue().isDone() && entry.getValue().getErrorInfo() != null) {
+ // If any child has an error, we arbitrarily add a dep on the first one (needed
+ // for error bubbling) and throw an exception coming from it.
+ if (!visitor.preventNewEvaluations()) {
+ // An error was already thrown in the evaluator. Don't do anything here.
+ return;
+ }
+ SkyKey errorKey = entry.getKey();
+ NodeEntry errorEntry = entry.getValue();
+ state.addTemporaryDirectDeps(
+ GroupedListHelper.create(ImmutableList.of(errorKey)));
+ DependencyState errorState = entry.getValue().addReverseDepAndCheckIfDone(skyKey);
+ Preconditions.checkState(errorState == DependencyState.DONE, "%s %s %s %s",
+ skyKey, state, errorKey, errorEntry);
+ throw SchedulerException.ofError(errorEntry.getErrorInfo(), entry.getKey());
+ }
+ }
+ }
+ // If this isn't the error transience value, it is safe to add these deps back to the
+ // node -- even if one of them has changed, the contract of pruning is that the node
+ // will request these deps again when it rebuilds. We must add these deps before
+ // enqueuing them, so that the node knows that it depends on them.
+ state.addTemporaryDirectDeps(GroupedListHelper.create(directDepsToCheck));
for (SkyKey directDep : directDepsToCheck) {
enqueueChild(skyKey, state, directDep);
}
@@ -646,6 +677,12 @@ public final class ParallelEvaluator implements Evaluator {
// Tell the receiver that the value was not actually changed this run.
progressReceiver.evaluated(skyKey, value, EvaluationState.CLEAN);
}
+ if (!keepGoing && state.getErrorInfo() != null) {
+ if (!visitor.preventNewEvaluations()) {
+ return;
+ }
+ throw SchedulerException.ofError(state.getErrorInfo(), skyKey);
+ }
signalValuesAndEnqueueIfReady(visitor, reverseDeps, state.getVersion());
return;
case REBUILDING:
@@ -682,10 +719,7 @@ public final class ParallelEvaluator implements Evaluator {
// After we commit this error to the graph but before the eval call completes with the
// error there is a race-like opportunity for the error to be used, either by an
// in-flight computation or by a future computation.
- if (errorEncountered.compareAndSet(false, true)) {
- // This is the first error encountered.
- visitor.preventNewEvaluations();
- } else {
+ if (!visitor.preventNewEvaluations()) {
// This is not the first error encountered, so we ignore it so that we can terminate
// with the first error.
return;
@@ -727,8 +761,8 @@ public final class ParallelEvaluator implements Evaluator {
return;
}
- if (!newDirectDeps.isEmpty() && env.getDepErrorKey() != null) {
- Preconditions.checkState(!keepGoing);
+ if (env.getDepErrorKey() != null) {
+ Preconditions.checkState(!keepGoing, "%s %s %s", skyKey, state, env.getDepErrorKey());
// We encountered a child error in noKeepGoing mode, so we want to fail fast. But we first
// need to add the edge between the current node and the child error it requested so that
// error bubbling can occur. Note that this edge will subsequently be removed during graph
@@ -1016,7 +1050,7 @@ public final class ParallelEvaluator implements Evaluator {
new ThrowableRecordingRunnableWrapper("ParallelEvaluator#clean");
for (final SkyKey key : inflightNodes) {
final NodeEntry entry = graph.get(key);
- Preconditions.checkState(!entry.isDone(), key);
+ Preconditions.checkState(!entry.isDone(), "%s %s", key, entry);
executor.execute(wrapper.wrap(new Runnable() {
@Override
public void run() {
@@ -1177,7 +1211,7 @@ public final class ParallelEvaluator implements Evaluator {
parentEntry = bubbleParentEntry;
break;
}
- Preconditions.checkNotNull(parent, "", errorKey, bubbleErrorInfo);
+ Preconditions.checkNotNull(parent, "%s %s", errorKey, bubbleErrorInfo);
errorKey = parent;
SkyFunction factory = skyFunctions.get(parent.functionName());
if (parentEntry.isDirty()) {
@@ -1365,7 +1399,7 @@ public final class ParallelEvaluator implements Evaluator {
// 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.
key = graphPath.remove(graphPath.size() - 1);
- entry = graph.get(key);
+ entry = Preconditions.checkNotNull(graph.get(key), key);
pathSet.remove(key);
// Skip this node if it was first/last node of a cycle, and so has already been processed.
if (entry.isDone()) {
@@ -1395,6 +1429,7 @@ public final class ParallelEvaluator implements Evaluator {
env.commit(/*enqueueParents=*/false);
}
+ Preconditions.checkNotNull(entry, key);
// Nothing to be done for this node if it already has an entry.
if (entry.isDone()) {
continue;
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 230cae00b2..3975c97a5f 100644
--- a/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
@@ -44,6 +44,7 @@ import com.google.devtools.build.lib.events.Reporter;
import com.google.devtools.build.lib.testutil.JunitTestUtils;
import com.google.devtools.build.lib.testutil.TestThread;
import com.google.devtools.build.lib.testutil.TestUtils;
+import com.google.devtools.build.lib.util.Pair;
import com.google.devtools.build.skyframe.GraphTester.StringValue;
import com.google.devtools.build.skyframe.GraphTester.TestFunction;
import com.google.devtools.build.skyframe.GraphTester.ValueComputer;
@@ -68,6 +69,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
import javax.annotation.Nullable;
@@ -1884,6 +1886,7 @@ public class MemoizingEvaluatorTest {
final AtomicBoolean interruptInvalidation = new AtomicBoolean(false);
initializeTester(new TrackingInvalidationReceiver() {
private final AtomicBoolean firstInvalidation = new AtomicBoolean(true);
+
@Override
public void invalidated(SkyValue value, InvalidationState state) {
if (interruptInvalidation.get() && !firstInvalidation.getAndSet(false)) {
@@ -2758,6 +2761,189 @@ public class MemoizingEvaluatorTest {
assertThat(tester.getEnqueuedValues()).containsExactly(topKey, transientErrorKey);
}
+ /**
+ * The following two tests check that the evaluator shuts down properly when encountering an error
+ * that is marked dirty but later verified to be unchanged from a prior build. In that case, the
+ * invariant that its parents are not enqueued for evaluation should be maintained.
+ */
+ /**
+ * Test that a parent of a cached but invalidated error doesn't successfully build. First build
+ * the error. Then invalidate the error via a dependency (so it will not actually change) and
+ * build two new parents. Parent A will request error and abort since error isn't done yet. error
+ * is then revalidated, and A is restarted. If A does not throw upon encountering the error, and
+ * instead sets its value, then we throw in parent B, which waits for error to be done before
+ * requesting it. Then there will be the impossible situation of a node that was built during this
+ * evaluation depending on a node in error.
+ */
+ @Test
+ public void shutDownBuildOnCachedError_Done() throws Exception {
+ // errorKey will be invalidated due to its dependence on invalidatedKey, but later revalidated
+ // since invalidatedKey re-evaluates to the same value on a subsequent build.
+ final SkyKey errorKey = GraphTester.toSkyKey("error");
+ SkyKey invalidatedKey = GraphTester.toSkyKey("invalidated-leaf");
+ tester.set(invalidatedKey, new StringValue("invalidated-leaf-value"));
+ tester.getOrCreate(errorKey).addDependency(invalidatedKey).setHasError(true);
+ // Names are alphabetized in reverse deps of errorKey.
+ final SkyKey fastToRequestSlowToSetValueKey = GraphTester.toSkyKey("A-slow-set-value-parent");
+ final SkyKey failingKey = GraphTester.toSkyKey("B-fast-fail-parent");
+ tester.getOrCreate(fastToRequestSlowToSetValueKey).addDependency(errorKey)
+ .setComputedValue(CONCATENATE);
+ tester.getOrCreate(failingKey).addDependency(errorKey).setComputedValue(CONCATENATE);
+ // We only want to force a particular order of operations at some points during evaluation.
+ 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);
+ // Keep track of any exceptions thrown during evaluation.
+ final AtomicReference<Pair<SkyKey, ? extends Exception>> unexpectedException =
+ new AtomicReference<>();
+ 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);
+ unexpectedException.set(Pair.of(key, new IllegalStateException("uninterrupted")));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+ }));
+ // Initialize graph.
+ tester.eval(/*keepGoing=*/true, errorKey);
+ tester.getOrCreate(invalidatedKey, /*markAsModified=*/true);
+ tester.invalidate();
+ synchronizeThreads.set(true);
+ tester.eval(/*keepGoing=*/false, fastToRequestSlowToSetValueKey, failingKey);
+ Pair<SkyKey, ? extends Exception> unexpected = unexpectedException.get();
+ if (unexpected != null) {
+ throw new AssertionError(unexpected.first + ", " + unexpected.second + ", "
+ + Arrays.toString(unexpected.second.getStackTrace()));
+ }
+ }
+
+ /**
+ * Test that the invalidated parent of a cached but invalidated error doesn't get marked clean.
+ * First build the parent -- it will contain an error. Then invalidate the error via a dependency
+ * (so it will not actually change) and then build the parent and another node that depends on the
+ * error. The other node will wait to throw until the parent is signaled that all of its
+ * dependencies are done, or until it is interrupted. If it throws, the parent will be
+ * VERIFIED_CLEAN but not done, which is not a valid state once evaluation shuts down. The
+ * evaluator avoids this situation by throwing when the error is encountered, even though the
+ * error isn't evaluated or requested by an evaluating node.
+ */
+ @Test
+ public void shutDownBuildOnCachedError_Verified() throws Exception {
+ // errorKey will be invalidated due to its dependence on invalidatedKey, but later revalidated
+ // since invalidatedKey re-evaluates to the same value on a subsequent build.
+ SkyKey errorKey = GraphTester.toSkyKey("error");
+ SkyKey invalidatedKey = GraphTester.toSkyKey("invalidated-leaf");
+ SkyKey changedKey = GraphTester.toSkyKey("changed-leaf");
+ tester.set(invalidatedKey, new StringValue("invalidated-leaf-value"));
+ tester.set(changedKey, new StringValue("changed-leaf-value"));
+ // Names are alphabetized in reverse deps of errorKey.
+ final SkyKey cachedParentKey = GraphTester.toSkyKey("A-cached-parent");
+ final SkyKey uncachedParentKey = GraphTester.toSkyKey("B-uncached-parent");
+ tester.getOrCreate(errorKey).addDependency(invalidatedKey).setHasError(true);
+ tester.getOrCreate(cachedParentKey).addDependency(errorKey)
+ .setComputedValue(CONCATENATE);
+ tester.getOrCreate(uncachedParentKey).addDependency(changedKey).addDependency(errorKey)
+ .setComputedValue(CONCATENATE);
+ // We only want to force a particular order of operations at some points during evaluation. In
+ // particular, we don't want to force anything during error bubbling.
+ final AtomicBoolean synchronizeThreads = new AtomicBoolean(false);
+ // Keep track of any exceptions thrown during evaluation.
+ final AtomicReference<Pair<SkyKey, ? extends Exception>> unexpectedException =
+ new AtomicReference<>();
+ setGraphForTesting(new DeterministicInMemoryGraph(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;
+ }
+ 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);
+ unexpectedException.set(Pair.of(key, new IllegalStateException("uninterrupted")));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+ }));
+ // Initialize graph.
+ tester.eval(/*keepGoing=*/true, cachedParentKey, uncachedParentKey);
+ tester.getOrCreate(invalidatedKey, /*markAsModified=*/true);
+ tester.set(changedKey, new StringValue("new value"));
+ tester.invalidate();
+ synchronizeThreads.set(true);
+ SkyKey waitForShutdownKey = GraphTester.skyKey("wait-for-shutdown");
+ tester.getOrCreate(waitForShutdownKey).setBuilder(new SkyFunction() {
+ @Override
+ public SkyValue compute(SkyKey skyKey, Environment env) throws InterruptedException {
+ try {
+ TrackingAwaiter.waitAndMaybeThrowInterrupt(
+ ((ParallelEvaluator.SkyFunctionEnvironment) env).getExceptionLatchForTesting(), "");
+ } catch (InterruptedException e) {
+ unexpectedException.set(Pair.of(skyKey, e));
+ }
+ // Threadpool is shutting down. Don't try to synchronize anything in the future during
+ // error bubbling.
+ synchronizeThreads.set(false);
+ throw new InterruptedException();
+ }
+
+ @Nullable
+ @Override
+ public String extractTag(SkyKey skyKey) {
+ return null;
+ }
+ });
+ tester.eval(/*keepGoing=*/false, cachedParentKey, uncachedParentKey, waitForShutdownKey);
+ Pair<SkyKey, ? extends Exception> unexpected = unexpectedException.get();
+ if (unexpected != null) {
+ throw new AssertionError(unexpected.first + ", " + unexpected.second + ", "
+ + Arrays.toString(unexpected.second.getStackTrace()));
+ }
+ }
+
@Test
public void cachedChildErrorDepWithSiblingDepOnNoKeepGoingEval() throws Exception {
SkyKey parent1Key = GraphTester.toSkyKey("parent1");
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 49128ca91d..0618cc32f6 100644
--- a/src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.java
+++ b/src/test/java/com/google/devtools/build/skyframe/NotifyingInMemoryGraph.java
@@ -63,6 +63,7 @@ public class NotifyingInMemoryGraph extends InMemoryGraph {
SIGNAL,
SET_VALUE,
MARK_DIRTY,
+ MARK_CLEAN,
IS_CHANGED,
GET_VALUE_WITH_METADATA,
IS_DIRTY
@@ -115,6 +116,14 @@ public class NotifyingInMemoryGraph extends InMemoryGraph {
}
@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();