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());