Add a sample of modified output files to `ExecutionFinishedEvent`.

`SequencedSkyframeExecutor` collects statistics about modified output files since (and during) last build. Expand this information to include a sample of the exec paths for those files.

Refactor `FilesystemValueChecker::getDirtyActionValues` to accept a callback for modified output files as opposed to collecting the statistics internally. This makes the class immutable.

PiperOrigin-RevId: 438857703
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/BUILD b/src/main/java/com/google/devtools/build/lib/skyframe/BUILD
index 7c0f2cb..fd0ef11 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/BUILD
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/BUILD
@@ -1276,6 +1276,7 @@
     deps = [
         "//src/main/java/com/google/devtools/build/lib/buildeventstream/proto:build_event_stream_java_proto",
         "//third_party:auto_value",
+        "//third_party:guava",
     ],
 )
 
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/ExecutionFinishedEvent.java b/src/main/java/com/google/devtools/build/lib/skyframe/ExecutionFinishedEvent.java
index 0c4b56d..651a704 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/ExecutionFinishedEvent.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/ExecutionFinishedEvent.java
@@ -14,6 +14,7 @@
 package com.google.devtools.build.lib.skyframe;
 
 import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
 import com.google.devtools.build.lib.buildeventstream.BuildEventStreamProtos.BuildMetrics.ArtifactMetrics;
 import java.time.Duration;
 
@@ -28,6 +29,7 @@
     ArtifactMetrics.FilesMetric emptyFilesMetric = ArtifactMetrics.FilesMetric.getDefaultInstance();
     return builder()
         .setOutputDirtyFiles(0)
+        .setOutputDirtyFileExecPathSample(ImmutableList.of())
         .setOutputModifiedFilesDuringPreviousBuild(0)
         .setSourceDiffCheckingDuration(Duration.ZERO)
         .setNumSourceFilesCheckedBecauseOfMissingDiffs(0)
@@ -40,6 +42,8 @@
 
   public abstract int outputDirtyFiles();
 
+  public abstract ImmutableList<String> outputDirtyFileExecPathSample();
+
   public abstract int outputModifiedFilesDuringPreviousBuild();
 
   public abstract Duration sourceDiffCheckingDuration();
