Distinguish rewinding from force-rebuilding.

The term "force rebuild" is currently used for both 1) rewinding and 2) retrying a transient error on an incremental build. Make it clear that they are two different operations triggered by two different scenarios.

Explicitly prohibit `markDirty(DirtyType.REWIND)` on incremental nodes (pending b/228090759) and `forceRebuild()` on non-incremental nodes.

PiperOrigin-RevId: 559485758
Change-Id: Iec1ea5585d6ae6d2686b146ec8ca2b4015a11983
diff --git a/src/main/java/com/google/devtools/build/skyframe/AbstractInMemoryNodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/AbstractInMemoryNodeEntry.java
index c7dcbad..87c1381 100644
--- a/src/main/java/com/google/devtools/build/skyframe/AbstractInMemoryNodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/AbstractInMemoryNodeEntry.java
@@ -145,13 +145,6 @@
   }
 
   @Override
-  public final synchronized void forceRebuild() {
-    checkNotNull(dirtyBuildingState, this);
-    checkState(isEvaluating(), this);
-    dirtyBuildingState.forceRebuild(getNumTemporaryDirectDeps());
-  }
-
-  @Override
   public final synchronized DirtyState getDirtyState() {
     checkNotNull(dirtyBuildingState, this);
     return dirtyBuildingState.getDirtyState();
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 f2065d4..504ac19 100644
--- a/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/AbstractParallelEvaluator.java
@@ -408,9 +408,6 @@
         case NEEDS_REBUILDING:
           nodeEntry.markRebuilding();
           return DirtyOutcome.NEEDS_EVALUATION;
-        case NEEDS_FORCED_REBUILDING:
-          nodeEntry.forceRebuild();
-          return DirtyOutcome.NEEDS_EVALUATION;
         case REBUILDING:
           return DirtyOutcome.NEEDS_EVALUATION;
         default:
@@ -943,9 +940,7 @@
                 childToRestart,
                 key);
 
