aboutsummaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
authorGravatar Janak Ramakrishnan <janakr@google.com>2016-08-25 23:49:45 +0000
committerGravatar John Cater <jcater@google.com>2016-08-26 18:39:32 +0000
commit1ac4568f25f93548927db9080308b0d2e736f7a3 (patch)
tree32ab67af11de8ff304d23d7a4dc9d71eeeda9a51
parentaa9107b4fd1a8b8193428617bab4c694f24124a1 (diff)
Allow cycle detector to be injected into ParallelEvaluator, so that callers can use different cycle detection algorithms if they wish.
-- MOS_MIGRATED_REVID=131347160
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/CycleDetector.java27
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/NodeEntryVisitor.java2
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java26
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java41
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/SimpleCycleDetector.java10
-rw-r--r--src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java7
-rw-r--r--src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java173
7 files changed, 202 insertions, 84 deletions
diff --git a/src/main/java/com/google/devtools/build/skyframe/CycleDetector.java b/src/main/java/com/google/devtools/build/skyframe/CycleDetector.java
new file mode 100644
index 0000000000..6cd1585498
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/CycleDetector.java
@@ -0,0 +1,27 @@
+// 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;
+
+/**
+ * Detects cycles after a {@link ParallelEvaluator} evaluation that did not complete due to cycles.
+ *
+ * <p>Public only for the benefit of alternative graph implementations outside the skyframe package.
+ */
+public interface CycleDetector {
+ void checkForCycles(
+ Iterable<SkyKey> badRoots,
+ EvaluationResult.Builder<?> result,
+ ParallelEvaluatorContext evaluatorContext)
+ throws InterruptedException;
+}
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 6d0dbfa6ff..dc394ed61c 100644
--- a/src/main/java/com/google/devtools/build/skyframe/NodeEntryVisitor.java
+++ b/src/main/java/com/google/devtools/build/skyframe/NodeEntryVisitor.java
@@ -32,7 +32,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
* track of pending nodes.
*/
class NodeEntryVisitor {
- private static final ErrorClassifier NODE_ENTRY_VISITOR_ERROR_CLASSIFIER =
+ static final ErrorClassifier NODE_ENTRY_VISITOR_ERROR_CLASSIFIER =
new ErrorClassifier() {
@Override
protected ErrorClassification classifyException(Exception e) {
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 b800207690..aee81ead3b 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -33,6 +33,7 @@ import com.google.devtools.build.skyframe.EvaluationProgressReceiver.EvaluationS
import com.google.devtools.build.skyframe.MemoizingEvaluator.EmittedEventState;
import com.google.devtools.build.skyframe.NodeEntry.DependencyState;
import com.google.devtools.build.skyframe.NodeEntry.DirtyState;
+import com.google.devtools.build.skyframe.ParallelEvaluatorContext.EnqueueParentBehavior;
import com.google.devtools.build.skyframe.QueryableGraph.Reason;
import com.google.devtools.build.skyframe.SkyFunctionException.ReifiedSkyFunctionException;
import java.util.ArrayList;
@@ -93,7 +94,7 @@ public final class ParallelEvaluator implements Evaluator {
private final Receiver<Collection<SkyKey>> inflightKeysReceiver;
private final ParallelEvaluatorContext evaluatorContext;
- private final SimpleCycleDetector cycleDetector = new SimpleCycleDetector();
+ private final CycleDetector cycleDetector;
public ParallelEvaluator(
ProcessableGraph graph,
@@ -124,6 +125,7 @@ public final class ParallelEvaluator implements Evaluator {
dirtyKeyTracker,
createEvaluateRunnable(),
threadCount);
+ cycleDetector = new SimpleCycleDetector();
}
public ParallelEvaluator(
@@ -138,9 +140,11 @@ public final class ParallelEvaluator implements Evaluator {
@Nullable EvaluationProgressReceiver progressReceiver,
DirtyKeyTracker dirtyKeyTracker,
Receiver<Collection<SkyKey>> inflightKeysReceiver,
- ForkJoinPool forkJoinPool) {
+ ForkJoinPool forkJoinPool,
+ CycleDetector cycleDetector) {
this.graph = graph;
this.inflightKeysReceiver = inflightKeysReceiver;
+ this.cycleDetector = cycleDetector;
Preconditions.checkState(storeErrorsAlongsideValues || keepGoing);
this.dirtyKeyTracker = Preconditions.checkNotNull(dirtyKeyTracker);
evaluatorContext =
@@ -341,7 +345,7 @@ public final class ParallelEvaluator implements Evaluator {
throw SchedulerException.ofError(state.getErrorInfo(), skyKey);
}
evaluatorContext.signalValuesAndEnqueueIfReady(
- skyKey, reverseDeps, state.getVersion(), /*enqueueParents=*/ true);
+ skyKey, reverseDeps, state.getVersion(), EnqueueParentBehavior.ENQUEUE);
return DirtyOutcome.ALREADY_PROCESSED;
case NEEDS_REBUILDING:
maybeMarkRebuilding(state);
@@ -425,7 +429,11 @@ public final class ParallelEvaluator implements Evaluator {
state,
errorInfo,
/*isDirectlyTransient=*/ reifiedBuilderException.isTransient());
- env.commit(state, /*enqueueParents=*/ evaluatorContext.keepGoing());
+ env.commit(
+ state,
+ evaluatorContext.keepGoing()
+ ? EnqueueParentBehavior.ENQUEUE
+ : EnqueueParentBehavior.SIGNAL);
if (!shouldFailFast) {
return;
}
@@ -463,7 +471,7 @@ public final class ParallelEvaluator implements Evaluator {
graph.getBatch(skyKey, Reason.RDEP_ADDITION, env.getNewlyRequestedDeps()),
oldDeps,
env);
- env.commit(state, /*enqueueParents=*/true);
+ env.commit(state, EnqueueParentBehavior.ENQUEUE);
return;
}
@@ -530,10 +538,10 @@ public final class ParallelEvaluator implements Evaluator {
skyKey,
state,
env.getChildErrorInfos());
- // If the child error was catastrophic, committing this parent to the graph is not
- // necessary, but since we don't do error bubbling in catastrophes, it doesn't violate any
- // invariants either.
- env.commit(state, /*enqueueParents=*/ true);
+ // If the child error was catastrophic, committing this parent to the graph is not
+ // necessary, but since we don't do error bubbling in catastrophes, it doesn't violate any
+ // invariants either.
+ env.commit(state, EnqueueParentBehavior.ENQUEUE);
return;
}
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 dbfd56760e..f364a71f4b 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java
@@ -36,6 +36,12 @@ import javax.annotation.Nullable;
* <p>Also used during cycle detection.
*/
class ParallelEvaluatorContext {
+ enum EnqueueParentBehavior {
+ ENQUEUE,
+ SIGNAL,
+ NO_ACTION
+ }
+
private final QueryableGraph graph;
private final Version graphVersion;
private final ImmutableMap<SkyFunctionName, ? extends SkyFunction> skyFunctions;
@@ -133,27 +139,34 @@ class ParallelEvaluatorContext {
* cycles).
*/
void signalValuesAndEnqueueIfReady(
- SkyKey skyKey, Iterable<SkyKey> keys, Version version, boolean enqueueParents)
+ SkyKey skyKey, Iterable<SkyKey> keys, Version version, EnqueueParentBehavior enqueueParents)
throws InterruptedException {
// No fields of the entry are needed here, since we're just enqueuing for evaluation, but more
// importantly, these hints are not respected for not-done nodes. If they are, we may need to
// alter this hint.
Map<SkyKey, ? extends NodeEntry> batch = graph.getBatch(skyKey, Reason.SIGNAL_DEP, keys);
- if (enqueueParents) {
- for (SkyKey key : keys) {
- NodeEntry entry = Preconditions.checkNotNull(batch.get(key), key);
- if (entry.signalDep(version)) {
- getVisitor().enqueueEvaluation(key);
+ switch (enqueueParents) {
+ case ENQUEUE:
+ for (SkyKey key : keys) {
+ NodeEntry entry = Preconditions.checkNotNull(batch.get(key), key);
+ if (entry.signalDep(version)) {
+ getVisitor().enqueueEvaluation(key);
+ }
}
- }
- } else {
- for (SkyKey key : keys) {
- NodeEntry entry = Preconditions.checkNotNull(batch.get(key), key);
- if (!entry.isDone()) {
- // In cycles, we can have parents that are already done.
- entry.signalDep(version);
+ return;
+ case SIGNAL:
+ for (SkyKey key : keys) {
+ NodeEntry entry = Preconditions.checkNotNull(batch.get(key), key);
+ if (!entry.isDone()) {
+ // In cycles, we can have parents that are already done.
+ entry.signalDep(version);
+ }
}
- }
+ return;
+ case NO_ACTION:
+ return;
+ default:
+ throw new IllegalStateException(enqueueParents + ", " + skyKey);
}
}
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 6a736b2f4f..1c41fa56c8 100644
--- a/src/main/java/com/google/devtools/build/skyframe/SimpleCycleDetector.java
+++ b/src/main/java/com/google/devtools/build/skyframe/SimpleCycleDetector.java
@@ -25,6 +25,7 @@ import com.google.common.collect.Sets;
import com.google.devtools.build.lib.profiler.AutoProfiler;
import com.google.devtools.build.lib.util.GroupedList;
import com.google.devtools.build.lib.util.Preconditions;
+import com.google.devtools.build.skyframe.ParallelEvaluatorContext.EnqueueParentBehavior;
import com.google.devtools.build.skyframe.QueryableGraph.Reason;
import java.util.ArrayDeque;
import java.util.ArrayList;
@@ -41,10 +42,11 @@ import javax.annotation.Nullable;
* Depth-first implementation of cycle detection after a {@link ParallelEvaluator} evaluation has
* completed with at least one root unfinished.
*/
-class SimpleCycleDetector {
+class SimpleCycleDetector implements CycleDetector {
private static final Logger logger = Logger.getLogger(SimpleCycleDetector.class.getName());
- void checkForCycles(
+ @Override
+ public void checkForCycles(
Iterable<SkyKey> badRoots,
EvaluationResult.Builder<?> result,
ParallelEvaluatorContext evaluatorContext)
@@ -161,7 +163,7 @@ class SimpleCycleDetector {
evaluatorContext);
env.setError(
entry, ErrorInfo.fromChildErrors(key, errorDeps), /*isDirectlyTransient=*/ false);
- env.commit(entry, /*enqueueParents=*/ false);
+ env.commit(entry, EnqueueParentBehavior.SIGNAL);
} else {
entry = evaluatorContext.getGraph().get(null, Reason.CYCLE_CHECKING, key);
}
@@ -220,7 +222,7 @@ class SimpleCycleDetector {
// Add in this cycle.
allErrors.add(ErrorInfo.fromCycle(cycleInfo));
env.setError(entry, ErrorInfo.fromChildErrors(key, allErrors), /*isTransient=*/ false);
- env.commit(entry, /*enqueueParents=*/ false);
+ env.commit(entry, EnqueueParentBehavior.SIGNAL);
continue;
} else {
// We need to return right away in the noKeepGoing case, so construct the cycle (with the
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 ec84b1b33e..a15522025b 100644
--- a/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionEnvironment.java
@@ -35,6 +35,7 @@ import com.google.devtools.build.lib.events.StoredEventHandler;
import com.google.devtools.build.lib.util.GroupedList;
import com.google.devtools.build.lib.util.GroupedList.GroupedListHelper;
import com.google.devtools.build.lib.util.Preconditions;
+import com.google.devtools.build.skyframe.ParallelEvaluatorContext.EnqueueParentBehavior;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -515,7 +516,8 @@ class SkyFunctionEnvironment extends AbstractSkyFunctionEnvironment {
* <p>The node entry is informed if the node's value and error are definitive via the flag {@code
* completeValue}.
*/
- void commit(NodeEntry primaryEntry, boolean enqueueParents) throws InterruptedException {
+ void commit(NodeEntry primaryEntry, EnqueueParentBehavior enqueueParents)
+ throws InterruptedException {
// Construct the definitive error info, if there is one.
finalizeErrorInfo();
@@ -534,7 +536,8 @@ class SkyFunctionEnvironment extends AbstractSkyFunctionEnvironment {
valueWithMetadata = ValueWithMetadata.error(errorInfo, events);
} else {
// We must be enqueueing parents if we have a value.
- Preconditions.checkState(enqueueParents, "%s %s", skyKey, primaryEntry);
+ Preconditions.checkState(
+ enqueueParents == EnqueueParentBehavior.ENQUEUE, "%s %s", skyKey, primaryEntry);
valueWithMetadata = ValueWithMetadata.normal(value, errorInfo, events);
}
if (!oldDeps.isEmpty()) {
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 90b67816e1..f0f34b4276 100644
--- a/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/MemoizingEvaluatorTest.java
@@ -130,6 +130,10 @@ public class MemoizingEvaluatorTest {
return true;
}
+ protected boolean cyclesDetected() {
+ return true;
+ }
+
private void initializeReporter() {
eventCollector = new EventCollector();
reporter = eventCollector;
@@ -908,16 +912,20 @@ public class MemoizingEvaluatorTest {
assertEquals(null, result.get(topKey));
ErrorInfo errorInfo = result.getError(topKey);
CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
- assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
+ }
tester.invalidate();
result = tester.eval(/*keepGoing=*/false, topKey, goodKey);
assertEquals(null, result.get(topKey));
errorInfo = result.getError(topKey);
cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
- assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
+ }
}
@Test
@@ -927,15 +935,19 @@ public class MemoizingEvaluatorTest {
EvaluationResult<StringValue> result = tester.eval(/*keepGoing=*/ true, selfEdge);
assertThatEvaluationResult(result).hasError();
CycleInfo cycleInfo = Iterables.getOnlyElement(result.getError(selfEdge).getCycleInfo());
- CycleInfoSubjectFactory.assertThat(cycleInfo).hasCycleThat().containsExactly(selfEdge);
- CycleInfoSubjectFactory.assertThat(cycleInfo).hasPathToCycleThat().isEmpty();
+ if (cyclesDetected()) {
+ CycleInfoSubjectFactory.assertThat(cycleInfo).hasCycleThat().containsExactly(selfEdge);
+ CycleInfoSubjectFactory.assertThat(cycleInfo).hasPathToCycleThat().isEmpty();
+ }
SkyKey parent = GraphTester.toSkyKey("parent");
tester.getOrCreate(parent).addDependency(selfEdge).setComputedValue(CONCATENATE);
EvaluationResult<StringValue> result2 = tester.eval(/*keepGoing=*/ true, parent);
assertThatEvaluationResult(result).hasError();
CycleInfo cycleInfo2 = Iterables.getOnlyElement(result2.getError(parent).getCycleInfo());
- CycleInfoSubjectFactory.assertThat(cycleInfo2).hasCycleThat().containsExactly(selfEdge);
- CycleInfoSubjectFactory.assertThat(cycleInfo2).hasPathToCycleThat().containsExactly(parent);
+ if (cyclesDetected()) {
+ CycleInfoSubjectFactory.assertThat(cycleInfo2).hasCycleThat().containsExactly(selfEdge);
+ CycleInfoSubjectFactory.assertThat(cycleInfo2).hasPathToCycleThat().containsExactly(parent);
+ }
}
private void changeCycle(boolean keepGoing) throws Exception {
@@ -952,8 +964,10 @@ public class MemoizingEvaluatorTest {
assertEquals(null, result.get(topKey));
ErrorInfo errorInfo = result.getError(topKey);
CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
- assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
+ }
tester.getOrCreate(bKey).removeDependency(aKey);
tester.set(bKey, new StringValue("bValue"));
@@ -1009,14 +1023,23 @@ public class MemoizingEvaluatorTest {
EvaluationResult<StringValue> result = tester.eval(/*keepGoing=*/ true, aKey);
// aKey has an error,
assertEquals(null, result.get(aKey));
- // And both cycles were found underneath aKey: the (aKey->bKey->cKey) cycle, and the
- // aKey->(bKey->cKey) cycle. This is because cKey depended on aKey and then bKey, so it pushed
- // them down on the stack in that order, so bKey was processed first. It found its cycle, then
- // popped off the stack, and then aKey was processed and found its cycle.
- assertThat(result.getError(aKey).getCycleInfo())
- .containsExactly(
- new CycleInfo(ImmutableList.of(aKey, bKey, cKey)),
- new CycleInfo(ImmutableList.of(aKey), ImmutableList.of(bKey, cKey)));
+ if (cyclesDetected()) {
+ // And both cycles were found underneath aKey: the (aKey->bKey->cKey) cycle, and the
+ // aKey->(bKey->cKey) cycle. This is because cKey depended on aKey and then bKey, so it pushed
+ // them down on the stack in that order, so bKey was processed first. It found its cycle, then
+ // popped off the stack, and then aKey was processed and found its cycle.
+ assertThatEvaluationResult(result)
+ .hasErrorEntryForKeyThat(aKey)
+ .hasCycleInfoThat()
+ .containsExactly(
+ new CycleInfo(ImmutableList.of(aKey, bKey, cKey)),
+ new CycleInfo(ImmutableList.of(aKey), ImmutableList.of(bKey, cKey)));
+ } else {
+ assertThatEvaluationResult(result)
+ .hasErrorEntryForKeyThat(aKey)
+ .hasCycleInfoThat()
+ .hasSize(1);
+ }
// When leafKey is changed, so that aKey will be marked as NEEDS_REBUILDING,
tester.set(leafKey, new StringValue("crunchy"));
// And cKey is invalidated, so that cycle checking will have to explore the full graph,
@@ -1026,10 +1049,19 @@ public class MemoizingEvaluatorTest {
EvaluationResult<StringValue> result2 = tester.eval(/*keepGoing=*/ true, aKey);
// Things are just as before.
assertEquals(null, result2.get(aKey));
- assertThat(result2.getError(aKey).getCycleInfo())
- .containsExactly(
- new CycleInfo(ImmutableList.of(aKey, bKey, cKey)),
- new CycleInfo(ImmutableList.of(aKey), ImmutableList.of(bKey, cKey)));
+ if (cyclesDetected()) {
+ assertThatEvaluationResult(result)
+ .hasErrorEntryForKeyThat(aKey)
+ .hasCycleInfoThat()
+ .containsExactly(
+ new CycleInfo(ImmutableList.of(aKey, bKey, cKey)),
+ new CycleInfo(ImmutableList.of(aKey), ImmutableList.of(bKey, cKey)));
+ } else {
+ assertThatEvaluationResult(result)
+ .hasErrorEntryForKeyThat(aKey)
+ .hasCycleInfoThat()
+ .hasSize(1);
+ }
}
/** Regression test: "crash in cycle checker with dirty values". */
@@ -1048,21 +1080,27 @@ public class MemoizingEvaluatorTest {
assertEquals(null, result.get(cycleKey1));
ErrorInfo errorInfo = result.getError(cycleKey1);
CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1).inOrder();
- assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ }
tester.getOrCreate(cycleKey1, /*markAsModified=*/true);
tester.invalidate();
result = tester.eval(/*keepGoing=*/true, cycleKey1, cycleKey2);
assertEquals(null, result.get(cycleKey1));
errorInfo = result.getError(cycleKey1);
cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1).inOrder();
- assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ }
cycleInfo =
Iterables.getOnlyElement(
tester.driver.getExistingErrorForTesting(cycleKey2).getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1).inOrder();
- assertThat(cycleInfo.getPathToCycle()).containsExactly(cycleKey2).inOrder();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).containsExactly(cycleKey2).inOrder();
+ }
}
@Test
@@ -1098,8 +1136,10 @@ public class MemoizingEvaluatorTest {
assertEquals(null, result.get(cycleKey1));
ErrorInfo errorInfo = result.getError(cycleKey1);
CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1).inOrder();
- assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ }
}
}
@@ -1115,16 +1155,20 @@ public class MemoizingEvaluatorTest {
assertEquals(null, result.get(cycleKey1));
ErrorInfo errorInfo = result.getError(cycleKey1);
CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1, cycleKey2).inOrder();
- assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1, cycleKey2).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ }
tester.getOrCreate(cycleKey1, /*markAsModified=*/true);
tester.invalidate();
result = tester.eval(/*keepGoing=*/true, cycleKey1);
assertEquals(null, result.get(cycleKey1));
errorInfo = result.getError(cycleKey1);
cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
- assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1, cycleKey2).inOrder();
- assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ if (cyclesDetected()) {
+ assertThat(cycleInfo.getCycle()).containsExactly(cycleKey1, cycleKey2).inOrder();
+ assertThat(cycleInfo.getPathToCycle()).isEmpty();
+ }
}
/**
@@ -1161,10 +1205,14 @@ public class MemoizingEvaluatorTest {
assertThatErrorInfo(errorInfo).isNotTransient();
assertThatErrorInfo(errorInfo).hasExceptionThat().isNull();
}
- assertWithMessage(errorInfo.toString())
- .that(errorInfo.getCycleInfo())
- .containsExactly(
- new CycleInfo(ImmutableList.of(top), ImmutableList.of(cycleKey1, cycleKey2)));
+ if (cyclesDetected()) {
+ assertThatErrorInfo(errorInfo)
+ .hasCycleInfoThat()
+ .containsExactly(
+ new CycleInfo(ImmutableList.of(top), ImmutableList.of(cycleKey1, cycleKey2)));
+ } else {
+ assertThatErrorInfo(errorInfo).hasCycleInfoThat().hasSize(1);
+ }
// But the parent itself shouldn't have a direct dep on the special error transience node.
assertThatEvaluationResult(evalResult)
.hasDirectDepsInGraphThat(top)
@@ -1276,12 +1324,14 @@ public class MemoizingEvaluatorTest {
// they appear here.
EvaluationResult<StringValue> result =
tester.eval(/*keepGoing=*/false, otherTop, topKey, exceptionMarker);
- assertThat(result.errorMap().keySet()).containsExactly(topKey);
Iterable<CycleInfo> cycleInfos = result.getError(topKey).getCycleInfo();
assertWithMessage(result.toString()).that(cycleInfos).isNotEmpty();
CycleInfo cycleInfo = Iterables.getOnlyElement(cycleInfos);
- assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey);
- assertThat(cycleInfo.getCycle()).containsExactly(cycle1Key, cycle2Key);
+ if (cyclesDetected()) {
+ assertThat(result.errorMap().keySet()).containsExactly(topKey);
+ assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey);
+ assertThat(cycleInfo.getCycle()).containsExactly(cycle1Key, cycle2Key);
+ }
}
@Test
@@ -1302,15 +1352,19 @@ public class MemoizingEvaluatorTest {
.hasCycleInfoThat()
.isNotEmpty();
CycleInfo aCycleInfo = Iterables.getOnlyElement(result.getError(aKey).getCycleInfo());
- assertThat(aCycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
- assertThat(aCycleInfo.getPathToCycle()).isEmpty();
+ if (cyclesDetected()) {
+ assertThat(aCycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
+ assertThat(aCycleInfo.getPathToCycle()).isEmpty();
+ }
assertThatEvaluationResult(result)
.hasErrorEntryForKeyThat(bKey)
.hasCycleInfoThat()
.isNotEmpty();
CycleInfo bCycleInfo = Iterables.getOnlyElement(result.getError(bKey).getCycleInfo());
- assertThat(bCycleInfo.getCycle()).containsExactly(bKey, aKey).inOrder();
- assertThat(bCycleInfo.getPathToCycle()).isEmpty();
+ if (cyclesDetected()) {
+ assertThat(bCycleInfo.getCycle()).containsExactly(bKey, aKey).inOrder();
+ assertThat(bCycleInfo.getPathToCycle()).isEmpty();
+ }
// When both dependencies are broken,
tester.getOrCreate(bKey).removeDependency(aKey);
@@ -1366,15 +1420,26 @@ public class MemoizingEvaluatorTest {
// Then evaluation is as expected -- topKey has removed its dep on depKey (since depKey was not
// done when topKey found its cycle), and both topKey and depKey have cycles.
EvaluationResult<StringValue> result2 = tester.eval(/*keepGoing=*/ true, topKey, depKey);
- assertThatEvaluationResult(result2)
- .hasErrorEntryForKeyThat(topKey)
- .hasCycleInfoThat()
- .containsExactly(new CycleInfo(ImmutableList.of(topKey)));
- assertThatEvaluationResult(result2).hasDirectDepsInGraphThat(topKey).containsExactly(topKey);
- assertThatEvaluationResult(result2)
- .hasErrorEntryForKeyThat(depKey)
- .hasCycleInfoThat()
- .containsExactly(new CycleInfo(ImmutableList.of(depKey)));
+ if (cyclesDetected()) {
+ assertThatEvaluationResult(result2)
+ .hasErrorEntryForKeyThat(topKey)
+ .hasCycleInfoThat()
+ .containsExactly(new CycleInfo(ImmutableList.of(topKey)));
+ assertThatEvaluationResult(result2).hasDirectDepsInGraphThat(topKey).containsExactly(topKey);
+ assertThatEvaluationResult(result2)
+ .hasErrorEntryForKeyThat(depKey)
+ .hasCycleInfoThat()
+ .containsExactly(new CycleInfo(ImmutableList.of(depKey)));
+ } else {
+ assertThatEvaluationResult(result2)
+ .hasErrorEntryForKeyThat(topKey)
+ .hasCycleInfoThat()
+ .hasSize(1);
+ assertThatEvaluationResult(result2)
+ .hasErrorEntryForKeyThat(depKey)
+ .hasCycleInfoThat()
+ .hasSize(1);
+ }
}
@Test