@@ -64,6 +68,9 @@
   abstract static class Builder {
     abstract Builder setOutputDirtyFiles(int outputDirtyFiles);
 
+    abstract Builder setOutputDirtyFileExecPathSample(
+        ImmutableList<String> outputDirtyFileExecPathSample);
+
     abstract Builder setOutputModifiedFilesDuringPreviousBuild(
         int outputModifiedFilesDuringPreviousBuild);
 
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/FilesystemValueChecker.java b/src/main/java/com/google/devtools/build/lib/skyframe/FilesystemValueChecker.java
index cedbab3..d140652 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/FilesystemValueChecker.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/FilesystemValueChecker.java
@@ -13,6 +13,7 @@
 // limitations under the License.
 package com.google.devtools.build.lib.skyframe;
 
+import static java.util.concurrent.TimeUnit.MINUTES;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
@@ -23,7 +24,6 @@
 import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
 import com.google.common.flogger.GoogleLogger;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -33,6 +33,7 @@
 import com.google.devtools.build.lib.actions.FileStateType;
 import com.google.devtools.build.lib.concurrent.ExecutorUtil;
 import com.google.devtools.build.lib.concurrent.Sharder;
+import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
 import com.google.devtools.build.lib.profiler.AutoProfiler;
 import com.google.devtools.build.lib.profiler.AutoProfiler.ElapsedTimeReceiver;
 import com.google.devtools.build.lib.profiler.Profiler;
@@ -84,19 +85,14 @@
 
   @Nullable private final TimestampGranularityMonitor tsgm;
   private final SyscallCache syscallCache;
-  @Nullable private final Range<Long> lastExecutionTimeRange;
-  private AtomicInteger modifiedOutputFilesCounter = new AtomicInteger(0);
-  private AtomicInteger modifiedOutputFilesIntraBuildCounter = new AtomicInteger(0);
   private final int numThreads;
 
   public FilesystemValueChecker(
       @Nullable TimestampGranularityMonitor tsgm,
       SyscallCache syscallCache,
-      @Nullable Range<Long> lastExecutionTimeRange,
       int numThreads) {
     this.tsgm = tsgm;
     this.syscallCache = syscallCache;
-    this.lastExecutionTimeRange = lastExecutionTimeRange;
     this.numThreads = numThreads;
   }
   /**
@@ -158,6 +154,20 @@
     }
   }
 
+  /** Callback for modified output files for logging/metrics. */
+  @FunctionalInterface
+  @ThreadSafe
+  interface ModifiedOutputsReceiver {
+
+    /**
+     * Called on every modified artifact detected by {@link #getDirtyActionValues}.
+     *
+     * @param maybeModifiedTime Best effort modified time, -1 when not available/missing.
+     * @param artifact Modified output artifact.
+     */
+    void reportModifiedOutputFile(long maybeModifiedTime, Artifact artifact);
+  }
+
   /**
    * Return a collection of action values which have output files that are not in-sync with the
    * on-disk file value (were modified externally).
@@ -166,7 +176,8 @@
       Map<SkyKey, SkyValue> valuesMap,
       @Nullable final BatchStat batchStatter,
       ModifiedFileSet modifiedOutputFiles,
-      boolean trustRemoteArtifacts)
+      boolean trustRemoteArtifacts,
+      ModifiedOutputsReceiver modifiedOutputsReceiver)
       throws InterruptedException {
     if (modifiedOutputFiles == ModifiedFileSet.NOTHING_MODIFIED) {
       logger.atInfo().log("Not checking for dirty actions since nothing was modified");
@@ -196,8 +207,6 @@
 
     Collection<SkyKey> dirtyKeys = Sets.newConcurrentHashSet();
 
-    modifiedOutputFilesCounter.set(0);
-    modifiedOutputFilesIntraBuildCounter.set(0);
     final ImmutableSet<PathFragment> knownModifiedOutputFiles =
         modifiedOutputFiles.treatEverythingAsModified()
             ? null
@@ -227,14 +236,16 @@
                     shard,
                     knownModifiedOutputFiles,
                     sortedKnownModifiedOutputFiles,
-                    trustRemoteArtifacts)
+                    trustRemoteArtifacts,
+                    modifiedOutputsReceiver)
                 : batchStatJob(
                     dirtyKeys,
                     shard,
                     batchStatter,
                     knownModifiedOutputFiles,
                     sortedKnownModifiedOutputFiles,
-                    trustRemoteArtifacts);
+                    trustRemoteArtifacts,
+                    modifiedOutputsReceiver);
         executor.execute(job);
       }
 
@@ -259,7 +270,8 @@
       BatchStat batchStatter,
       ImmutableSet<PathFragment> knownModifiedOutputFiles,
       Supplier<NavigableSet<PathFragment>> sortedKnownModifiedOutputFiles,
-      boolean trustRemoteArtifacts) {
+      boolean trustRemoteArtifacts,
+      ModifiedOutputsReceiver modifiedOutputsReceiver) {
     return () -> {
       Map<Artifact, Pair<SkyKey, ActionExecutionValue>> fileToKeyAndValue = new HashMap<>();
       Map<Artifact, Pair<SkyKey, ActionExecutionValue>> treeArtifactsToKeyAndValue =
@@ -316,7 +328,8 @@
                 shard,
                 knownModifiedOutputFiles,
                 sortedKnownModifiedOutputFiles,
-                trustRemoteArtifacts)
+                trustRemoteArtifacts,
+                modifiedOutputsReceiver)
             .run();
         return;
       } catch (InterruptedException e) {
@@ -342,15 +355,15 @@
               ActionMetadataHandler.fileArtifactValueFromArtifact(
                   artifact, stat, syscallCache, tsgm);
           if (newData.couldBeModifiedSince(lastKnownData)) {
-            updateIntraBuildModifiedCounter(stat != null ? stat.getLastChangeTime() : -1);
-            modifiedOutputFilesCounter.getAndIncrement();
+            modifiedOutputsReceiver.reportModifiedOutputFile(
+                stat != null ? stat.getLastChangeTime() : -1, artifact);
             dirtyKeys.add(key);
           }
         } catch (IOException e) {
           logger.atWarning().withCause(e).log(
               "Error for %s (%s %s %s)", artifact, stat, keyAndValue, lastKnownData);
           // This is an unexpected failure getting a digest or symlink target.
-          modifiedOutputFilesCounter.getAndIncrement();
+          modifiedOutputsReceiver.reportModifiedOutputFile(-1, artifact);
           dirtyKeys.add(key);
         }
       }
@@ -362,35 +375,23 @@
         Artifact artifact = entry.getKey();
         if (treeArtifactIsDirty(
             entry.getKey(), entry.getValue().getSecond().getTreeArtifactValue(artifact))) {
-          Path path = artifact.getPath();
           // Count the changed directory as one "file".
           // TODO(bazel-team): There are no tests for this codepath.
-          try {
-            updateIntraBuildModifiedCounter(path.exists() ? path.getLastModifiedTime() : -1);
-          } catch (IOException e) {
-            logger.atWarning().withCause(e).log("Error for %s", entry);
-            // Do nothing here.
-          }
-
-          modifiedOutputFilesCounter.getAndIncrement();
+          modifiedOutputsReceiver.reportModifiedOutputFile(
+              getBestEffortModifiedTime(artifact.getPath()), artifact);
           dirtyKeys.add(entry.getValue().getFirst());
         }
       }
     };
   }
 
-  private void updateIntraBuildModifiedCounter(long time) {
-    if (lastExecutionTimeRange != null && lastExecutionTimeRange.contains(time)) {
-      modifiedOutputFilesIntraBuildCounter.incrementAndGet();
-    }
-  }
-
   private Runnable outputStatJob(
-      final Collection<SkyKey> dirtyKeys,
-      final List<Pair<SkyKey, ActionExecutionValue>> shard,
-      final ImmutableSet<PathFragment> knownModifiedOutputFiles,
-      final Supplier<NavigableSet<PathFragment>> sortedKnownModifiedOutputFiles,
-      boolean trustRemoteArtifacts) {
+      Collection<SkyKey> dirtyKeys,
+      List<Pair<SkyKey, ActionExecutionValue>> shard,
+      ImmutableSet<PathFragment> knownModifiedOutputFiles,
+      Supplier<NavigableSet<PathFragment>> sortedKnownModifiedOutputFiles,
+      boolean trustRemoteArtifacts,
+      ModifiedOutputsReceiver modifiedOutputsReceiver) {
     return new Runnable() {
       @Override
       public void run() {
@@ -401,7 +402,8 @@
                   value,
                   knownModifiedOutputFiles,
                   sortedKnownModifiedOutputFiles,
-                  trustRemoteArtifacts)) {
+                  trustRemoteArtifacts,
+                  modifiedOutputsReceiver)) {
             dirtyKeys.add(keyAndValue.getFirst());
           }
         }
@@ -409,18 +411,6 @@
     };
   }
 
-  /**
-   * Returns the number of modified output files inside of dirty actions.
-   */
-  int getNumberOfModifiedOutputFiles() {
-    return modifiedOutputFilesCounter.get();
-  }
-
-  /** Returns the number of modified output files that occur during the previous build. */
-  int getNumberOfModifiedOutputFilesDuringPreviousBuild() {
-    return modifiedOutputFilesIntraBuildCounter.get();
-  }
-
   private boolean treeArtifactIsDirty(Artifact artifact, TreeArtifactValue value) {
     Path path = artifact.getPath();
     if (path.isSymbolicLink()) {
@@ -448,7 +438,8 @@
   private boolean artifactIsDirtyWithDirectSystemCalls(
       ImmutableSet<PathFragment> knownModifiedOutputFiles,
       boolean trustRemoteArtifacts,
-      Map.Entry<? extends Artifact, FileArtifactValue> entry) {
+      Map.Entry<? extends Artifact, FileArtifactValue> entry,
+      ModifiedOutputsReceiver modifiedOutputsReceiver) {
     Artifact file = entry.getKey();
     FileArtifactValue lastKnownData = entry.getValue();
     if (file.isMiddlemanArtifact() || !shouldCheckFile(knownModifiedOutputFiles, file)) {
@@ -462,17 +453,17 @@
               && lastKnownData.isRemote()
               && trustRemoteArtifacts;
       if (!trustRemoteValue && fileMetadata.couldBeModifiedSince(lastKnownData)) {
-        updateIntraBuildModifiedCounter(
+        modifiedOutputsReceiver.reportModifiedOutputFile(
             fileMetadata.getType() != FileStateType.NONEXISTENT
                 ? file.getPath().getLastModifiedTime(Symlinks.FOLLOW)
-                : -1);
-        modifiedOutputFilesCounter.getAndIncrement();
+                : -1,
+            file);
         return true;
       }
       return false;
     } catch (IOException e) {
       // This is an unexpected failure getting a digest or symlink target.
-      modifiedOutputFilesCounter.getAndIncrement();
+      modifiedOutputsReceiver.reportModifiedOutputFile(/*maybeModifiedTime=*/ -1, file);
       return true;
     }
   }
@@ -481,11 +472,12 @@
       ActionExecutionValue actionValue,
       ImmutableSet<PathFragment> knownModifiedOutputFiles,
       Supplier<NavigableSet<PathFragment>> sortedKnownModifiedOutputFiles,
-      boolean trustRemoteArtifacts) {
+      boolean trustRemoteArtifacts,
+      ModifiedOutputsReceiver modifiedOutputsReceiver) {
     boolean isDirty = false;
     for (Map.Entry<Artifact, FileArtifactValue> entry : actionValue.getAllFileValues().entrySet()) {
       if (artifactIsDirtyWithDirectSystemCalls(
-          knownModifiedOutputFiles, trustRemoteArtifacts, entry)) {
+          knownModifiedOutputFiles, trustRemoteArtifacts, entry, modifiedOutputsReceiver)) {
         isDirty = true;
       }
     }
@@ -498,7 +490,10 @@
         for (Map.Entry<TreeFileArtifact, FileArtifactValue> childEntry :
             tree.getChildValues().entrySet()) {
           if (artifactIsDirtyWithDirectSystemCalls(
-              knownModifiedOutputFiles, trustRemoteArtifacts, childEntry)) {
+              knownModifiedOutputFiles,
+              trustRemoteArtifacts,
+              childEntry,
+              modifiedOutputsReceiver)) {
             isDirty = true;
           }
         }
@@ -512,23 +507,17 @@
                                 trustRemoteArtifacts,
                                 Maps.immutableEntry(
                                     archivedRepresentation.archivedTreeFileArtifact(),
-                                    archivedRepresentation.archivedFileValue())))
+                                    archivedRepresentation.archivedFileValue()),
+                                modifiedOutputsReceiver))
                     .orElse(false);
       }
 
       Artifact treeArtifact = entry.getKey();