-        // Nodes are marked "force-rebuild" to ensure that they run, and to allow them to evaluate
-        // to a different value than before, even if their versions remain the same.
-        if (childEntry.markDirty(DirtyType.FORCE_REBUILD) != null) {
+        if (childEntry.markDirty(DirtyType.REWIND) != null) {
           evaluatorContext
               .getProgressReceiver()
               .invalidated(childToRestart, EvaluationProgressReceiver.InvalidationState.DIRTY);
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 a3f3142..b9c0060 100644
--- a/src/main/java/com/google/devtools/build/skyframe/DirtyBuildingState.java
+++ b/src/main/java/com/google/devtools/build/skyframe/DirtyBuildingState.java
@@ -132,13 +132,22 @@
    * Group of children to be checked next in the process of determining if this entry needs to be
    * re-evaluated. Used by {@link DirtyBuildingState#getNextDirtyDirectDeps} and {@link #signalDep}.
    */
-  protected int dirtyDirectDepIndex;
+  protected int dirtyDirectDepIndex = 0;
 
   protected DirtyBuildingState(DirtyType dirtyType) {
-    dirtyState = dirtyType.getInitialDirtyState();
-    // We need to iterate through the deps to see if they have changed, or to remove them if one
-    // has. Initialize the iterating index.
-    dirtyDirectDepIndex = 0;
+    dirtyState = initialDirtyState(dirtyType);
+  }
+
+  private static DirtyState initialDirtyState(DirtyType dirtyType) {
+    switch (dirtyType) {
+      case DIRTY:
+        return DirtyState.CHECK_DEPENDENCIES;
+      case CHANGE:
+        return DirtyState.NEEDS_REBUILDING;
+      case REWIND:
+        throw new IllegalArgumentException(dirtyType.toString());
+    }
+    throw new AssertionError(dirtyType);
   }
 
   /** Returns true if this state has information about a previously built version. */
@@ -150,27 +159,11 @@
     dirtyState = DirtyState.NEEDS_REBUILDING;
   }
 
-  final void markForceRebuild() {
-    if (dirtyState == DirtyState.CHECK_DEPENDENCIES) {
-      dirtyState = DirtyState.NEEDS_REBUILDING;
-    }
-  }
-
-  // TODO(b/228090759): Tighten up state checks for the force rebuild lifecycle.
   final void forceRebuild(int numTemporaryDirectDeps) {
+    checkState(dirtyState == DirtyState.CHECK_DEPENDENCIES, this);
     checkState(numTemporaryDirectDeps + externalDeps == signaledDeps, this);
-    switch (dirtyState) {
-      case CHECK_DEPENDENCIES:
-        checkState(getNumOfGroupsInLastBuildDirectDeps() == dirtyDirectDepIndex, this);
-        dirtyState = DirtyState.REBUILDING;
-        break;
-      case NEEDS_REBUILDING: // Valid for NonIncrementalInMemoryNodeEntry.
-      case NEEDS_FORCED_REBUILDING: // Valid for IncrementalInMemoryNodeEntry.
-        dirtyState = DirtyState.REBUILDING;
-        break;
-      default:
-        throw new IllegalStateException("Unexpected dirty state " + dirtyState + ": " + this);
-    }
+    checkState(getNumOfGroupsInLastBuildDirectDeps() == dirtyDirectDepIndex, this);
+    dirtyState = DirtyState.REBUILDING;
   }
 
   final boolean isEvaluating() {
@@ -178,9 +171,7 @@
   }
 
   final boolean isChanged() {
-    return dirtyState == DirtyState.NEEDS_REBUILDING
-        || dirtyState == DirtyState.NEEDS_FORCED_REBUILDING
-        || dirtyState == DirtyState.REBUILDING;
+    return dirtyState == DirtyState.NEEDS_REBUILDING || dirtyState == DirtyState.REBUILDING;
   }
 
   private void checkFinishedBuildingWhenAboutToSetValue() {
diff --git a/src/main/java/com/google/devtools/build/skyframe/IncrementalInMemoryNodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/IncrementalInMemoryNodeEntry.java
index ce4e784..4236dce 100644
--- a/src/main/java/com/google/devtools/build/skyframe/IncrementalInMemoryNodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/IncrementalInMemoryNodeEntry.java
@@ -13,6 +13,7 @@
 // limitations under the License.
 package com.google.devtools.build.skyframe;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 
@@ -306,40 +307,41 @@
   @ForOverride
   protected DirtyBuildingState createDirtyBuildingStateForDoneNode(
       DirtyType dirtyType, GroupedDeps directDeps, SkyValue value) {
-    return new IncrementalDirtyBuildingState(dirtyType, getKey(), directDeps, value);
+    return new IncrementalDirtyBuildingState(dirtyType, directDeps, value);
   }
 
   @Nullable
   @Override
   public synchronized MarkedDirtyResult markDirty(DirtyType dirtyType) {
+    checkNotNull(dirtyType, this);
+    // TODO(b/228090759): Support rewinding with incrementality.
+    checkArgument(dirtyType != DirtyType.REWIND, "Rewinding not supported with incrementality");
+
     if (isDone()) {
       GroupedDeps directDeps = GroupedDeps.decompress(getCompressedDirectDepsForDoneEntry());
+      checkState(
+          dirtyType == DirtyType.CHANGE || !directDeps.isEmpty(),
+          "%s is being marked dirty but has no children that could have dirtied it",
+          getKey());
       dirtyBuildingState = createDirtyBuildingStateForDoneNode(dirtyType, directDeps, value);
       value = null;
       this.directDeps = null;
       return MarkedDirtyResult.withReverseDeps(
           ReverseDepsUtility.getReverseDeps(this, /* checkConsistency= */ true));
     }
-    if (dirtyType.equals(DirtyType.FORCE_REBUILD)) {
-      if (dirtyBuildingState != null) {
-        dirtyBuildingState.markForceRebuild();
-      }
-      return null;
-    }
+
     // The caller may be simultaneously trying to mark this node dirty and changed, and the dirty
-    // thread may have lost the race, but it is the caller's responsibility not to try to mark
-    // this node changed twice. The end result of racing markers must be a changed node, since one
-    // of the markers is trying to mark the node changed.
-    checkState(
-        dirtyType.equals(DirtyType.CHANGE) != isChanged(),
-        "Cannot mark node dirty twice or changed twice: %s",
-        this);
+    // thread may have lost the race, but it is the caller's responsibility not to try to mark this
+    // node changed twice. The end result of racing markers must be a changed node, since one of the
+    // markers is trying to mark the node changed.
     checkState(value == null, "Value should have been reset already %s", this);
-    if (dirtyType.equals(DirtyType.CHANGE)) {
-      checkNotNull(dirtyBuildingState, this);
+    if (dirtyType == DirtyType.CHANGE) {
       // If the changed marker lost the race, we just need to mark changed in this method -- all
       // other work was done by the dirty marker.
-      dirtyBuildingState.markChanged();
+      checkState(!isChanged(), "Cannot mark node changed twice: %s", this);
+      checkNotNull(dirtyBuildingState, this).markChanged();
+    } else {
+      checkState(isChanged(), "Cannot mark node dirty twice: %s", this);
     }
     return null;
   }
@@ -375,6 +377,11 @@
   }
 
   @Override
+  public final synchronized void forceRebuild() {
+    checkNotNull(dirtyBuildingState, this).forceRebuild(getNumTemporaryDirectDeps());
+  }
+
+  @Override
   final synchronized int getNumTemporaryDirectDeps() {
     return directDeps == null ? 0 : getTemporaryDirectDeps().numElements();
   }
@@ -402,14 +409,10 @@
     private final SkyValue lastBuildValue;
 
     private IncrementalDirtyBuildingState(
-        DirtyType dirtyType, SkyKey key, GroupedDeps lastBuildDirectDeps, SkyValue lastBuildValue) {
+        DirtyType dirtyType, GroupedDeps lastBuildDirectDeps, SkyValue lastBuildValue) {
       super(dirtyType);
       this.lastBuildDirectDeps = lastBuildDirectDeps;
       this.lastBuildValue = lastBuildValue;
-      checkState(
-          !dirtyType.equals(DirtyType.DIRTY) || getNumOfGroupsInLastBuildDirectDeps() > 0,
-          "%s is being marked dirty but has no children that could have dirtied it",
-          key);
     }
 
     @Override
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 b17fbfd..e4c4699 100644
--- a/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
@@ -71,46 +71,48 @@
      * did.
      */
     NEEDS_REBUILDING,
-    /**
-     * A forced rebuilding is required, likely because of a recoverable inconsistency in the current
-     * build.
-     */
-    NEEDS_FORCED_REBUILDING,
     /** A rebuilding is in progress. */
     REBUILDING,
   }
 
   /** Ways that a node may be dirtied. */
   enum DirtyType {
-    /**
-     * A node P dirtied with DIRTY is re-evaluated during the evaluation phase if it's requested and
-     * directly depends on some node C whose value changed since the last evaluation of P. If it's
-     * requested and there is no such node C, P is marked clean.
-     */
-    DIRTY(DirtyState.CHECK_DEPENDENCIES),
 
     /**
-     * A node dirtied with CHANGE is re-evaluated during the evaluation phase if it's requested
-     * (regardless of the state of its dependencies). Such a node is expected to evaluate to the
-     * same value if evaluated at the same graph version.
+     * Indicates that the node is being marked dirty because it has a dependency that was marked
+     * dirty.
+     *
+     * <p>A node P dirtied with {@code DIRTY} is re-evaluated during the evaluation phase if it is
+     * requested and directly depends on some node C whose value changed since the last evaluation
+     * of P. If it is requested and there is no such node C, P is {@linkplain #markClean marked
+     * clean}.
      */
-    CHANGE(DirtyState.NEEDS_REBUILDING),
+    DIRTY,
 
     /**
-     * A node dirtied with FORCE_REBUILD behaves like a {@link #CHANGE}d node, except that it may
-     * evaluate to a different value even if evaluated at the same graph version.
+     * Indicates that the node is being marked dirty because its value from a previous evaluation is
+     * no longer valid, even if none of its dependencies change.
+     *
+     * <p>This is typically used to indicate that a value produced by a {@link
+     * FunctionHermeticity#NONHERMETIC} function is no longer valid because some state outside of
+     * Skyframe has changed (e.g. a change to the filesystem).
+     *
+     * <p>A node dirtied with {@code CHANGE} is re-evaluated during the evaluation phase if it is
+     * requested, regardless of the state of its dependencies. If it re-evaluates to the same value,
+     * dirty parents are not necessarily re-evaluated.
      */
-    FORCE_REBUILD(DirtyState.NEEDS_FORCED_REBUILDING);
+    CHANGE,
 
-    private final DirtyState initialDirtyState;
-
-    DirtyType(DirtyState initialDirtyState) {
-      this.initialDirtyState = initialDirtyState;
-    }
-
-    DirtyState getInitialDirtyState() {
-      return initialDirtyState;
-    }
+    /**
+     * Similar to {@link #CHANGE} except may be used intra-evaluation to indicate that the node's
+     * value (which may be from either a previous evaluation or the current evaluation) is no longer
+     * valid.
+     *
+     * <p>A node dirtied with {@code REWIND} is re-evaluated during the evaluation phase if it is
+     * requested, regardless of the state of its dependencies. Even if it re-evaluates to the same
+     * value, dirty parents are re-evaluated.
+     */
+    REWIND
   }
 
   /** Returns whether the entry has been built and is finished evaluating. */
@@ -139,8 +141,8 @@
    * markDirty(DirtyType.CHANGE)} may only be called on a node P for which {@code P.isDone() ||
    * !P.isChanged()}. Otherwise, this will throw {@link IllegalStateException}.
    *
-   * <p>{@code markDirty(DirtyType.FORCE_REBUILD)} may be called multiple times; only the first has
-   * any effect.
+   * <p>{@code markDirty(DirtyType.REWIND)} may be called at any time (even multiple times
+   * concurrently), although it only has an effect if the node {@link #isDone}.
    *
    * @return if the node was done, a {@link MarkedDirtyResult} which may include the node's reverse
    *     deps; otherwise {@code null}
@@ -395,12 +397,11 @@
   }
 
   /**
-   * Forces this node to be re-evaluated, even if none of its dependencies are known to have
-   * changed.
+   * Called on a dirty node during {@linkplain DirtyState#CHECK_DEPENDENCIES dependency checking} to
+   * force the node to be re-evaluated, even if none of its dependencies are known to have changed.
    *
-   * <p>Used when an external caller has reason to believe that re-evaluating may yield a new
-   * result. This method should not be used if one of the normal deps of this node has changed, the
-   * usual change-pruning process should work in that case.
+   * <p>Used when a caller has reason to believe that re-evaluating may yield a new result, such as
+   * when the prior evaluation encountered a transient error.
    */
   @ThreadSafe
   void forceRebuild();
diff --git a/src/main/java/com/google/devtools/build/skyframe/NonIncrementalInMemoryNodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/NonIncrementalInMemoryNodeEntry.java
index 276d9a0..e053a92 100644
--- a/src/main/java/com/google/devtools/build/skyframe/NonIncrementalInMemoryNodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/NonIncrementalInMemoryNodeEntry.java
@@ -86,7 +86,7 @@
         return DependencyState.DONE;
       }
       if (dirtyBuildingState == null) {
-        dirtyBuildingState = newBuildingState();
+        dirtyBuildingState = new NonIncrementalBuildingState();
       }
       if (reverseDep != null) {
         dirtyBuildingState.addReverseDep(reverseDep);
@@ -138,14 +138,11 @@
   @Nullable
   @Override
   public final synchronized MarkedDirtyResult markDirty(DirtyType dirtyType) {
-    checkState(dirtyType == DirtyType.FORCE_REBUILD, "Unexpected dirty type: %s", dirtyType);
+    checkArgument(dirtyType == DirtyType.REWIND, "Unexpected dirty type: %s", dirtyType);
     if (!isDone()) {
-      if (dirtyBuildingState != null) {
-        dirtyBuildingState.markForceRebuild();
-      }
-      return null;
+      return null; // Tolerate concurrent requests to rewind.
     }
-    dirtyBuildingState = newBuildingState();
+    dirtyBuildingState = new NonIncrementalBuildingState();
     value = null;
     return MarkedDirtyResult.withReverseDeps(ImmutableList.of());
   }
@@ -216,12 +213,13 @@
     throw unsupported();
   }
 
