Allow Skyframe graph lookups and value retrievals to throw InterruptedException.
The only place we now don't handle InterruptedException is in the action graph created after analysis, since I'm not sure that will be around for that much longer.
--
MOS_MIGRATED_REVID=130327770
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 7353e54..834a614 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -31,7 +31,6 @@
import com.google.devtools.build.lib.collect.nestedset.NestedSetVisitor;
import com.google.devtools.build.lib.concurrent.AbstractQueueVisitor;
import com.google.devtools.build.lib.concurrent.ErrorClassifier;
-import com.google.devtools.build.lib.concurrent.ErrorHandler;
import com.google.devtools.build.lib.concurrent.ForkJoinQuiescingExecutor;
import com.google.devtools.build.lib.concurrent.QuiescingExecutor;
import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadCompatible;
@@ -132,7 +131,16 @@
@Override
public SkyValue get() {
- return state.getValue();
+ try {
+ return state.getValue();
+ } catch (InterruptedException e) {
+ throw new IllegalStateException(
+ "Graph implementations in which value retrieval can block should not be used in "
+ + "frameworks that use the value in EvaluationProgressReceiver, since that could "
+ + "result in significant slowdowns: "
+ + state,
+ e);
+ }
}
}
@@ -157,7 +165,6 @@
private final DirtyKeyTracker dirtyKeyTracker;
private final Receiver<Collection<SkyKey>> inflightKeysReceiver;
private final EventFilter storedEventFilter;
- private final ErrorHandler errorHandler;
public ParallelEvaluator(
ProcessableGraph graph,
@@ -185,7 +192,6 @@
new NestedSetVisitor<>(new NestedSetEventReceiver(reporter), emittedEventState);
this.storedEventFilter = storedEventFilter;
this.forkJoinPool = null;
- this.errorHandler = ErrorHandler.NullHandler.INSTANCE;
}
public ParallelEvaluator(
@@ -200,8 +206,7 @@
@Nullable EvaluationProgressReceiver progressReceiver,
DirtyKeyTracker dirtyKeyTracker,
Receiver<Collection<SkyKey>> inflightKeysReceiver,
- ForkJoinPool forkJoinPool,
- ErrorHandler errorHandler) {
+ ForkJoinPool forkJoinPool) {
this.graph = graph;
this.skyFunctions = skyFunctions;
this.graphVersion = graphVersion;
@@ -217,13 +222,10 @@
new NestedSetVisitor<>(new NestedSetEventReceiver(reporter), emittedEventState);
this.storedEventFilter = storedEventFilter;
this.forkJoinPool = Preconditions.checkNotNull(forkJoinPool);
- this.errorHandler = errorHandler;
}
- private Map<SkyKey, NodeEntry> getBatchValues(
- SkyKey parent,
- Reason reason,
- Iterable<SkyKey> keys) {
+ private Map<SkyKey, ? extends NodeEntry> getBatchValues(
+ SkyKey parent, Reason reason, Iterable<SkyKey> keys) throws InterruptedException {
return graph.getBatch(parent, reason, keys);
}
@@ -301,7 +303,8 @@
};
private SkyFunctionEnvironment(
- SkyKey skyKey, GroupedList<SkyKey> directDeps, Set<SkyKey> oldDeps, ValueVisitor visitor) {
+ SkyKey skyKey, GroupedList<SkyKey> directDeps, Set<SkyKey> oldDeps, ValueVisitor visitor)
+ throws InterruptedException {
this(skyKey, directDeps, null, oldDeps, visitor);
}
@@ -310,7 +313,8 @@
GroupedList<SkyKey> directDeps,
@Nullable Map<SkyKey, ValueWithMetadata> bubbleErrorInfo,
Set<SkyKey> oldDeps,
- ValueVisitor visitor) {
+ ValueVisitor visitor)
+ throws InterruptedException {
this.skyKey = skyKey;
this.oldDeps = oldDeps;
this.directDeps = Collections.unmodifiableMap(batchPrefetch(
@@ -323,9 +327,13 @@
skyKey);
}
- private Map<SkyKey, NodeEntry> batchPrefetch(
- SkyKey requestor, GroupedList<SkyKey> depKeys, Set<SkyKey> oldDeps, boolean assertDone,
- SkyKey keyForDebugging) {
+ private Map<SkyKey, ? extends NodeEntry> batchPrefetch(
+ SkyKey requestor,
+ GroupedList<SkyKey> depKeys,
+ Set<SkyKey> oldDeps,
+ boolean assertDone,
+ SkyKey keyForDebugging)
+ throws InterruptedException {
Iterable<SkyKey> depKeysAsIterable = Iterables.concat(depKeys);
Iterable<SkyKey> keysToPrefetch = depKeysAsIterable;
if (PREFETCH_OLD_DEPS) {
@@ -333,10 +341,12 @@
keysToPrefetchBuilder.addAll(depKeysAsIterable).addAll(oldDeps);
keysToPrefetch = keysToPrefetchBuilder.build();
}
- Map<SkyKey, NodeEntry> batchMap = getBatchValues(requestor, Reason.PREFETCH, keysToPrefetch);
+ Map<SkyKey, ? extends NodeEntry> batchMap =
+ getBatchValues(requestor, Reason.PREFETCH, keysToPrefetch);
if (PREFETCH_OLD_DEPS) {
- batchMap = ImmutableMap.copyOf(
- Maps.filterKeys(batchMap, Predicates.in(ImmutableSet.copyOf(depKeysAsIterable))));
+ batchMap =
+ ImmutableMap.<SkyKey, NodeEntry>copyOf(
+ Maps.filterKeys(batchMap, Predicates.in(ImmutableSet.copyOf(depKeysAsIterable))));
}
if (batchMap.size() != depKeys.numElements()) {
throw new IllegalStateException(
@@ -346,7 +356,7 @@
+ Sets.difference(depKeys.toSet(), batchMap.keySet()));
}
if (assertDone) {
- for (Map.Entry<SkyKey, NodeEntry> entry : batchMap.entrySet()) {
+ for (Map.Entry<SkyKey, ? extends NodeEntry> entry : batchMap.entrySet()) {
Preconditions.checkState(
entry.getValue().isDone(), "%s had not done %s", keyForDebugging, entry);
}
@@ -358,7 +368,8 @@
Preconditions.checkState(building, skyKey);
}
- private NestedSet<TaggedEvents> buildEvents(NodeEntry entry, boolean missingChildren) {
+ private NestedSet<TaggedEvents> buildEvents(NodeEntry entry, boolean missingChildren)
+ throws InterruptedException {
// Aggregate the nested set of events from the direct deps, also adding the events from
// building this value.
NestedSetBuilder<TaggedEvents> eventBuilder = NestedSetBuilder.stableOrder();
@@ -426,7 +437,8 @@
* dependencies of this node <i>must</i> already have been registered, since this method may
* register a dependence on the error transience node, which should always be the last dep.
*/
- private void setError(NodeEntry state, ErrorInfo errorInfo, boolean isDirectlyTransient) {
+ private void setError(NodeEntry state, ErrorInfo errorInfo, boolean isDirectlyTransient)
+ throws InterruptedException {
Preconditions.checkState(value == null, "%s %s %s", skyKey, value, errorInfo);
Preconditions.checkState(this.errorInfo == null,
"%s %s %s", skyKey, this.errorInfo, errorInfo);
@@ -454,7 +466,8 @@
@Nullable SkyKey requestor,
Iterable<SkyKey> keys,
@Nullable Map<SkyKey, ValueWithMetadata> bubbleErrorInfo,
- int keySize) {
+ int keySize)
+ throws InterruptedException {
ImmutableMap.Builder<SkyKey, SkyValue> builder = ImmutableMap.builder();
ArrayList<SkyKey> missingKeys = new ArrayList<>(keySize);
for (SkyKey key : keys) {
@@ -468,7 +481,7 @@
missingKeys.add(key);
}
}
- Map<SkyKey, NodeEntry> missingEntries =
+ Map<SkyKey, ? extends NodeEntry> missingEntries =
getBatchValues(requestor, Reason.DEP_REQUESTED, missingKeys);
for (SkyKey key : missingKeys) {
builder.put(key, maybeGetValueFromError(key, missingEntries.get(key), bubbleErrorInfo));
@@ -477,8 +490,8 @@
}
@Override
- protected Map<SkyKey, ValueOrUntypedException> getValueOrUntypedExceptions(
- Set<SkyKey> depKeys) {
+ protected Map<SkyKey, ValueOrUntypedException> getValueOrUntypedExceptions(Set<SkyKey> depKeys)
+ throws InterruptedException {
checkActive();
Preconditions.checkState(
!depKeys.contains(ErrorTransienceValue.KEY),
@@ -578,15 +591,20 @@
}
@Override
- public <E1 extends Exception, E2 extends Exception, E3 extends Exception,
- E4 extends Exception, E5 extends Exception>
+ public <
+ E1 extends Exception,
+ E2 extends Exception,
+ E3 extends Exception,
+ E4 extends Exception,
+ E5 extends Exception>
Map<SkyKey, ValueOrException5<E1, E2, E3, E4, E5>> getValuesOrThrow(
Iterable<SkyKey> depKeys,
Class<E1> exceptionClass1,
Class<E2> exceptionClass2,
Class<E3> exceptionClass3,
Class<E4> exceptionClass4,
- Class<E5> exceptionClass5) {
+ Class<E5> exceptionClass5)
+ throws InterruptedException {
newlyRequestedDeps.startGroup();
Map<SkyKey, ValueOrException5<E1, E2, E3, E4, E5>> result = super.getValuesOrThrow(
depKeys,
@@ -638,7 +656,7 @@
* <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) {
+ void commit(NodeEntry primaryEntry, boolean enqueueParents) throws InterruptedException {
// Construct the definitive error info, if there is one.
finalizeErrorInfo();
@@ -664,10 +682,8 @@
// Remove the rdep on this entry for each of its old deps that is no longer a direct dep.
Set<SkyKey> depsToRemove =
Sets.difference(oldDeps, primaryEntry.getTemporaryDirectDeps().toSet());
- Collection<NodeEntry> oldDepEntries = graph.getBatch(
- skyKey,
- Reason.RDEP_REMOVAL,
- depsToRemove).values();
+ Collection<? extends NodeEntry> oldDepEntries =
+ graph.getBatch(skyKey, Reason.RDEP_REMOVAL, depsToRemove).values();
for (NodeEntry oldDepEntry : oldDepEntries) {
oldDepEntry.removeReverseDep(skyKey);
}
@@ -754,7 +770,7 @@
private ValueVisitor(ForkJoinPool forkJoinPool) {
quiescingExecutor =
- new ForkJoinQuiescingExecutor(forkJoinPool, VALUE_VISITOR_ERROR_CLASSIFIER, errorHandler);
+ new ForkJoinQuiescingExecutor(forkJoinPool, VALUE_VISITOR_ERROR_CLASSIFIER);
}
private ValueVisitor(int threadCount) {
@@ -766,8 +782,7 @@
TimeUnit.SECONDS,
/*failFastOnException*/ true,
"skyframe-evaluator",
- VALUE_VISITOR_ERROR_CLASSIFIER,
- errorHandler);
+ VALUE_VISITOR_ERROR_CLASSIFIER);
}
private void waitForCompletion() throws InterruptedException {
@@ -884,14 +899,15 @@
* Returns true if this depGroup consists of the error transience value and the error transience
* value is newer than the entry, meaning that the entry must be re-evaluated.
*/
- private boolean invalidatedByErrorTransience(Collection<SkyKey> depGroup, NodeEntry entry) {
+ private boolean invalidatedByErrorTransience(Collection<SkyKey> depGroup, NodeEntry entry)
+ throws InterruptedException {
return depGroup.size() == 1
&& depGroup.contains(ErrorTransienceValue.KEY)
&& !graph.get(
null, Reason.OTHER, ErrorTransienceValue.KEY).getVersion().atMost(entry.getVersion());
}
- private DirtyOutcome maybeHandleDirtyNode(NodeEntry state) {
+ private DirtyOutcome maybeHandleDirtyNode(NodeEntry state) throws InterruptedException {
if (!state.isDirty()) {
return DirtyOutcome.NEEDS_EVALUATION;
}
@@ -934,9 +950,9 @@
// 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.
- Map<SkyKey, NodeEntry> entriesToCheck =
+ Map<SkyKey, ? extends NodeEntry> entriesToCheck =
graph.getBatch(skyKey, Reason.OTHER, directDepsToCheck);
- for (Map.Entry<SkyKey, NodeEntry> entry : entriesToCheck.entrySet()) {
+ for (Entry<SkyKey, ? extends NodeEntry> entry : entriesToCheck.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.
@@ -945,7 +961,7 @@
state.addTemporaryDirectDeps(GroupedListHelper.create(ImmutableList.of(errorKey)));
errorEntry.checkIfDoneForDirtyReverseDep(skyKey);
// Perform the necessary bookkeeping for any deps that are not being used.
- for (Map.Entry<SkyKey, NodeEntry> depEntry : entriesToCheck.entrySet()) {
+ for (Entry<SkyKey, ? extends NodeEntry> depEntry : entriesToCheck.entrySet()) {
if (!depEntry.getKey().equals(errorKey)) {
depEntry.getValue().removeReverseDep(skyKey);
}
@@ -969,8 +985,10 @@
// TODO(bazel-team): If this signals the current node, consider falling through to the
// VERIFIED_CLEAN case below directly, without scheduling a new Evaluate().
- for (Map.Entry<SkyKey, NodeEntry> e : graph.createIfAbsentBatch(
- skyKey, Reason.ENQUEUING_CHILD, directDepsToCheck).entrySet()) {
+ for (Map.Entry<SkyKey, ? extends NodeEntry> e :
+ graph
+ .createIfAbsentBatch(skyKey, Reason.ENQUEUING_CHILD, directDepsToCheck)
+ .entrySet()) {
SkyKey directDep = e.getKey();
NodeEntry directDepEntry = e.getValue();
enqueueChild(skyKey, state, directDep, directDepEntry, /*depAlreadyExists=*/ true);
@@ -1005,6 +1023,7 @@
@Override
public void run() {
+ try {
NodeEntry state = Preconditions.checkNotNull(
graph.get(null, Reason.EVALUATION, skyKey),
skyKey);
@@ -1047,8 +1066,8 @@
}
}
- Map<SkyKey, NodeEntry> newlyRequestedDeps =
- getBatchValues(skyKey, Reason.RDEP_ADDITION, env.newlyRequestedDeps);
+ Map<SkyKey, ? extends NodeEntry> newlyRequestedDeps =
+ getBatchValues(skyKey, Reason.RDEP_ADDITION, env.newlyRequestedDeps);
boolean isTransitivelyTransient = reifiedBuilderException.isTransient();
for (NodeEntry depEntry
: Iterables.concat(env.directDeps.values(), newlyRequestedDeps.values())) {
@@ -1073,11 +1092,6 @@
}
throw SchedulerException.ofError(errorInfo, skyKey);
}
- } catch (InterruptedException ie) {
- // InterruptedException cannot be thrown by Runnable.run, so we must wrap it.
- // Interrupts can be caught by both the Evaluator and the AbstractQueueVisitor.
- // The former will unwrap the IE and propagate it as is; the latter will throw a new IE.
- throw SchedulerException.ofInterruption(ie, skyKey);
} catch (RuntimeException re) {
// Programmer error (most likely NPE or a failed precondition in a SkyFunction). Output
// some context together with the exception.
@@ -1127,24 +1141,24 @@
skyKey,
state,
childErrorKey);
- if (newDirectDeps.contains(childErrorKey)) {
- // Add this dep if it was just requested. In certain rare race conditions (see
- // MemoizingEvaluatorTest.cachedErrorCausesRestart) this dep may have already been
- // requested.
- state.addTemporaryDirectDeps(GroupedListHelper.create(ImmutableList.of(childErrorKey)));
- DependencyState childErrorState;
- if (oldDeps.contains(childErrorKey)) {
- childErrorState = childErrorEntry.checkIfDoneForDirtyReverseDep(skyKey);
- } else {
- childErrorState = childErrorEntry.addReverseDepAndCheckIfDone(skyKey);
- }
- Preconditions.checkState(
- childErrorState == DependencyState.DONE,
- "skyKey: %s, state: %s childErrorKey: %s",
- skyKey,
- state,
- childErrorKey,
- childErrorEntry);
+ if (newDirectDeps.contains(childErrorKey)) {
+ // Add this dep if it was just requested. In certain rare race conditions (see
+ // MemoizingEvaluatorTest.cachedErrorCausesRestart) this dep may have already been
+ // requested.
+ state.addTemporaryDirectDeps(GroupedListHelper.create(ImmutableList.of(childErrorKey)));
+ DependencyState childErrorState;
+ if (oldDeps.contains(childErrorKey)) {
+ childErrorState = childErrorEntry.checkIfDoneForDirtyReverseDep(skyKey);
+ } else {
+ childErrorState = childErrorEntry.addReverseDepAndCheckIfDone(skyKey);
+ }
+ Preconditions.checkState(
+ childErrorState == DependencyState.DONE,
+ "skyKey: %s, state: %s childErrorKey: %s",
+ skyKey,
+ state,
+ childErrorKey,
+ childErrorEntry);
}
ErrorInfo childErrorInfo = Preconditions.checkNotNull(childErrorEntry.getErrorInfo());
visitor.preventNewEvaluations();
@@ -1180,8 +1194,8 @@
return;
}
- for (Map.Entry<SkyKey, 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(
@@ -1191,6 +1205,12 @@
newDirectDepEntry,
/*depAlreadyExists=*/ oldDeps.contains(newDirectDep));
}
+ } catch (InterruptedException ie) {
+ // InterruptedException cannot be thrown by Runnable.run, so we must wrap it.
+ // Interrupts can be caught by both the Evaluator and the AbstractQueueVisitor.
+ // The former will unwrap the IE and propagate it as is; the latter will throw a new IE.
+ throw SchedulerException.ofInterruption(ie, skyKey);
+ }
// It is critical that there is no code below this point.
}
@@ -1219,17 +1239,16 @@
/**
* Signals all parents that this node is finished. If visitor is not null, also enqueues any
- * parents that are ready. If visitor is null, indicating that we are building this node after
- * the main build aborted, then skip any parents that are already done (that can happen with
- * cycles).
+ * parents that are ready. If visitor is null, indicating that we are building this node after the
+ * main build aborted, then skip any parents that are already done (that can happen with cycles).
*/
private void signalValuesAndEnqueueIfReady(
- @Nullable ValueVisitor visitor, SkyKey skyKey, Iterable<SkyKey> keys, Version version) {
+ @Nullable ValueVisitor visitor, SkyKey skyKey, Iterable<SkyKey> keys, Version version)
+ 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, NodeEntry> batch =
- graph.getBatch(skyKey, Reason.SIGNAL_DEP, keys);
+ Map<SkyKey, ? extends NodeEntry> batch = graph.getBatch(skyKey, Reason.SIGNAL_DEP, keys);
if (visitor != null) {
for (SkyKey key : keys) {
NodeEntry entry = Preconditions.checkNotNull(batch.get(key), key);
@@ -1252,7 +1271,8 @@
* If child is not done, removes {@param inProgressParent} from {@param child}'s reverse deps.
* Returns whether child should be removed from inProgressParent's entry's direct deps.
*/
- private boolean removeIncompleteChildForCycle(SkyKey inProgressParent, SkyKey child) {
+ private boolean removeIncompleteChildForCycle(SkyKey inProgressParent, SkyKey child)
+ throws InterruptedException {
NodeEntry childEntry = graph.get(inProgressParent, Reason.CYCLE_CHECKING, child);
if (!isDoneForBuild(childEntry)) {
childEntry.removeInProgressReverseDep(inProgressParent);
@@ -1272,7 +1292,7 @@
private static void registerNewlyDiscoveredDepsForDoneEntry(
SkyKey skyKey,
NodeEntry entry,
- Map<SkyKey, NodeEntry> newlyRequestedDepMap,
+ Map<SkyKey, ? extends NodeEntry> newlyRequestedDepMap,
Set<SkyKey> oldDeps,
SkyFunctionEnvironment env) {
Set<SkyKey> unfinishedDeps = new HashSet<>();
@@ -1300,7 +1320,8 @@
Preconditions.checkState(entry.isReady(), "%s %s %s", skyKey, entry, env.newlyRequestedDeps);
}
- private void informProgressReceiverThatValueIsDone(SkyKey key, NodeEntry entry) {
+ private void informProgressReceiverThatValueIsDone(SkyKey key, NodeEntry entry)
+ throws InterruptedException {
if (progressReceiver != null) {
Preconditions.checkState(entry.isDone(), entry);
SkyValue value = entry.getValue();
@@ -1327,7 +1348,8 @@
// directly without launching the heavy machinery, spawning threads, etc.
// Inform progressReceiver that these nodes are done to be consistent with the main code path.
boolean allAreDone = true;
- Map<SkyKey, NodeEntry> batch = getBatchValues(null, Reason.PRE_OR_POST_EVALUATION, skyKeySet);
+ Map<SkyKey, ? extends NodeEntry> batch =
+ getBatchValues(null, Reason.PRE_OR_POST_EVALUATION, skyKeySet);
for (SkyKey key : skyKeySet) {
if (!isDoneForBuild(batch.get(key))) {
allAreDone = false;
@@ -1401,8 +1423,8 @@
graph,
dirtyKeyTracker);
}
- for (Map.Entry<SkyKey, NodeEntry> e
- : graph.createIfAbsentBatch(null, Reason.PRE_OR_POST_EVALUATION, skyKeys).entrySet()) {
+ for (Entry<SkyKey, ? extends NodeEntry> e :
+ graph.createIfAbsentBatch(null, Reason.PRE_OR_POST_EVALUATION, skyKeys).entrySet()) {
SkyKey skyKey = e.getKey();
NodeEntry entry = e.getValue();
// This must be equivalent to the code in enqueueChild above, in order to be thread-safe.
@@ -1477,23 +1499,31 @@
}
/**
- * Walk up graph to find a top-level node (without parents) that wanted this failure. Store
- * the failed nodes along the way in a map, with ErrorInfos that are appropriate for that layer.
+ * Walk up graph to find a top-level node (without parents) that wanted this failure. Store the
+ * failed nodes along the way in a map, with ErrorInfos that are appropriate for that layer.
* Example:
+ *
+ * <pre>
* foo bar
* \ /
* unrequested baz
* \ |
* failed-node
+ * </pre>
+ *
* User requests foo, bar. When failed-node fails, we look at its parents. unrequested is not
* in-flight, so we replace failed-node by baz and repeat. We look at baz's parents. foo is
- * in-flight, so we replace baz by foo. Since foo is a top-level node and doesn't have parents,
- * we then break, since we know a top-level node, foo, that depended on the failed node.
+ * in-flight, so we replace baz by foo. Since foo is a top-level node and doesn't have parents, we
+ * then break, since we know a top-level node, foo, that depended on the failed node.
*
- * There's the potential for a weird "track jump" here in the case:
+ * <p>There's the potential for a weird "track jump" here in the case:
+ *
+ * <pre>
* foo
* / \
* fail1 fail2
+ * </pre>
+ *
* If fail1 and fail2 fail simultaneously, fail2 may start propagating up in the loop below.
* However, foo requests fail1 first, and then throws an exception based on that. This is not
* incorrect, but may be unexpected.
@@ -1506,8 +1536,9 @@
* <p>Note that we are not propagating error to the first top-level node but to the highest one,
* because during this process we can add useful information about error from other nodes.
*/
- private Map<SkyKey, ValueWithMetadata> bubbleErrorUp(final ErrorInfo leafFailure,
- SkyKey errorKey, Iterable<SkyKey> skyKeys, ValueVisitor visitor) {
+ private Map<SkyKey, ValueWithMetadata> bubbleErrorUp(
+ final ErrorInfo leafFailure, SkyKey errorKey, Iterable<SkyKey> skyKeys, ValueVisitor visitor)
+ throws InterruptedException {
Set<SkyKey> rootValues = ImmutableSet.copyOf(skyKeys);
ErrorInfo error = leafFailure;
Map<SkyKey, ValueWithMetadata> bubbleErrorInfo = new HashMap<>();
@@ -1651,9 +1682,9 @@
}
/**
- * Constructs an {@link EvaluationResult} from the {@link #graph}. Looks for cycles if there
- * are unfinished nodes but no error was already found through bubbling up
- * (as indicated by {@code bubbleErrorInfo} being null).
+ * Constructs an {@link EvaluationResult} from the {@link #graph}. Looks for cycles if there are
+ * unfinished nodes but no error was already found through bubbling up (as indicated by {@code
+ * bubbleErrorInfo} being null).
*
* <p>{@code visitor} may be null, but only in the case where all graph entries corresponding to
* {@code skyKeys} are known to be in the DONE state ({@code entry.isDone()} returns true).
@@ -1662,7 +1693,8 @@
@Nullable ValueVisitor visitor,
Iterable<SkyKey> skyKeys,
@Nullable Map<SkyKey, ValueWithMetadata> bubbleErrorInfo,
- boolean catastrophe) {
+ boolean catastrophe)
+ throws InterruptedException {
Preconditions.checkState(
catastrophe == (keepGoing && bubbleErrorInfo != null),
"Catastrophe not consistent with keepGoing mode and bubbleErrorInfo: %s %s %s %s",
@@ -1739,8 +1771,11 @@
}
private <T extends SkyValue> void checkForCycles(
- Iterable<SkyKey> badRoots, EvaluationResult.Builder<T> result, final ValueVisitor visitor,
- boolean keepGoing) {
+ Iterable<SkyKey> badRoots,
+ EvaluationResult.Builder<T> result,
+ final ValueVisitor visitor,
+ boolean keepGoing)
+ throws InterruptedException {
try (AutoProfiler p = AutoProfiler.logged("Checking for Skyframe cycles", LOG, 10)) {
for (SkyKey root : badRoots) {
ErrorInfo errorInfo = checkForCycles(root, visitor, keepGoing);
@@ -1774,13 +1809,13 @@
/**
* The algorithm for this cycle detector is as follows. We visit the graph depth-first, keeping
* track of the path we are currently on. We skip any DONE nodes (they are transitively
- * error-free). If we come to a node already on the path, we immediately construct a cycle. If
- * we are in the noKeepGoing case, we return ErrorInfo with that cycle to the caller. Otherwise,
- * we continue. Once all of a node's children are done, we construct an error value for it, based
- * on those children. Finally, when the original root's node is constructed, we return its
- * ErrorInfo.
+ * error-free). If we come to a node already on the path, we immediately construct a cycle. If we
+ * are in the noKeepGoing case, we return ErrorInfo with that cycle to the caller. Otherwise, we
+ * continue. Once all of a node's children are done, we construct an error value for it, based on
+ * those children. Finally, when the original root's node is constructed, we return its ErrorInfo.
*/
- private ErrorInfo checkForCycles(SkyKey root, ValueVisitor visitor, boolean keepGoing) {
+ private ErrorInfo checkForCycles(SkyKey root, ValueVisitor visitor, boolean keepGoing)
+ throws InterruptedException {
// The number of cycles found. Do not keep on searching for more cycles after this many were
// found.
int cyclesFound = 0;
@@ -1926,7 +1961,7 @@
// out.
// TODO(janakr): If graph implementations start using these hints for not-done nodes, we may
// have to change this.
- Map<SkyKey, NodeEntry> childrenNodes =
+ Map<SkyKey, ? extends NodeEntry> childrenNodes =
graph.getBatch(key, Reason.EXISTENCE_CHECKING, children);
Preconditions.checkState(childrenNodes.size() == Iterables.size(children), childrenNodes);
children = Maps.filterValues(childrenNodes, new Predicate<NodeEntry>() {
@@ -1962,7 +1997,8 @@
* @param children child nodes to query for errors.
* @return List of ErrorInfos from all children that had errors.
*/
- private List<ErrorInfo> getChildrenErrorsForCycle(SkyKey parent, Iterable<SkyKey> children) {
+ private List<ErrorInfo> getChildrenErrorsForCycle(SkyKey parent, Iterable<SkyKey> children)
+ throws InterruptedException {
List<ErrorInfo> allErrors = new ArrayList<>();
boolean foundCycle = false;
for (NodeEntry childNode : getAndCheckDoneBatchForCycle(parent, children).values()) {
@@ -1984,11 +2020,11 @@
* @return List of ErrorInfos from all children that had errors.
*/
private List<ErrorInfo> getChildrenErrorsForCycleChecking(
- Iterable<SkyKey> children, SkyKey unfinishedChild) {
+ Iterable<SkyKey> children, SkyKey unfinishedChild) throws InterruptedException {
List<ErrorInfo> allErrors = new ArrayList<>();
- Set<Entry<SkyKey, NodeEntry>> childEntries =
+ Set<? extends Entry<SkyKey, ? extends NodeEntry>> childEntries =
getBatchValues(null, Reason.CYCLE_CHECKING, children).entrySet();
- for (Entry<SkyKey, NodeEntry> childMapEntry : childEntries) {
+ for (Entry<SkyKey, ? extends NodeEntry> childMapEntry : childEntries) {
SkyKey childKey = childMapEntry.getKey();
NodeEntry childNodeEntry = childMapEntry.getValue();
ErrorInfo errorInfo = getErrorMaybe(childKey, childNodeEntry,
@@ -2001,7 +2037,8 @@
}
@Nullable
- private ErrorInfo getErrorMaybe(SkyKey key, NodeEntry childNodeEntry, boolean allowUnfinished) {
+ private static ErrorInfo getErrorMaybe(
+ SkyKey key, NodeEntry childNodeEntry, boolean allowUnfinished) throws InterruptedException {
Preconditions.checkNotNull(childNodeEntry, key);
if (!allowUnfinished) {
return checkDone(key, childNodeEntry).getErrorInfo();
@@ -2027,7 +2064,8 @@
NodeEntry entry,
@Nullable SkyKey cycleChild,
Iterable<SkyKey> toVisit,
- int cycleLength) {
+ int cycleLength)
+ throws InterruptedException {
GroupedList<SkyKey> directDeps = entry.getTemporaryDirectDeps();
Set<SkyKey> unvisitedDeps = Sets.newHashSetWithExpectedSize(directDeps.numElements());
Iterables.addAll(unvisitedDeps, Iterables.concat(directDeps));
@@ -2069,7 +2107,7 @@
}
private Set<SkyKey> removeIncompleteChildrenForCycle(
- SkyKey key, NodeEntry entry, Iterable<SkyKey> children) {
+ SkyKey key, NodeEntry entry, Iterable<SkyKey> children) throws InterruptedException {
Set<SkyKey> unfinishedDeps = new HashSet<>();
for (SkyKey child : children) {
if (removeIncompleteChildForCycle(key, child)) {
@@ -2086,14 +2124,14 @@
return entry;
}
- private NodeEntry getAndCheckDoneForCycle(SkyKey key) {
+ private NodeEntry getAndCheckDoneForCycle(SkyKey key) throws InterruptedException {
return checkDone(key, graph.get(null, Reason.CYCLE_CHECKING, key));
}
- private Map<SkyKey, NodeEntry> getAndCheckDoneBatchForCycle(
- SkyKey parent, Iterable<SkyKey> keys) {
- Map<SkyKey, NodeEntry> nodes = getBatchValues(parent, Reason.CYCLE_CHECKING, keys);
- for (Map.Entry<SkyKey, NodeEntry> nodeEntryMapEntry : nodes.entrySet()) {
+ private Map<SkyKey, ? extends NodeEntry> getAndCheckDoneBatchForCycle(
+ SkyKey parent, Iterable<SkyKey> keys) throws InterruptedException {
+ Map<SkyKey, ? extends NodeEntry> nodes = getBatchValues(parent, Reason.CYCLE_CHECKING, keys);
+ for (Entry<SkyKey, ? extends NodeEntry> nodeEntryMapEntry : nodes.entrySet()) {
checkDone(nodeEntryMapEntry.getKey(), nodeEntryMapEntry.getValue());
}
return nodes;
@@ -2104,7 +2142,8 @@
private static SkyValue maybeGetValueFromError(
SkyKey key,
@Nullable NodeEntry entry,
- @Nullable Map<SkyKey, ValueWithMetadata> bubbleErrorInfo) {
+ @Nullable Map<SkyKey, ValueWithMetadata> bubbleErrorInfo)
+ throws InterruptedException {
SkyValue value = bubbleErrorInfo == null ? null : bubbleErrorInfo.get(key);
if (value != null) {
Preconditions.checkNotNull(
@@ -2127,8 +2166,9 @@
Map<SkyKey, SkyValue> injectionMap,
Version version,
EvaluableGraph graph,
- DirtyKeyTracker dirtyKeyTracker) {
- Map<SkyKey, NodeEntry> prevNodeEntries =
+ DirtyKeyTracker dirtyKeyTracker)
+ throws InterruptedException {
+ Map<SkyKey, ? extends NodeEntry> prevNodeEntries =
graph.createIfAbsentBatch(null, Reason.OTHER, injectionMap.keySet());
for (Map.Entry<SkyKey, SkyValue> injectionEntry : injectionMap.entrySet()) {
SkyKey key = injectionEntry.getKey();