-
       if (shouldCheckTreeArtifact(sortedKnownModifiedOutputFiles.get(), treeArtifact)
           && treeArtifactIsDirty(treeArtifact, entry.getValue())) {
-        Path path = treeArtifact.getPath();
         // Count the changed directory as one "file".
-        try {
-          updateIntraBuildModifiedCounter(path.exists() ? path.getLastModifiedTime() : -1);
-        } catch (IOException e) {
-          // Do nothing here.
-        }
-
-        modifiedOutputFilesCounter.getAndIncrement();
+        modifiedOutputsReceiver.reportModifiedOutputFile(
+            getBestEffortModifiedTime(treeArtifact.getPath()), treeArtifact);
         isDirty = true;
       }
     }
@@ -536,6 +525,16 @@
     return isDirty;
   }
 
+  private static long getBestEffortModifiedTime(Path path) {
+    try {
+      return path.exists() ? path.getLastModifiedTime() : -1;
+    } catch (IOException e) {
+      logger.atWarning().atMostEvery(1, MINUTES).withCause(e).log(
+          "Failed to get modified time for output at: %s", path);
+      return -1;
+    }
+  }
+
   private static boolean shouldCheckFile(ImmutableSet<PathFragment> knownModifiedOutputFiles,
       Artifact artifact) {
     return knownModifiedOutputFiles == null
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutor.java b/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutor.java
index 6d1a3e9..8993ec5 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutor.java
@@ -118,7 +118,9 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 import javax.annotation.Nullable;
 
@@ -129,6 +131,7 @@
 public final class SequencedSkyframeExecutor extends SkyframeExecutor {
 
   private static final GoogleLogger logger = GoogleLogger.forEnclosingClass();
+  private static final int MODIFIED_OUTPUT_PATHS_SAMPLE_SIZE = 100;
 
   /**
    * If false, the graph will not store state useful for incremental builds, saving memory but
@@ -149,9 +152,12 @@
   @Nullable private final SkyframeExecutorRepositoryHelpersHolder repositoryHelpersHolder;
   private Set<String> previousClientEnvironment = ImmutableSet.of();
 
-  private int modifiedFiles;
-  private int outputDirtyFiles;
-  private int modifiedFilesDuringPreviousBuild;
+  private final AtomicInteger modifiedFiles = new AtomicInteger();
+  private final AtomicInteger outputDirtyFiles = new AtomicInteger();
+  private final ArrayBlockingQueue<String> outputDirtyFilesExecPathSample =
+      new ArrayBlockingQueue<>(MODIFIED_OUTPUT_PATHS_SAMPLE_SIZE);
+  private final AtomicInteger modifiedFilesDuringPreviousBuild = new AtomicInteger();
+
   private Duration sourceDiffCheckingDuration = Duration.ofSeconds(-1L);
   private int numSourceFilesCheckedBecauseOfMissingDiffs;
   private Duration outputTreeDiffCheckingDuration = Duration.ofSeconds(-1L);
@@ -359,7 +365,7 @@
       OptionsProvider options)
       throws InterruptedException, AbruptExitException {
     TimestampGranularityMonitor tsgm = this.tsgm.get();
-    modifiedFiles = 0;
+    modifiedFiles.set(0);
     numSourceFilesCheckedBecauseOfMissingDiffs = 0;
 
     boolean managedDirectoriesChanged =
@@ -521,8 +527,7 @@
       memoizingEvaluator.evaluate(ImmutableList.of(), evaluationContext);
 
       FilesystemValueChecker fsvc =
-          new FilesystemValueChecker(
-              tsgm, perCommandSyscallCache, /*lastExecutionTimeRange=*/ null, fsvcThreads);
+          new FilesystemValueChecker(tsgm, perCommandSyscallCache, fsvcThreads);
       // We need to manually check for changes to known files. This entails finding all dirty file
       // system values under package roots for which we don't have diff information. If at least
       // one path entry doesn't have diff information, then we're going to have to iterate over
@@ -596,8 +601,7 @@
           "About to scan %d external files",
           externalFilesKnowledge.nonOutputExternalFilesSeen.size());
       FilesystemValueChecker fsvc =