-  private UnsupportedOperationException unsupported() {
-    return new UnsupportedOperationException("Not keeping edges: " + this);
+  @Override
+  public void forceRebuild() {
+    throw unsupported();
   }
 
-  private NonIncrementalBuildingState newBuildingState() {
-    return new NonIncrementalBuildingState();
+  private UnsupportedOperationException unsupported() {
+    return new UnsupportedOperationException("Not keeping edges: " + this);
   }
 
   /**
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 d880509..707b4da 100644
--- a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -399,9 +399,6 @@
           case NEEDS_REBUILDING:
             maybeMarkRebuilding(parentEntry);
             break;
-          case NEEDS_FORCED_REBUILDING:
-            parentEntry.forceRebuild();
-            break;
           case REBUILDING:
             break;
           default:
diff --git a/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java b/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
index 7d69c68..e42b03e 100644
--- a/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
+++ b/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
@@ -199,10 +199,7 @@
     /** The node is being looked up merely to see if it is done or not. */
     DONE_CHECKING,
 
-    /**
-     * The node is being looked up so that it can be {@linkplain DirtyType#FORCE_REBUILD force
-     * rebuilt} by rewinding.
-     */
+    /** The node is being looked up so that it can be {@linkplain DirtyType#REWIND rewound}. */
     REWINDING,
 
     /**
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/rewinding/RewindingTestsHelper.java b/src/test/java/com/google/devtools/build/lib/skyframe/rewinding/RewindingTestsHelper.java
index 41b49ed..4fb0f35 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/rewinding/RewindingTestsHelper.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/rewinding/RewindingTestsHelper.java
@@ -236,7 +236,7 @@
         NotifyingHelper.MarkDirtyAfterContext markDirtyAfterContext =
             (NotifyingHelper.MarkDirtyAfterContext) context;
         checkState(
-            markDirtyAfterContext.dirtyType().equals(DirtyType.FORCE_REBUILD),
+            markDirtyAfterContext.dirtyType().equals(DirtyType.REWIND),
             "Unexpected DirtyType %s for key %s",
             context,
             key);
@@ -374,7 +374,7 @@
                   .getSkyframeExecutor()
                   .getEvaluator()
                   .getExistingEntryAtCurrentlyEvaluatingVersion(key)
-                  .markDirty(DirtyType.FORCE_REBUILD);
+                  .markDirty(DirtyType.REWIND);
             } catch (InterruptedException e) {
               throw new IllegalStateException(e);
             }
@@ -1234,7 +1234,7 @@
             NotifyingHelper.MarkDirtyAfterContext markDirtyAfterContext =
                 (NotifyingHelper.MarkDirtyAfterContext) context;
             checkState(
-                markDirtyAfterContext.dirtyType().equals(DirtyType.FORCE_REBUILD),
+                markDirtyAfterContext.dirtyType().equals(DirtyType.REWIND),
                 "Unexpected DirtyType %s for key %s",
                 context,
                 key);
diff --git a/src/test/java/com/google/devtools/build/skyframe/InMemoryNodeEntryTest.java b/src/test/java/com/google/devtools/build/skyframe/InMemoryNodeEntryTest.java
index 8d4faac..e14b67a 100644
--- a/src/test/java/com/google/devtools/build/skyframe/InMemoryNodeEntryTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/InMemoryNodeEntryTest.java
@@ -25,8 +25,6 @@
 import com.google.devtools.build.lib.collect.nestedset.Order;
 import com.google.devtools.build.lib.events.Reportable;
 import com.google.devtools.build.skyframe.NodeEntry.DependencyState;
-import com.google.devtools.build.skyframe.NodeEntry.DirtyState;
-import com.google.devtools.build.skyframe.NodeEntry.DirtyType;
 import com.google.devtools.build.skyframe.SkyFunctionException.ReifiedSkyFunctionException;
 import com.google.devtools.build.skyframe.SkyFunctionException.Transience;
 import com.google.errorprone.annotations.CanIgnoreReturnValue;
@@ -237,66 +235,6 @@
   }
 
   @Test
-  public void forceRebuildLifecycle() throws InterruptedException {
-    InMemoryNodeEntry entry = createEntry();
-    entry.addReverseDepAndCheckIfDone(null); // Start evaluation.
-    entry.markRebuilding();
-    SkyKey dep = key("dep");
-    entry.addSingletonTemporaryDirectDep(dep);
-    entry.signalDep(initialVersion, dep);
-    setValue(entry, new SkyValue() {}, /* errorInfo= */ null, initialVersion);
-    assertThat(entry.isDirty()).isFalse();
-    assertThat(entry.isDone()).isTrue();
-
-    entry.markDirty(DirtyType.FORCE_REBUILD);
-    assertThat(entry.isDirty()).isTrue();
-    assertThat(entry.isChanged()).isTrue();
-    assertThat(entry.isDone()).isFalse();
-    assertThat(entry.getTemporaryDirectDeps() instanceof GroupedDeps.WithHashSet)
-        .isEqualTo(isPartialReevaluation);
-
-    assertThatNodeEntry(entry)
-        .addReverseDepAndCheckIfDone(null)
-        .isEqualTo(DependencyState.NEEDS_SCHEDULING);
-    assertThat(entry.isReadyToEvaluate()).isTrue();
-    assertThat(entry.hasUnsignaledDeps()).isFalse();
-
-    SkyKey parent = key("parent");
-    entry.addReverseDepAndCheckIfDone(parent);
-    // A non-incremental node that is force rebuilt acts as if it was never built.
-    // TODO(b/228090759): Better distinguish the force rebuild lifecycles.
-    assertThat(entry.getDirtyState())
-        .isEqualTo(
-            entry.keepsEdges() ? DirtyState.NEEDS_FORCED_REBUILDING : DirtyState.NEEDS_REBUILDING);
-    assertThat(entry.isReadyToEvaluate()).isTrue();
-    assertThat(entry.hasUnsignaledDeps()).isFalse();
-    assertThat(entry.getTemporaryDirectDeps()).isEmpty();
-
-    // A force-rebuilt node tolerates evaluating to different values within the same version.
-    entry.forceRebuild();
-    assertThat(entry.getDirtyState()).isEqualTo(DirtyState.REBUILDING);
-    assertThat(setValue(entry, new SkyValue() {}, /* errorInfo= */ null, initialVersion))
-        .containsExactly(parent);
-
-    assertThat(entry.getVersion()).isEqualTo(initialVersion);
-  }
-
-  @Test
-  public void allowTwiceMarkedForceRebuild() throws InterruptedException {
-    InMemoryNodeEntry entry = createEntry();
-    entry.addReverseDepAndCheckIfDone(null); // Start evaluation.
-    entry.markRebuilding();
-    setValue(entry, new SkyValue() {}, /* errorInfo= */ null, initialVersion);
-    assertThat(entry.isDirty()).isFalse();
-    assertThat(entry.isDone()).isTrue();
-    entry.markDirty(DirtyType.FORCE_REBUILD);
-    entry.markDirty(DirtyType.FORCE_REBUILD);
-    assertThat(entry.isDirty()).isTrue();
-    assertThat(entry.isChanged()).isTrue();
-    assertThat(entry.isDone()).isFalse();
-  }
-
-  @Test
   public void crashOnAddReverseDepTwice() {
     InMemoryNodeEntry entry = createEntry();
     SkyKey parent = key("parent");
diff --git a/src/test/java/com/google/devtools/build/skyframe/IncrementalInMemoryNodeEntryTest.java b/src/test/java/com/google/devtools/build/skyframe/IncrementalInMemoryNodeEntryTest.java
index 8833036..a849992 100644
--- a/src/test/java/com/google/devtools/build/skyframe/IncrementalInMemoryNodeEntryTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/IncrementalInMemoryNodeEntryTest.java
@@ -21,6 +21,7 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.devtools.build.skyframe.NodeEntry.DependencyState;
+import com.google.devtools.build.skyframe.NodeEntry.DirtyState;
 import com.google.devtools.build.skyframe.NodeEntry.DirtyType;
 import com.google.devtools.build.skyframe.SkyFunctionException.ReifiedSkyFunctionException;
 import com.google.devtools.build.skyframe.SkyFunctionException.Transience;
@@ -199,6 +200,62 @@
   }
 
   @Test
