Allow SkyFunctions to return a sentinel value indicating that all of a node's in-progress data should be forgotten, and its evaluation should be restarted from scratch, as if it were freshly created/dirtied. To guard against this happening unexpectedly, any such events are passed to a GraphInconsistencyReceiver, which can verify that the SkyFunction is behaving properly.
This is the first change in a series to permit action rewinding when it is discovered that a previously generated input file is no longer available. When an action detects that one of its inputs is unusable, it can return this sentinel value, causing it to be re-evaluated from scratch. Follow-up changes will make the node corresponding to the input, and the node corresponding to the action that generated the input, dirty when this happens, causing the upstream action to be re-run, regenerating the desired input.
Currently works for builds that do not keep edges, although follow-ups may make this possible for all builds.
PiperOrigin-RevId: 194863097
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
index 4f192ac..1fa364f 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
@@ -151,6 +151,7 @@
import com.google.devtools.build.skyframe.ErrorInfo;
import com.google.devtools.build.skyframe.EvaluationProgressReceiver;
import com.google.devtools.build.skyframe.EvaluationResult;
+import com.google.devtools.build.skyframe.GraphInconsistencyReceiver;
import com.google.devtools.build.skyframe.ImmutableDiff;
import com.google.devtools.build.skyframe.Injectable;
import com.google.devtools.build.skyframe.MemoizingEvaluator;
@@ -654,6 +655,7 @@
skyFunctions,
evaluatorDiffer(),
progressReceiver,
+ GraphInconsistencyReceiver.THROWING,
emittedEventState,
tracksStateForIncrementality());
buildDriver = getBuildDriver();
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java b/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
index 0c63193..d718304 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
@@ -76,6 +76,7 @@
import com.google.devtools.build.skyframe.ErrorInfo;
import com.google.devtools.build.skyframe.EvaluationProgressReceiver;
import com.google.devtools.build.skyframe.EvaluationResult;
+import com.google.devtools.build.skyframe.GraphInconsistencyReceiver;
import com.google.devtools.build.skyframe.ImmutableDiff;
import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
import com.google.devtools.build.skyframe.Injectable;
@@ -333,6 +334,7 @@
makeFreshSkyFunctions(),
preinjectedDifferencer,
new EvaluationProgressReceiver.NullEvaluationProgressReceiver(),
+ GraphInconsistencyReceiver.THROWING,
new MemoizingEvaluator.EmittedEventState(),
/*keepEdges=*/ false));
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/AbstractExceptionalParallelEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/AbstractExceptionalParallelEvaluator.java
index 9526c85..e4a4193 100644
--- a/src/main/java/com/google/devtools/build/skyframe/AbstractExceptionalParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/AbstractExceptionalParallelEvaluator.java
@@ -86,7 +86,8 @@
ErrorInfoManager errorInfoManager,
boolean keepGoing,
int threadCount,
- DirtyTrackingProgressReceiver progressReceiver) {
+ DirtyTrackingProgressReceiver progressReceiver,
+ GraphInconsistencyReceiver graphInconsistencyReceiver) {
super(
graph,
graphVersion,
@@ -98,6 +99,7 @@
keepGoing,
threadCount,
progressReceiver,
+ graphInconsistencyReceiver,
new SimpleCycleDetector());
}
@@ -111,6 +113,7 @@
ErrorInfoManager errorInfoManager,
boolean keepGoing,
DirtyTrackingProgressReceiver progressReceiver,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
ForkJoinPool forkJoinPool,
CycleDetector cycleDetector) {
super(
@@ -123,6 +126,7 @@
errorInfoManager,
keepGoing,
progressReceiver,
+ graphInconsistencyReceiver,
forkJoinPool,
cycleDetector);
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java
index 6ed51ad..40ed157 100644
--- a/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java
@@ -70,6 +70,7 @@
boolean keepGoing,
int threadCount,
DirtyTrackingProgressReceiver progressReceiver,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
CycleDetector cycleDetector) {
this.graph = graph;
this.cycleDetector = cycleDetector;
@@ -85,6 +86,7 @@
storedEventFilter,
errorInfoManager,
Evaluate::new,
+ graphInconsistencyReceiver,
threadCount);
}
@@ -98,6 +100,7 @@
ErrorInfoManager errorInfoManager,
boolean keepGoing,
DirtyTrackingProgressReceiver progressReceiver,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
ForkJoinPool forkJoinPool,
CycleDetector cycleDetector) {
this.graph = graph;
@@ -114,6 +117,7 @@
storedEventFilter,
errorInfoManager,
Evaluate::new,
+ graphInconsistencyReceiver,
Preconditions.checkNotNull(forkJoinPool));
}
@@ -442,6 +446,11 @@
env.doneBuilding();
}
+ if (maybeEraseNodeToRestartFromScratch(skyKey, state, value)) {
+ evaluatorContext.getVisitor().enqueueEvaluation(skyKey);
+ return;
+ }
+
// Helper objects for all the newly requested deps that weren't known to the environment,
// and may contain duplicate elements.
GroupedListHelper<SkyKey> newDirectDeps = env.getNewlyRequestedDeps();
@@ -601,6 +610,25 @@
private static final int MAX_REVERSEDEP_DUMP_LENGTH = 1000;
}
+ private boolean maybeEraseNodeToRestartFromScratch(
+ SkyKey key, NodeEntry entry, SkyValue returnedValue) {
+ if (!SkyFunction.SENTINEL_FOR_RESTART_FROM_SCRATCH.equals(returnedValue)) {
+ return false;
+ }
+ evaluatorContext
+ .getGraphInconsistencyReceiver()
+ .noteInconsistencyAndMaybeThrow(
+ key, /*otherKey=*/ null, GraphInconsistencyReceiver.Inconsistency.RESET_REQUESTED);
+ entry.resetForRestartFromScratch();
+ // TODO(mschaller): rdeps of children have to be handled here. If the graph does not keep edges,
+ // nothing has to be done, since there are no reverse deps to keep consistent. If the graph
+ // keeps edges, it's a harder problem. The reverse deps could just be removed, but in the case
+ // that this node is dirty, the deps shouldn't be removed, they should just be transformed back
+ // to "known reverse deps" from "reverse deps declared during this evaluation" (the inverse of
+ // NodeEntry#checkIfDoneForDirtyReverseDep). Such a method doesn't currently exist, but could.
+ return true;
+ }
+
void propagateEvaluatorContextCrashIfAny() {
if (!evaluatorContext.getVisitor().getCrashes().isEmpty()) {
evaluatorContext
diff --git a/src/main/java/com/google/devtools/build/skyframe/DelegatingNodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/DelegatingNodeEntry.java
index 689b1cd..95124ec 100644
--- a/src/main/java/com/google/devtools/build/skyframe/DelegatingNodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/DelegatingNodeEntry.java
@@ -141,6 +141,11 @@
}
@Override
+ public void resetForRestartFromScratch() {
+ getDelegate().resetForRestartFromScratch();
+ }
+
+ @Override
public Set<SkyKey> addTemporaryDirectDeps(GroupedListHelper<SkyKey> helper) {
return getDelegate().addTemporaryDirectDeps(helper);
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/DirtyBuildingState.java b/src/main/java/com/google/devtools/build/skyframe/DirtyBuildingState.java
index d802c3d..59a2e13 100644
--- a/src/main/java/com/google/devtools/build/skyframe/DirtyBuildingState.java
+++ b/src/main/java/com/google/devtools/build/skyframe/DirtyBuildingState.java
@@ -193,6 +193,11 @@
return result.build();
}
+ final void resetForRestartFromScratch() {
+ Preconditions.checkState(dirtyState == DirtyState.REBUILDING, this);
+ dirtyDirectDepIndex = 0;
+ }
+
protected void markRebuilding() {
Preconditions.checkState(dirtyState == DirtyState.NEEDS_REBUILDING, this);
dirtyState = DirtyState.REBUILDING;
diff --git a/src/main/java/com/google/devtools/build/skyframe/GraphInconsistencyReceiver.java b/src/main/java/com/google/devtools/build/skyframe/GraphInconsistencyReceiver.java
new file mode 100644
index 0000000..bba9264
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/GraphInconsistencyReceiver.java
@@ -0,0 +1,41 @@
+// Copyright 2018 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 javax.annotation.Nullable;
+
+/**
+ * A receiver that can be informed of inconsistencies detected in Skyframe. Such inconsistencies are
+ * usually the result of external data loss (such as nodes in the graph, or the results of external
+ * computations stored in a remote execution service).
+ *
+ * <p>The receiver can tolerate such inconsistencies, or throw hard if they are unexpected.
+ */
+public interface GraphInconsistencyReceiver {
+ void noteInconsistencyAndMaybeThrow(
+ SkyKey key, @Nullable SkyKey otherKey, Inconsistency inconsistency);
+
+ /** The type of inconsistency detected. */
+ enum Inconsistency {
+ RESET_REQUESTED
+ }
+
+ /** A {@link GraphInconsistencyReceiver} that crashes on any inconsistency. */
+ GraphInconsistencyReceiver THROWING =
+ (key, otherKey, inconsistency) -> {
+ throw new IllegalStateException(
+ "Unexpected inconsistency: " + key + ", " + otherKey + ", " + inconsistency);
+ };
+}
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 5d29b82..923c712 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
@@ -61,6 +61,7 @@
private Map<SkyKey, SkyValue> valuesToInject = new HashMap<>();
private final InvalidationState deleterState = new DeletingInvalidationState();
private final Differencer differencer;
+ private final GraphInconsistencyReceiver graphInconsistencyReceiver;
// Keep edges in graph. Can be false to save memory, in which case incremental builds are
// not possible.
@@ -83,18 +84,26 @@
Map<SkyFunctionName, ? extends SkyFunction> skyFunctions,
Differencer differencer,
@Nullable EvaluationProgressReceiver progressReceiver) {
- this(skyFunctions, differencer, progressReceiver, new EmittedEventState(), true);
+ this(
+ skyFunctions,
+ differencer,
+ progressReceiver,
+ GraphInconsistencyReceiver.THROWING,
+ new EmittedEventState(),
+ true);
}
public InMemoryMemoizingEvaluator(
Map<SkyFunctionName, ? extends SkyFunction> skyFunctions,
Differencer differencer,
@Nullable EvaluationProgressReceiver progressReceiver,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
EmittedEventState emittedEventState,
boolean keepEdges) {
this.skyFunctions = ImmutableMap.copyOf(skyFunctions);
this.differencer = Preconditions.checkNotNull(differencer);
this.progressReceiver = new DirtyTrackingProgressReceiver(progressReceiver);
+ this.graphInconsistencyReceiver = Preconditions.checkNotNull(graphInconsistencyReceiver);
this.graph = new InMemoryGraphImpl(keepEdges);
this.emittedEventState = emittedEventState;
this.keepEdges = keepEdges;
@@ -179,7 +188,8 @@
ErrorInfoManager.UseChildErrorInfoIfNecessary.INSTANCE,
keepGoing,
numThreads,
- progressReceiver);
+ progressReceiver,
+ graphInconsistencyReceiver);
EvaluationResult<T> result = evaluator.eval(roots);
return EvaluationResult.<T>builder()
.mergeFrom(result)
diff --git a/src/main/java/com/google/devtools/build/skyframe/InMemoryNodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/InMemoryNodeEntry.java
index 7b48275..5a13c57 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryNodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryNodeEntry.java
@@ -582,6 +582,16 @@
}
@Override
+ public synchronized void resetForRestartFromScratch() {
+ Preconditions.checkState(!isDone(), "Reset entry can't be done: %s", this);
+ directDeps = null;
+ signaledDeps = 0;
+ if (dirtyBuildingState != null) {
+ dirtyBuildingState.resetForRestartFromScratch();
+ }
+ }
+
+ @Override
public synchronized Set<SkyKey> addTemporaryDirectDeps(GroupedListHelper<SkyKey> helper) {
Preconditions.checkState(!isDone(), "add temp shouldn't be done: %s %s", helper, this);
return getTemporaryDirectDeps().append(helper);
diff --git a/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
index dac0927..adff0d0 100644
--- a/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
@@ -165,6 +165,7 @@
ImmutableMap<SkyFunctionName, ? extends SkyFunction> skyFunctions,
Differencer differencer,
EvaluationProgressReceiver progressReceiver,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
EmittedEventState emittedEventState,
boolean keepEdges);
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
index 0279639..aee68b3 100644
--- a/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
@@ -355,6 +355,15 @@
void removeUnfinishedDeps(Set<SkyKey> unfinishedDeps);
/**
+ * Erases all stored work during this evaluation from this entry, namely all temporary direct
+ * deps. The entry will be as if it had never evaluated at this version. Called after the {@link
+ * SkyFunction} for this entry returns {@link SkyFunction#SENTINEL_FOR_RESTART_FROM_SCRATCH},
+ * indicating that something went wrong in external state and the evaluation has to be restarted.
+ */
+ @ThreadSafe
+ void resetForRestartFromScratch();
+
+ /**
* Adds the temporary direct deps given in {@code helper} and returns the set of unique deps
* added. It is the users responsibility to ensure that there are no elements in common between
* helper and the already existing temporary direct deps.
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 cdcec82..afd0279 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -42,7 +42,8 @@
ErrorInfoManager errorInfoManager,
boolean keepGoing,
int threadCount,
- DirtyTrackingProgressReceiver progressReceiver) {
+ DirtyTrackingProgressReceiver progressReceiver,
+ GraphInconsistencyReceiver graphInconsistencyReceiver) {
super(
graph,
graphVersion,
@@ -53,7 +54,8 @@
errorInfoManager,
keepGoing,
threadCount,
- progressReceiver);
+ progressReceiver,
+ graphInconsistencyReceiver);
}
public ParallelEvaluator(
@@ -66,6 +68,7 @@
ErrorInfoManager errorInfoManager,
boolean keepGoing,
DirtyTrackingProgressReceiver progressReceiver,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
ForkJoinPool forkJoinPool,
CycleDetector cycleDetector) {
super(
@@ -78,6 +81,7 @@
errorInfoManager,
keepGoing,
progressReceiver,
+ graphInconsistencyReceiver,
forkJoinPool,
cycleDetector);
}
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 6860433..14306d4 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluatorContext.java
@@ -52,6 +52,7 @@
private final DirtyTrackingProgressReceiver progressReceiver;
private final EventFilter storedEventFilter;
private final ErrorInfoManager errorInfoManager;
+ private final GraphInconsistencyReceiver graphInconsistencyReceiver;
/**
* The visitor managing the thread pool. Used to enqueue parents when an entry is finished, and,
@@ -72,29 +73,20 @@
EventFilter storedEventFilter,
ErrorInfoManager errorInfoManager,
final Function<SkyKey, Runnable> runnableMaker,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
final int threadCount) {
- this.graph = graph;
- this.graphVersion = graphVersion;
- this.skyFunctions = skyFunctions;
- this.reporter = reporter;
- this.replayingNestedSetEventVisitor =
- new NestedSetVisitor<>(new NestedSetEventReceiver(reporter), emittedEventState.eventState);
- this.replayingNestedSetPostableVisitor =
- new NestedSetVisitor<>(
- new NestedSetPostableReceiver(reporter), emittedEventState.postableState);
- this.keepGoing = keepGoing;
- this.progressReceiver = Preconditions.checkNotNull(progressReceiver);
- this.storedEventFilter = storedEventFilter;
- this.errorInfoManager = errorInfoManager;
- visitorSupplier =
- Suppliers.memoize(
- new Supplier<NodeEntryVisitor>() {
- @Override
- public NodeEntryVisitor get() {
- return new NodeEntryVisitor(
- threadCount, progressReceiver, runnableMaker);
- }
- });
+ this(
+ graph,
+ graphVersion,
+ skyFunctions,
+ reporter,
+ emittedEventState,
+ keepGoing,
+ progressReceiver,
+ storedEventFilter,
+ errorInfoManager,
+ graphInconsistencyReceiver,
+ () -> new NodeEntryVisitor(threadCount, progressReceiver, runnableMaker));
}
ParallelEvaluatorContext(
@@ -108,11 +100,39 @@
EventFilter storedEventFilter,
ErrorInfoManager errorInfoManager,
final Function<SkyKey, Runnable> runnableMaker,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
final ForkJoinPool forkJoinPool) {
+ this(
+ graph,
+ graphVersion,
+ skyFunctions,
+ reporter,
+ emittedEventState,
+ keepGoing,
+ progressReceiver,
+ storedEventFilter,
+ errorInfoManager,
+ graphInconsistencyReceiver,
+ () -> new NodeEntryVisitor(forkJoinPool, progressReceiver, runnableMaker));
+ }
+
+ private ParallelEvaluatorContext(
+ QueryableGraph graph,
+ Version graphVersion,
+ ImmutableMap<SkyFunctionName, ? extends SkyFunction> skyFunctions,
+ ExtendedEventHandler reporter,
+ EmittedEventState emittedEventState,
+ boolean keepGoing,
+ final DirtyTrackingProgressReceiver progressReceiver,
+ EventFilter storedEventFilter,
+ ErrorInfoManager errorInfoManager,
+ GraphInconsistencyReceiver graphInconsistencyReceiver,
+ Supplier<NodeEntryVisitor> visitorSupplier) {
this.graph = graph;
this.graphVersion = graphVersion;
this.skyFunctions = skyFunctions;
this.reporter = reporter;
+ this.graphInconsistencyReceiver = graphInconsistencyReceiver;
this.replayingNestedSetEventVisitor =
new NestedSetVisitor<>(new NestedSetEventReceiver(reporter), emittedEventState.eventState);
this.replayingNestedSetPostableVisitor =
@@ -122,15 +142,7 @@
this.progressReceiver = Preconditions.checkNotNull(progressReceiver);
this.storedEventFilter = storedEventFilter;
this.errorInfoManager = errorInfoManager;
- visitorSupplier =
- Suppliers.memoize(
- new Supplier<NodeEntryVisitor>() {
- @Override
- public NodeEntryVisitor get() {
- return new NodeEntryVisitor(
- forkJoinPool, progressReceiver, runnableMaker);
- }
- });
+ this.visitorSupplier = Suppliers.memoize(visitorSupplier);
}
Map<SkyKey, ? extends NodeEntry> getBatchValues(
@@ -196,6 +208,10 @@
return progressReceiver;
}
+ GraphInconsistencyReceiver getGraphInconsistencyReceiver() {
+ return graphInconsistencyReceiver;
+ }
+
NestedSetVisitor<TaggedEvents> getReplayingNestedSetEventVisitor() {
return replayingNestedSetEventVisitor;
}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ReverseDepsUtility.java b/src/main/java/com/google/devtools/build/skyframe/ReverseDepsUtility.java
index e1d16bb..5eea55a 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ReverseDepsUtility.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ReverseDepsUtility.java
@@ -13,6 +13,7 @@
// limitations under the License.
package com.google.devtools.build.skyframe;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.MoreObjects;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
@@ -53,7 +54,7 @@
abstract class ReverseDepsUtility {
private ReverseDepsUtility() {}
- static final int MAYBE_CHECK_THRESHOLD = 10;
+ @VisibleForTesting static final int MAYBE_CHECK_THRESHOLD = 10;
/**
* We can store one type of operation bare in order to save memory. For done nodes, most
diff --git a/src/main/java/com/google/devtools/build/skyframe/SkyFunction.java b/src/main/java/com/google/devtools/build/skyframe/SkyFunction.java
index 9975996..1ae2916 100644
--- a/src/main/java/com/google/devtools/build/skyframe/SkyFunction.java
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyFunction.java
@@ -36,15 +36,18 @@
* When a value is requested, this method is called with the name of the value and a
* dependency-tracking environment.
*
- * <p>This method should return a non-{@code null} value, or {@code null} if any dependencies
- * were missing ({@link Environment#valuesMissing} was true before returning). In that case the
- * missing dependencies will be computed and the {@code compute} method called again.
+ * <p>This method should return a non-{@code null} value, or {@code null} if any dependencies were
+ * missing ({@link Environment#valuesMissing} was true before returning). In that case the missing
+ * dependencies will be computed and the {@code compute} method called again.
*
- * <p>This method should throw if it fails, or if one of its dependencies fails with an
- * exception and this method cannot recover. If one of its dependencies fails and this method can
- * enrich the exception with additional context, then this method should catch that exception and
- * throw another containing that additional context. If it has no such additional context, then
- * it should allow its dependency's exception to be thrown through it.
+ * <p>This method should throw if it fails, or if one of its dependencies fails with an exception
+ * and this method cannot recover. If one of its dependencies fails and this method can enrich the
+ * exception with additional context, then this method should catch that exception and throw
+ * another containing that additional context. If it has no such additional context, then it
+ * should allow its dependency's exception to be thrown through it.
+ *
+ * <p>This method may return {@link #SENTINEL_FOR_RESTART_FROM_SCRATCH} in rare circumstances. See
+ * its docs. Do not return this value unless you know exactly what you are doing.
*
* @throws SkyFunctionException on failure
* @throws InterruptedException if interrupted
@@ -66,6 +69,17 @@
String extractTag(SkyKey skyKey);
/**
+ * Sentinel value for {@link #compute} to return, indicating that something went wrong in external
+ * state and the evaluation has to be restarted from scratch, ignoring any deps that the {@link
+ * #compute} function may have declared during evaluation at this version (including deps declared
+ * during previous calls that returned null). Common causes for returning this would be data loss,
+ * if a dep's data is not actually available externally. In this case, the {@link SkyFunction}
+ * will likely dirty the unusable dep to force its re-evalution when re-requested by the restarted
+ * entry's computation.
+ */
+ SkyValue SENTINEL_FOR_RESTART_FROM_SCRATCH = new SkyValue() {};
+
+ /**
* The services provided to the {@link SkyFunction#compute} implementation by the Skyframe
* evaluation framework.
*/