-          new FilesystemValueChecker(
-              tsgm, perCommandSyscallCache, /*lastExecutionTimeRange=*/ null, fsvcThreads);
+          new FilesystemValueChecker(tsgm, perCommandSyscallCache, fsvcThreads);
       ImmutableBatchDirtyResult batchDirtyResult;
       try (SilentCloseable c = Profiler.instance().profile("fsvc.getDirtyExternalKeys")) {
         Map<SkyKey, SkyValue> externalDirtyNodes = new ConcurrentHashMap<>();
@@ -656,8 +660,9 @@
 
     recordingDiffer.invalidate(keysToBeChangedLaterInThisBuild);
     recordingDiffer.inject(changedKeysWithNewValues);
-    modifiedFiles += getNumberOfModifiedFiles(keysToBeChangedLaterInThisBuild);
-    modifiedFiles += getNumberOfModifiedFiles(changedKeysWithNewValues.keySet());
+    modifiedFiles.addAndGet(
+        getNumberOfModifiedFiles(keysToBeChangedLaterInThisBuild)
+            + getNumberOfModifiedFiles(changedKeysWithNewValues.keySet()));
     numSourceFilesCheckedBecauseOfMissingDiffs += numSourceFilesCheckedIfDiffWasMissing;
     incrementalBuildMonitor.accrue(keysToBeChangedLaterInThisBuild);
     incrementalBuildMonitor.accrue(changedKeysWithNewValues.keySet());
@@ -783,7 +788,6 @@
           new FilesystemValueChecker(
                   tsgm,
                   perCommandSyscallCache,
-                  /*lastExecutionTimeRange=*/ null,
                   /*numThreads=*/ 200)
               .getDirtyKeys(memoizingEvaluator.getValues(), new BasicFilesystemDirtinessChecker());
     } else {
@@ -813,7 +817,6 @@
         new FilesystemValueChecker(
             Preconditions.checkNotNull(tsgm.get()),
             perCommandSyscallCache,
-            lastExecutionTimeRange,
             fsvcThreads);
     BatchStat batchStatter = outputService == null ? null : outputService.getBatchStatter();
     recordingDiffer.invalidate(
@@ -821,11 +824,19 @@
             memoizingEvaluator.getValues(),
             batchStatter,
             modifiedOutputFiles,
-            trustRemoteArtifacts));
-    modifiedFiles += fsvc.getNumberOfModifiedOutputFiles();
-    outputDirtyFiles += fsvc.getNumberOfModifiedOutputFiles();
-    modifiedFilesDuringPreviousBuild += fsvc.getNumberOfModifiedOutputFilesDuringPreviousBuild();
-    logger.atInfo().log("Found %d modified files from last build", modifiedFiles);
+            trustRemoteArtifacts,
+            (maybeModifiedTime, artifact) -> {
+              modifiedFiles.incrementAndGet();
+              int dirtyOutputsCount = outputDirtyFiles.incrementAndGet();
+              if (lastExecutionTimeRange != null
+                  && lastExecutionTimeRange.contains(maybeModifiedTime)) {
+                modifiedFilesDuringPreviousBuild.incrementAndGet();
+              }
+              if (dirtyOutputsCount <= MODIFIED_OUTPUT_PATHS_SAMPLE_SIZE) {
+                outputDirtyFilesExecPathSample.offer(artifact.getExecPathString());
+              }
+            }));
+    logger.atInfo().log("Found %d modified files from last build", modifiedFiles.get());
     long stopTime = System.nanoTime();
     Profiler.instance()
         .logSimpleTask(startTime, stopTime, ProfilerTask.INFO, "detectModifiedOutputFiles");