+  public void forceRebuildAfterTransientError() throws Exception {
+    InMemoryNodeEntry entry = createEntry();
+    entry.addReverseDepAndCheckIfDone(null); // Start evaluation.
+    entry.markRebuilding();
+    SkyKey dep = key("dep");
+    entry.addSingletonTemporaryDirectDep(dep);
+    entry.signalDep(initialVersion, dep);
+    entry.addSingletonTemporaryDirectDep(ErrorTransienceValue.KEY);
+    entry.signalDep(initialVersion, ErrorTransienceValue.KEY);
+
+    setValue(
+        entry,
+        /* value= */ null,
+        ErrorInfo.fromException(
+            new ReifiedSkyFunctionException(
+                new GenericFunctionException(
+                    new SomeErrorException("transient error"), Transience.TRANSIENT)),
+            /* isTransitivelyTransient= */ true),
+        initialVersion);
+    assertThat(entry.isDirty()).isFalse();
+    assertThat(entry.isDone()).isTrue();
+
+    entry.markDirty(DirtyType.DIRTY);
+    assertThat(entry.isDirty()).isTrue();
+    assertThat(entry.isChanged()).isFalse();
+    assertThat(entry.isDone()).isFalse();
+    assertThat(entry.getTemporaryDirectDeps() instanceof GroupedDeps.WithHashSet)
+        .isEqualTo(isPartialReevaluation);
+
+    SkyKey parent = key("parent");
+    assertThatNodeEntry(entry)
+        .addReverseDepAndCheckIfDone(parent)
+        .isEqualTo(DependencyState.NEEDS_SCHEDULING);
+    assertThat(entry.isReadyToEvaluate()).isTrue();
+    assertThat(entry.hasUnsignaledDeps()).isFalse();
+    assertThat(entry.getDirtyState()).isEqualTo(DirtyState.CHECK_DEPENDENCIES);
+    assertThat(entry.isReadyToEvaluate()).isTrue();
+    assertThat(entry.hasUnsignaledDeps()).isFalse();
+    assertThat(entry.getTemporaryDirectDeps()).isEmpty();
+
+    assertThat(entry.getNextDirtyDirectDeps()).containsExactly(dep);
+    entry.addSingletonTemporaryDirectDep(dep);
+    assertThat(entry.signalDep(initialVersion, dep)).isTrue();
+    assertThat(entry.getDirtyState()).isEqualTo(DirtyState.CHECK_DEPENDENCIES);
+
+    assertThat(entry.getNextDirtyDirectDeps()).containsExactly(ErrorTransienceValue.KEY);
+    entry.forceRebuild();
+    assertThat(entry.getDirtyState()).isEqualTo(DirtyState.REBUILDING);
+
+    assertThat(setValue(entry, new SkyValue() {}, /* errorInfo= */ null, incrementalVersion))
+        .containsExactly(parent);
+    assertThat(entry.getVersion()).isEqualTo(incrementalVersion);
+    assertThat(entry.getDirectDeps()).containsExactly(dep); // No more dep on error transience.
+  }
+
+  @Test
   public void crashOnAddReverseDepTwiceAfterDone() throws InterruptedException {
     InMemoryNodeEntry entry = createEntry();
     entry.addReverseDepAndCheckIfDone(null); // Start evaluation.
diff --git a/src/test/java/com/google/devtools/build/skyframe/NonIncrementalInMemoryNodeEntryTest.java b/src/test/java/com/google/devtools/build/skyframe/NonIncrementalInMemoryNodeEntryTest.java
index 46b6830..1ddc356 100644
--- a/src/test/java/com/google/devtools/build/skyframe/NonIncrementalInMemoryNodeEntryTest.java
+++ b/src/test/java/com/google/devtools/build/skyframe/NonIncrementalInMemoryNodeEntryTest.java
@@ -13,8 +13,15 @@
 // limitations under the License.
 package com.google.devtools.build.skyframe;
 
+import static com.google.common.truth.Truth.assertThat;
+import static com.google.devtools.build.skyframe.NodeEntrySubjectFactory.assertThatNodeEntry;
+
+import com.google.devtools.build.skyframe.NodeEntry.DependencyState;
+import com.google.devtools.build.skyframe.NodeEntry.DirtyState;
+import com.google.devtools.build.skyframe.NodeEntry.DirtyType;
 import com.google.devtools.build.skyframe.Version.ConstantVersion;
 import com.google.testing.junit.testparameterinjector.TestParameterInjector;
+import org.junit.Test;
 import org.junit.runner.RunWith;
 
 /** Tests for {@link NonIncrementalInMemoryNodeEntry}. */
@@ -30,4 +37,56 @@
   final ConstantVersion getInitialVersion() {
     return Version.constant();
   }
+
+  @Test
+  public void rewindingLifecycle() throws InterruptedException {
+    InMemoryNodeEntry entry = createEntry();
+    entry.addReverseDepAndCheckIfDone(null); // Start evaluation.
+    entry.markRebuilding();
+    setValue(entry, new IntegerValue(1), /* errorInfo= */ null, initialVersion);
+    assertThat(entry.isDirty()).isFalse();
+    assertThat(entry.isDone()).isTrue();
+
+    entry.markDirty(DirtyType.REWIND);
+    assertThat(entry.isDirty()).isTrue();
+    assertThat(entry.isChanged()).isTrue();
+    assertThat(entry.isDone()).isFalse();
+    assertThat(entry.getTemporaryDirectDeps() instanceof GroupedDeps.WithHashSet)
+        .isEqualTo(isPartialReevaluation);
+
+    assertThatNodeEntry(entry)
+        .addReverseDepAndCheckIfDone(null)
+        .isEqualTo(DependencyState.NEEDS_SCHEDULING);
+    assertThat(entry.isReadyToEvaluate()).isTrue();
+    assertThat(entry.hasUnsignaledDeps()).isFalse();
+
+    SkyKey parent = key("parent");
+    entry.addReverseDepAndCheckIfDone(parent);
+    assertThat(entry.getDirtyState()).isEqualTo(DirtyState.NEEDS_REBUILDING);
+    assertThat(entry.isReadyToEvaluate()).isTrue();
+    assertThat(entry.hasUnsignaledDeps()).isFalse();
+    assertThat(entry.getTemporaryDirectDeps()).isEmpty();
+
+    assertThat(setValue(entry, new IntegerValue(2), /* errorInfo= */ null, initialVersion))
+        .containsExactly(parent);
+    assertThat(entry.getValue()).isEqualTo(new IntegerValue(2));
+    assertThat(entry.getVersion()).isEqualTo(initialVersion);
+  }
+
+  @Test
+  public void concurrentRewindingAllowed() throws InterruptedException {
+    InMemoryNodeEntry entry = createEntry();
+    entry.addReverseDepAndCheckIfDone(null); // Start evaluation.
+    entry.markRebuilding();
+    setValue(entry, new SkyValue() {}, /* errorInfo= */ null, initialVersion);
+    assertThat(entry.isDirty()).isFalse();
+    assertThat(entry.isDone()).isTrue();
+
+    assertThat(entry.markDirty(DirtyType.REWIND)).isNotNull();
+    assertThat(entry.markDirty(DirtyType.REWIND)).isNull();
+
+    assertThat(entry.isDirty()).isTrue();
+    assertThat(entry.isChanged()).isTrue();
+    assertThat(entry.isDone()).isFalse();
+  }
 }