@@ -954,14 +965,15 @@
   protected ExecutionFinishedEvent.Builder createExecutionFinishedEventInternal() {
     ExecutionFinishedEvent.Builder builder =
         ExecutionFinishedEvent.builder()
-            .setOutputDirtyFiles(outputDirtyFiles)
-            .setOutputModifiedFilesDuringPreviousBuild(modifiedFilesDuringPreviousBuild)
+            .setOutputDirtyFiles(outputDirtyFiles.getAndSet(0))
+            .setOutputDirtyFileExecPathSample(ImmutableList.copyOf(outputDirtyFilesExecPathSample))
+            .setOutputModifiedFilesDuringPreviousBuild(
+                modifiedFilesDuringPreviousBuild.getAndSet(0))
             .setSourceDiffCheckingDuration(sourceDiffCheckingDuration)
             .setNumSourceFilesCheckedBecauseOfMissingDiffs(
                 numSourceFilesCheckedBecauseOfMissingDiffs)
             .setOutputTreeDiffCheckingDuration(outputTreeDiffCheckingDuration);
-    outputDirtyFiles = 0;
-    modifiedFilesDuringPreviousBuild = 0;
+    outputDirtyFilesExecPathSample.clear();
     sourceDiffCheckingDuration = Duration.ZERO;
     outputTreeDiffCheckingDuration = Duration.ZERO;
     return builder;
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/FilesystemValueCheckerTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/FilesystemValueCheckerTest.java
index ca58d21..729375a 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/FilesystemValueCheckerTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/FilesystemValueCheckerTest.java
@@ -17,6 +17,9 @@
 import static com.google.common.truth.Truth.assertThat;
 import static com.google.common.truth.Truth.assertWithMessage;
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
 
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
@@ -53,6 +56,7 @@
 import com.google.devtools.build.lib.pkgcache.PathPackageLocator;
 import com.google.devtools.build.lib.skyframe.DirtinessCheckerUtils.BasicFilesystemDirtinessChecker;
 import com.google.devtools.build.lib.skyframe.ExternalFilesHelper.ExternalFileAction;
+import com.google.devtools.build.lib.skyframe.FilesystemValueChecker.ModifiedOutputsReceiver;
 import com.google.devtools.build.lib.skyframe.PackageLookupFunction.CrossRepositoryLabelViolationStrategy;
 import com.google.devtools.build.lib.testutil.ManualClock;
 import com.google.devtools.build.lib.testutil.TestConstants;
@@ -106,6 +110,7 @@
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
 
 /** Tests for {@link FilesystemValueChecker}. */
 @RunWith(TestParameterInjector.class)
@@ -144,6 +149,11 @@
   private Path pkgRoot;
   @TestParameter private BatchStatMode batchStat;
 
+  private final ModifiedOutputsReceiver mockModifiedOutputsReceiver =
+      mock(ModifiedOutputsReceiver.class);
+  private final ArgumentCaptor<Artifact> modifiedOutputsCaptor =
+      ArgumentCaptor.forClass(Artifact.class);
+
   private SpecialArtifact createTreeArtifact(String relPath) throws IOException {
     String outSegment = "bin";
     Path outputDir = fs.getPath("/" + outSegment);
@@ -335,6 +345,13 @@
     PrecomputedValue.PATH_PACKAGE_LOCATOR.set(differencer, pkgLocator.get());
   }
 
+  @Before
+  public void setupModifiedOutputReceiverMock() {
+    doNothing()
+        .when(mockModifiedOutputsReceiver)
+        .reportModifiedOutputFile(anyLong(), modifiedOutputsCaptor.capture());
+  }
+
   public static ImmutableList<Object[]> batchStatModes() {
     return Arrays.stream(BatchStatMode.values())
         .map(mode -> new BatchStatMode[] {mode})
@@ -482,16 +499,13 @@
 
   private Collection<SkyKey> getDirtyActionValues(ImmutableMap<SkyKey, SkyValue> valuesMap)
       throws InterruptedException {
-    return new FilesystemValueChecker(
-            /*tsgm=*/ null,
-            SyscallCache.NO_CACHE,
-            /*lastExecutionTimeRange=*/ null,
-            FSVC_THREADS_FOR_TEST)
+    return new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
         .getDirtyActionValues(
             valuesMap,
             batchStat.getBatchStat(fs),
             ModifiedFileSet.EVERYTHING_MODIFIED,
-            /*trustRemoteArtifacts=*/ false);
+            /*trustRemoteArtifacts=*/ false,
+            (ignored, ignored2) -> {});
   }
 
   private TreeFileArtifact createTreeFileArtifactWithContent(
@@ -519,7 +533,6 @@
         new FilesystemValueChecker(
             /*tsgm=*/ null,
             SyscallCache.NO_CACHE,
-            /*lastExecutionTimeRange=*/ null,
             FSVC_THREADS_FOR_TEST);
     assertEmptyDiff(getDirtyFilesystemKeys(evaluator, checker));
   }
@@ -530,7 +543,6 @@
         new FilesystemValueChecker(
             /*tsgm=*/ null,
             SyscallCache.NO_CACHE,
-            /*lastExecutionTimeRange=*/ null,
             FSVC_THREADS_FOR_TEST);
 
     Path path = fs.getPath("/foo");
@@ -571,7 +583,6 @@
         new FilesystemValueChecker(
             /*tsgm=*/ null,
             SyscallCache.NO_CACHE,
-            /*lastExecutionTimeRange=*/ null,
             FSVC_THREADS_FOR_TEST);
 
     Path path = fs.getPath("/foo");
@@ -639,7 +650,6 @@
         new FilesystemValueChecker(
             /*tsgm=*/ null,
             SyscallCache.NO_CACHE,
-            /*lastExecutionTimeRange=*/ null,
             FSVC_THREADS_FOR_TEST);
 
     Path path1 = fs.getPath("/foo1");
@@ -700,7 +710,6 @@
         new FilesystemValueChecker(
             /*tsgm=*/ null,
             SyscallCache.NO_CACHE,
-            /*lastExecutionTimeRange=*/ null,
             FSVC_THREADS_FOR_TEST);
     Diff diff = getDirtyFilesystemKeys(evaluator, checker);
     assertThat(diff.changedKeysWithoutNewValues()).isEmpty();
@@ -725,7 +734,6 @@
         new FilesystemValueChecker(
             /*tsgm=*/ null,
             SyscallCache.NO_CACHE,
-            /*lastExecutionTimeRange=*/ null,
             FSVC_THREADS_FOR_TEST);
     Diff diff = getDirtyFilesystemKeys(evaluator, checker);
     assertThat(diff.changedKeysWithoutNewValues()).isEmpty();
@@ -787,16 +795,13 @@
                         ImmutableSet.of(out2)))));
     assertThat(evaluator.evaluate(ImmutableList.of(), evaluationContext).hasError()).isFalse();
     assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     batchStatter,
                     ModifiedFileSet.EVERYTHING_MODIFIED,
-                    /* trustRemoteArtifacts= */ false))
+                    /* trustRemoteArtifacts= */ false,
+                    (ignored, ignored2) -> {}))
         .isEmpty();
 
     tsgm.waitForTimestampGranularity(OutErr.SYSTEM_OUT_ERR);
@@ -806,66 +811,51 @@
       Artifact file, SkyKey actionKey, BatchStat batchStatter, TimestampGranularityMonitor tsgm)
       throws InterruptedException {
     assertThat(
-            new FilesystemValueChecker(
-                    tsgm,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(tsgm, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     batchStatter,
                     ModifiedFileSet.EVERYTHING_MODIFIED,
-                    /* trustRemoteArtifacts= */ false))
+                    /* trustRemoteArtifacts= */ false,
+                    (ignored, ignored2) -> {}))
         .containsExactly(actionKey);
     assertThat(
-            new FilesystemValueChecker(
-                    tsgm,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(tsgm, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     batchStatter,
                     ModifiedFileSet.EVERYTHING_DELETED,
-                    /* trustRemoteArtifacts= */ false))
+                    /* trustRemoteArtifacts= */ false,
+                    (ignored, ignored2) -> {}))
         .containsExactly(actionKey);
     assertThat(
-            new FilesystemValueChecker(
-                    tsgm,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(tsgm, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     batchStatter,
                     new ModifiedFileSet.Builder().modify(file.getExecPath()).build(),
-                    /* trustRemoteArtifacts= */ false))
+                    /* trustRemoteArtifacts= */ false,
+                    (ignored, ignored2) -> {}))
         .containsExactly(actionKey);
     assertThat(
-            new FilesystemValueChecker(
-                    tsgm,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(tsgm, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     batchStatter,
                     new ModifiedFileSet.Builder()
                         .modify(file.getExecPath().getParentDirectory())
                         .build(),
-                    /* trustRemoteArtifacts= */ false))
+                    /* trustRemoteArtifacts= */ false,
+                    (ignored, ignored2) -> {}))
         .isEmpty();
     assertThat(
-            new FilesystemValueChecker(
-                    tsgm,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(tsgm, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     batchStatter,
                     ModifiedFileSet.NOTHING_MODIFIED,
-                    /* trustRemoteArtifacts= */ false))
+                    /* trustRemoteArtifacts= */ false,
+                    (ignored, ignored2) -> {}))
         .isEmpty();
   }
 
@@ -904,20 +894,19 @@
     differencer.inject(
         ImmutableMap.of(actionKey, actionValueWithTreeArtifacts(ImmutableList.of(treeFile))));
     evaluate();
-
     FileSystemUtils.touchFile(tree.getPath().getRelative(touchedTreePath));
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    modifiedSet.getModifiedFileSet(tree.getExecPath()),
-                    /*trustRemoteArtifacts=*/ false))
-        .isEmpty();
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                modifiedSet.getModifiedFileSet(tree.getExecPath()),
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).isEmpty();
+    assertThat(modifiedOutputsCaptor.getAllValues()).isEmpty();
   }
 
   @Test
@@ -929,20 +918,19 @@
     differencer.inject(
         ImmutableMap.of(actionKey, actionValueWithTreeArtifact(tree, TreeArtifactValue.empty())));
     evaluate();
-
     assertThat(tree.getPath().delete()).isTrue();
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    modifiedSet.getModifiedFileSet(tree.getExecPath()),
-                    /*trustRemoteArtifacts=*/ false))
-        .containsExactly(actionKey);
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                modifiedSet.getModifiedFileSet(tree.getExecPath()),
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).containsExactly(actionKey);
+    assertThat(modifiedOutputsCaptor.getAllValues()).containsExactly(tree);
   }
 
   @Test
@@ -954,24 +942,22 @@
     differencer.inject(
         ImmutableMap.of(actionKey, actionValueWithTreeArtifact(tree, TreeArtifactValue.empty())));
     evaluate();
-
     Path dummyEmptyDir = fs.getPath("/bin").getRelative("dir");
     dummyEmptyDir.createDirectoryAndParents();
     assertThat(tree.getPath().delete()).isTrue();
     tree.getPath().createSymbolicLink(dummyEmptyDir);
 
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    ModifiedFileSet.EVERYTHING_MODIFIED,
-                    /* trustRemoteArtifacts= */ false))
-        .containsExactly(actionKey); // Symbolic links should count as dirty
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                ModifiedFileSet.EVERYTHING_MODIFIED,
+                /* trustRemoteArtifacts= */ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).containsExactly(actionKey);
+    assertThat(modifiedOutputsCaptor.getAllValues()).containsExactly(tree);
   }
 
   @Test
@@ -984,20 +970,19 @@
     differencer.inject(
         ImmutableMap.of(actionKey, actionValueWithTreeArtifacts(ImmutableList.of(treeFile))));
     evaluate();
-
     FileSystemUtils.writeIsoLatin1(treeFile.getPath(), "other text");
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    modifiedSet.getModifiedFileSet(treeFile.getExecPath()),
-                    /*trustRemoteArtifacts=*/ false))
-        .containsExactly(actionKey);
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                modifiedSet.getModifiedFileSet(treeFile.getExecPath()),
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).containsExactly(actionKey);
+    assertThat(modifiedOutputsCaptor.getAllValues()).containsExactly(treeFile);
   }
 
   @Test
@@ -1013,18 +998,17 @@
 
     TreeFileArtifact newFile = TreeFileArtifact.createTreeOutput(tree, "file2");
     FileSystemUtils.writeIsoLatin1(newFile.getPath());
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    modifiedSet.getModifiedFileSet(newFile.getExecPath()),
-                    /*trustRemoteArtifacts=*/ false))
-        .containsExactly(actionKey);
+    Collection<SkyKey> dirtyActionValues =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                modifiedSet.getModifiedFileSet(newFile.getExecPath()),
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionValues).containsExactly(actionKey);
+    assertThat(modifiedOutputsCaptor.getAllValues()).containsExactly(tree);
   }
 
   @Test
@@ -1036,21 +1020,20 @@
     differencer.inject(
         ImmutableMap.of(actionKey, actionValueWithTreeArtifact(tree, TreeArtifactValue.empty())));
     evaluate();
-
     TreeFileArtifact newFile = TreeFileArtifact.createTreeOutput(tree, "file");
     FileSystemUtils.writeIsoLatin1(newFile.getPath());
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    modifiedSet.getModifiedFileSet(newFile.getExecPath()),
-                    /*trustRemoteArtifacts=*/ false))
-        .containsExactly(actionKey);
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                modifiedSet.getModifiedFileSet(newFile.getExecPath()),
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).containsExactly(actionKey);
+    assertThat(modifiedOutputsCaptor.getAllValues()).containsExactly(tree);
   }
 
   @Test
@@ -1063,20 +1046,19 @@
     differencer.inject(
         ImmutableMap.of(actionKey, actionValueWithTreeArtifacts(ImmutableList.of(treeFile))));
     evaluate();
-
     assertThat(treeFile.getPath().delete()).isTrue();
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    modifiedSet.getModifiedFileSet(treeFile.getExecPath()),
-                    /*trustRemoteArtifacts=*/ false))
-        .containsExactly(actionKey);
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                modifiedSet.getModifiedFileSet(treeFile.getExecPath()),
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).containsExactly(actionKey);
+    assertThat(modifiedOutputsCaptor.getAllValues()).containsExactly(treeFile, tree);
   }
 
   @Test
@@ -1094,23 +1076,22 @@
             actionKey1,
             actionValueWithTreeArtifacts(ImmutableList.of(tree1File)),
             actionKey2,
-            actionValueWithTreeArtifacts(ImmutableList.of(tree1File))));
+            actionValueWithTreeArtifacts(ImmutableList.of(tree2File))));
     evaluate();
-
     FileSystemUtils.writeIsoLatin1(tree1File.getPath(), "new text");
     assertThat(tree2File.getPath().delete()).isTrue();
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    ModifiedFileSet.EVERYTHING_MODIFIED,
-                    /*trustRemoteArtifacts=*/ false))
-        .containsExactly(actionKey1, actionKey2);
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                ModifiedFileSet.EVERYTHING_MODIFIED,
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).containsExactly(actionKey1, actionKey2);
+    assertThat(modifiedOutputsCaptor.getAllValues()).containsExactly(tree1File, tree2, tree2File);
   }
 
   @Test
@@ -1131,23 +1112,24 @@
             actionKey2,
             actionValueWithTreeArtifacts(ImmutableList.of(tree2File))));
     evaluate();
-
     FileSystemUtils.writeIsoLatin1(tree1File.getPath(), "new text");
     assertThat(tree2File.getPath().delete()).isTrue();
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    ModifiedFileSet.builder()
-                        .modify((reportFirst ? tree1File : tree2File).getExecPath())
-                        .build(),
-                    /*trustRemoteArtifacts=*/ false))
-        .containsExactly(reportFirst ? actionKey1 : actionKey2);
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                ModifiedFileSet.builder()
+                    .modify((reportFirst ? tree1File : tree2File).getExecPath())
+                    .build(),
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).containsExactly(reportFirst ? actionKey1 : actionKey2);
+    assertThat(modifiedOutputsCaptor.getAllValues())
+        .containsExactlyElementsIn(
+            reportFirst ? ImmutableList.of(tree1File) : ImmutableList.of(tree2File, tree2));
   }
 
   @Test
@@ -1174,25 +1156,25 @@
             actionKey3,
             actionValueWithTreeArtifacts(ImmutableList.of(treeCFile))));
     evaluate();
-
     assertThat(treeAFile.getPath().delete()).isTrue();
     assertThat(treeBFile.getPath().delete()).isTrue();
     assertThat(treeCFile.getPath().delete()).isTrue();
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    ModifiedFileSet.builder()
-                        .modify(treeAFile.getExecPath())
-                        .modify(treeCFile.getExecPath())
-                        .build(),
-                    /*trustRemoteArtifacts=*/ false))
-        .containsExactly(actionKey1, actionKey3);
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                ModifiedFileSet.builder()
+                    .modify(treeAFile.getExecPath())
+                    .modify(treeCFile.getExecPath())
+                    .build(),
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).containsExactly(actionKey1, actionKey3);
+    assertThat(modifiedOutputsCaptor.getAllValues())
+        .containsExactly(treeAFile, treeA, treeCFile, treeC);
   }
 
   @Test
@@ -1204,20 +1186,19 @@
     differencer.inject(
         ImmutableMap.of(actionKey, actionValueWithTreeArtifacts(ImmutableList.of(treeFile))));
     evaluate();
-
     assertThat(treeFile.getPath().delete()).isTrue();
-    assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
-                .getDirtyActionValues(
-                    evaluator.getValues(),
-                    batchStat.getBatchStat(fs),
-                    ModifiedFileSet.NOTHING_MODIFIED,
-                    /*trustRemoteArtifacts=*/ false))
-        .isEmpty();
+
+    Collection<SkyKey> dirtyActionKeys =
+        new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
+            .getDirtyActionValues(
+                evaluator.getValues(),
+                batchStat.getBatchStat(fs),
+                ModifiedFileSet.NOTHING_MODIFIED,
+                /*trustRemoteArtifacts=*/ false,
+                mockModifiedOutputsReceiver);
+
+    assertThat(dirtyActionKeys).isEmpty();
+    assertThat(modifiedOutputsCaptor.getAllValues()).isEmpty();
   }
 
   private void evaluate() throws InterruptedException {
@@ -1379,32 +1360,26 @@
                 .hasError())
         .isFalse();
     assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     /* batchStatter= */ null,
                     ModifiedFileSet.EVERYTHING_MODIFIED,
-                    /* trustRemoteArtifacts= */ true))
+                    /* trustRemoteArtifacts= */ true,
+                    (ignored, ignored2) -> {}))
         .isEmpty();
 
     // Create the "out1" artifact on the filesystem and test that it invalidates the generating
     // action's SkyKey.
     FileSystemUtils.writeContentAsLatin1(out1.getPath(), "new-foo-content");
     assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     /* batchStatter= */ null,
                     ModifiedFileSet.EVERYTHING_MODIFIED,
-                    /* trustRemoteArtifacts= */ true))
+                    /* trustRemoteArtifacts= */ true,
+                    (ignored, ignored2) -> {}))
         .containsExactly(actionKey1);
   }
 
@@ -1437,32 +1412,26 @@
     assertThat(evaluator.evaluate(ImmutableList.of(actionKey), evaluationContext).hasError())
         .isFalse();
     assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     /* batchStatter= */ null,
                     ModifiedFileSet.EVERYTHING_MODIFIED,
-                    /* trustRemoteArtifacts= */ false))
+                    /* trustRemoteArtifacts= */ false,
+                    (ignored, ignored2) -> {}))
         .isEmpty();
 
     // Create dir/foo on the local disk and test that it invalidates the associated sky key.
     TreeFileArtifact fooArtifact = TreeFileArtifact.createTreeOutput(treeArtifact, "foo");
     FileSystemUtils.writeContentAsLatin1(fooArtifact.getPath(), "new-foo-content");
     assertThat(
-            new FilesystemValueChecker(
-                    /*tsgm=*/ null,
-                    SyscallCache.NO_CACHE,
-                    /*lastExecutionTimeRange=*/ null,
-                    FSVC_THREADS_FOR_TEST)
+            new FilesystemValueChecker(/*tsgm=*/ null, SyscallCache.NO_CACHE, FSVC_THREADS_FOR_TEST)
                 .getDirtyActionValues(
                     evaluator.getValues(),
                     /* batchStatter= */ null,
                     ModifiedFileSet.EVERYTHING_MODIFIED,
-                    /* trustRemoteArtifacts= */ false))
+                    /* trustRemoteArtifacts= */ false,
+                    (ignored, ignored2) -> {}))
         .containsExactly(actionKey);
   }
 
@@ -1485,7 +1454,6 @@
         new FilesystemValueChecker(
             /*tsgm=*/ null,
             SyscallCache.NO_CACHE,
-            /*lastExecutionTimeRange=*/ null,
             FSVC_THREADS_FOR_TEST);
 
     assertEmptyDiff(getDirtyFilesystemKeys(evaluator, checker));
diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutorTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutorTest.java
index 5e637bc..4d3d771 100644
--- a/src/test/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutorTest.java
+++ b/src/test/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutorTest.java
@@ -676,7 +676,6 @@
         new FilesystemValueChecker(
                 new TimestampGranularityMonitor(BlazeClock.instance()),
                 SyscallCache.NO_CACHE,
-                /* lastExecutionTimeRange= */ null,
                 /* numThreads= */ 20)
             .getDirtyKeys(
                 skyframeExecutor.getEvaluator().getValues(), new BasicFilesystemDirtinessChecker());