From 15c55d5234571ffd7a42572e50820d5f22212878 Mon Sep 17 00:00:00 2001 From: ajurkowski Date: Fri, 6 Aug 2021 17:58:25 -0700 Subject: [PATCH] Implement handling of diffs which can skip ancestor directories. Implement a new diff checker which deduces affected `FileStateValue` and `DirectoryListingStateValue` keys from diffs which can skip ancestor directories in them. An extreme example of such diff would be one which only lists the leaf files when adding/deleting them (e.g. adding `a/b/c/d/file` where `a` does not currently exist). Use the new algorithm for diffs marked as potentially missing ancestor directory entries. The new logic has a very similar performance to the existing one with the difference of: - adding new file with within new directories -- it does less stats since it deduces the directory from the leaf existence - deleting a file -- it performs 1 extra stat since it cannot assume that the parent of a deleted file still exists - unknown entries -- it tries to be lazy about stating those; we would only do that if that can prevent invalidation of a directory listing. PiperOrigin-RevId: 389301057 --- .../google/devtools/build/lib/skyframe/BUILD | 9 +- ...eSystemValueCheckerInferringAncestors.java | 373 +++++++++ .../lib/skyframe/FilesystemValueChecker.java | 4 +- .../skyframe/SequencedSkyframeExecutor.java | 9 +- .../build/lib/skyframe/SkyframeExecutor.java | 26 +- .../common/DirectoryListingHelper.java | 5 + .../build/lib/vfs/ModifiedFileSet.java | 44 +- .../com/google/devtools/build/skyframe/BUILD | 1 + .../devtools/build/skyframe/Differencer.java | 32 +- src/main/protobuf/failure_details.proto | 10 + .../devtools/build/lib/analysis/util/BUILD | 1 + .../lib/analysis/util/BuildViewTestCase.java | 6 +- .../devtools/build/lib/bazel/repository/BUILD | 1 + .../LocalConfigPlatformFunctionTest.java | 4 +- .../build/lib/bazel/repository/starlark/BUILD | 2 + .../StarlarkRepositoryIntegrationTest.java | 3 +- .../google/devtools/build/lib/packages/BUILD | 1 + .../packages/util/PackageLoadingTestCase.java | 3 +- .../google/devtools/build/lib/pkgcache/BUILD | 5 + .../pkgcache/BuildFileModificationTest.java | 3 +- .../lib/pkgcache/LoadingPhaseRunnerTest.java | 3 +- .../lib/pkgcache/PackageLoadingTest.java | 3 +- .../lib/pkgcache/QueryPreloadingTestCase.java | 6 +- .../pkgcache/TargetPatternEvaluatorTest.java | 6 +- ...temValueCheckerInferringAncestorsTest.java | 722 ++++++++++++++++++ .../RepositoryMappingFunctionTest.java | 4 +- .../SequencedSkyframeExecutorTest.java | 6 +- .../skyframe/WorkspaceFileFunctionTest.java | 6 +- .../skyframe/WorkspaceNameFunctionTest.java | 4 +- .../build/lib/vfs/ModifiedFileSetTest.java | 39 +- 30 files changed, 1268 insertions(+), 73 deletions(-) create mode 100644 src/main/java/com/google/devtools/build/lib/skyframe/FileSystemValueCheckerInferringAncestors.java create mode 100644 src/test/java/com/google/devtools/build/lib/skyframe/FileSystemValueCheckerInferringAncestorsTest.java 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 ac2ee831b65260..fd5fdf137bcc41 100644 --- a/src/main/java/com/google/devtools/build/lib/skyframe/BUILD +++ b/src/main/java/com/google/devtools/build/lib/skyframe/BUILD @@ -1457,10 +1457,14 @@ java_library( java_library( name = "filesystem_value_checker", - srcs = ["FilesystemValueChecker.java"], + srcs = [ + "FileSystemValueCheckerInferringAncestors.java", + "FilesystemValueChecker.java", + ], deps = [ ":action_execution_value", ":action_metadata_handler", + ":directory_listing_state_value", ":sky_functions", ":sky_value_dirtiness_checker", ":tree_artifact_value", @@ -1469,11 +1473,14 @@ java_library( "//src/main/java/com/google/devtools/build/lib/concurrent", "//src/main/java/com/google/devtools/build/lib/profiler", "//src/main/java/com/google/devtools/build/lib/util", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", + "//src/main/java/com/google/devtools/build/lib/util:detailed_exit_code", "//src/main/java/com/google/devtools/build/lib/util/io", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment", "//src/main/java/com/google/devtools/build/skyframe", "//src/main/java/com/google/devtools/build/skyframe:skyframe-objects", + "//src/main/protobuf:failure_details_java_proto", "//third_party:flogger", "//third_party:guava", "//third_party:jsr305", diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/FileSystemValueCheckerInferringAncestors.java b/src/main/java/com/google/devtools/build/lib/skyframe/FileSystemValueCheckerInferringAncestors.java new file mode 100644 index 00000000000000..fd905a35929919 --- /dev/null +++ b/src/main/java/com/google/devtools/build/lib/skyframe/FileSystemValueCheckerInferringAncestors.java @@ -0,0 +1,373 @@ +// Copyright 2021 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.google.devtools.build.lib.skyframe; + +import static com.google.common.collect.ImmutableList.toImmutableList; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Futures; +import com.google.devtools.build.lib.actions.FileStateType; +import com.google.devtools.build.lib.actions.FileStateValue; +import com.google.devtools.build.lib.concurrent.ExecutorUtil; +import com.google.devtools.build.lib.server.FailureDetails; +import com.google.devtools.build.lib.server.FailureDetails.DiffAwareness.Code; +import com.google.devtools.build.lib.server.FailureDetails.FailureDetail; +import com.google.devtools.build.lib.util.AbruptExitException; +import com.google.devtools.build.lib.util.DetailedExitCode; +import com.google.devtools.build.lib.util.io.TimestampGranularityMonitor; +import com.google.devtools.build.lib.vfs.Dirent; +import com.google.devtools.build.lib.vfs.RootedPath; +import com.google.devtools.build.skyframe.ImmutableDiff; +import com.google.devtools.build.skyframe.SkyKey; +import com.google.devtools.build.skyframe.SkyValue; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import javax.annotation.Nullable; + +/** + * A helper class to find dirty {@link FileStateValue} and {@link DirectoryListingStateValue} nodes + * based on a potentially incomplete diffs. + * + *

Infers directories from files meaning that it will work for diffs which exclude entries for + * affected ancestor entries of nodes. It is also resilient to diffs which report only a root of + * deleted subtree. + */ +final class FileSystemValueCheckerInferringAncestors { + @Nullable private final TimestampGranularityMonitor tsgm; + private final Map graphValues; + private final Map graphDoneValues; + private final Map nodeStates; + private final Set valuesToInvalidate = Sets.newConcurrentHashSet(); + private final ConcurrentMap valuesToInject = new ConcurrentHashMap<>(); + + private static final class NodeVisitState { + + private NodeVisitState(boolean collectMaybeDeletedChildren) { + if (collectMaybeDeletedChildren) { + maybeDeletedChildren = ConcurrentHashMap.newKeySet(); + } + } + + private final AtomicInteger childrenToProcess = new AtomicInteger(); + // non-volatile since childrenToProcess ensures happens-before relationship. + private boolean needsToBeVisited; + + private volatile boolean isInferredDirectory; + private volatile Set maybeDeletedChildren; + + void markInferredDirectory() { + isInferredDirectory = true; + // maybeTypeChangedChildren is used to figure out if the entry is a directory, since we + // already inferred it, we can stop collecting those. + maybeDeletedChildren = null; + } + + void addMaybeDeletedChild(String child) { + Set localMaybeDeletedChildren = maybeDeletedChildren; + if (localMaybeDeletedChildren != null) { + localMaybeDeletedChildren.add(child); + } + } + + boolean signalFinishedChild(boolean needsToBeVisited) { + // The order is important, we must update this.needsToBeVisited before decrementing + // childrenToProcess -- that operation ensures this change is visible to other threads doing + // the same (including this thread picking up a true set by another one). + if (needsToBeVisited) { + this.needsToBeVisited = true; + } + int childrenLeft = childrenToProcess.decrementAndGet(); + // If we hit 0, we know that all other threads have set and propagated needsToBeVisited. + return childrenLeft == 0 && this.needsToBeVisited; + } + } + + private FileSystemValueCheckerInferringAncestors( + @Nullable TimestampGranularityMonitor tsgm, + Map graphValues, + Map graphDoneValues, + Map nodeStates) { + this.tsgm = tsgm; + this.graphValues = graphValues; + this.graphDoneValues = graphDoneValues; + this.nodeStates = nodeStates; + } + + @SuppressWarnings("ReferenceEquality") + static ImmutableDiff getDiffWithInferredAncestors( + @Nullable TimestampGranularityMonitor tsgm, + Map graphValues, + Map graphDoneValues, + Iterable modifiedKeys, + int nThreads) + throws InterruptedException, AbruptExitException { + Map nodeStates = new HashMap<>(); + for (SkyKey key : modifiedKeys) { + RootedPath top = ((FileStateValue.Key) key).argument(); + // Start with false since the reported diff does not mean we are adding a child. + boolean lastCreated = false; + for (RootedPath path = top; path != null; path = path.getParentDirectory()) { + @Nullable NodeVisitState existingState = nodeStates.get(path); + NodeVisitState state; + // We disable the optimization which detects whether directory still exists based on the + // list of deleted children and listing. It is possible for the diff to report a deleted + // directory without listing all of the files under it as deleted. + if (existingState == null) { + state = new NodeVisitState(/*collectMaybeDeletedChildren=*/ path != top); + nodeStates.put(path, state); + } else { + state = existingState; + if (path == top) { + state.maybeDeletedChildren = null; + } + } + if (lastCreated) { + state.childrenToProcess.incrementAndGet(); + } + lastCreated = existingState == null; + } + } + + return new FileSystemValueCheckerInferringAncestors( + tsgm, graphValues, graphDoneValues, Collections.unmodifiableMap(nodeStates)) + .processEntries(nThreads); + } + + private ImmutableDiff processEntries(int nThreads) + throws InterruptedException, AbruptExitException { + ExecutorService executor = Executors.newFixedThreadPool(nThreads); + + ImmutableList> futures = + nodeStates.entrySet().stream() + .filter(e -> e.getValue().childrenToProcess.get() == 0) + .map( + e -> + executor.submit( + () -> { + processEntry(e.getKey(), e.getValue()); + return null; + })) + .collect(toImmutableList()); + + if (ExecutorUtil.interruptibleShutdown(executor)) { + throw new InterruptedException(); + } + + for (Future future : futures) { + try { + Futures.getDone(future); + } catch (ExecutionException e) { + if (e.getCause() instanceof StatFailedException) { + throw new AbruptExitException( + DetailedExitCode.of( + FailureDetail.newBuilder() + .setMessage(e.getCause().getMessage()) + .setDiffAwareness( + FailureDetails.DiffAwareness.newBuilder().setCode(Code.DIFF_STAT_FAILED)) + .build()), + e); + } + throw new IllegalStateException(e); + } + } + + return new ImmutableDiff(valuesToInvalidate, valuesToInject); + } + + private void processEntry(RootedPath path, NodeVisitState state) throws StatFailedException { + NodeVisitState rootParentSentinel = new NodeVisitState(/*collectMaybeDeletedChildren=*/ false); + + while (state != rootParentSentinel) { + RootedPath parentPath = path.getParentDirectory(); + NodeVisitState parentState = + parentPath != null ? nodeStates.get(parentPath) : rootParentSentinel; + boolean visitParent = + visitEntry(path, state.isInferredDirectory, state.maybeDeletedChildren, parentState); + boolean processParent = parentState.signalFinishedChild(visitParent); + + if (!processParent) { + // This is a tree, only one child can trigger parent processing. + return; + } + + state = parentState; + path = path.getParentDirectory(); + } + } + + /** + * Visits the given node and return whether the type of it may have changed. + * + *

Returns false if we know that the type has not changed. It may however return true if the + * type has not changed. + * + * @param isInferredDirectory whether the node was already inferred as a directory from children. + * @param maybeDeletedChildren if not null, exhaustive list of all children which may have their + * file system type changed (including deletions). + */ + private boolean visitEntry( + RootedPath path, + boolean isInferredDirectory, + @Nullable Set maybeDeletedChildren, + NodeVisitState parentState) + throws StatFailedException { + FileStateValue.Key key = FileStateValue.key(path); + @Nullable FileStateValue fsv = (FileStateValue) graphValues.get(key); + if (fsv == null) { + visitUnknownEntry(key, isInferredDirectory, parentState); + parentState.addMaybeDeletedChild(path.getRootRelativePath().getBaseName()); + return true; + } + + if (isInferredDirectory + || (maybeDeletedChildren != null + && listingHasEntriesOutsideOf(path, maybeDeletedChildren))) { + parentState.markInferredDirectory(); + if (fsv.getType().isDirectory()) { + return false; + } + valuesToInject.put(key, FileStateValue.DIRECTORY_FILE_STATE_NODE); + parentListingKey(path).ifPresent(valuesToInvalidate::add); + return true; + } + + FileStateValue newFsv = getNewFileStateValueFromFileSystem(path); + if (!newFsv.equals(fsv)) { + valuesToInject.put(key, newFsv); + } + + if (newFsv.getType().exists()) { + parentState.markInferredDirectory(); + } else if (fsv.getType().exists()) { + // exists -> not exists -- deletion. + parentState.addMaybeDeletedChild(path.getRootRelativePath().getBaseName()); + } + + boolean typeChanged = newFsv.getType() != fsv.getType(); + if (typeChanged) { + parentListingKey(path).ifPresent(valuesToInvalidate::add); + } + return typeChanged; + } + + private void visitUnknownEntry( + FileStateValue.Key key, boolean isInferredDirectory, NodeVisitState parentState) + throws StatFailedException { + RootedPath path = key.argument(); + // Run stats on unknown files in order to preserve the parent listing if present unless we + // already know it has changed. + Optional parentListingKey = parentListingKey(path); + @Nullable + DirectoryListingStateValue parentListing = + parentListingKey + // Only look for done listings since already invalidated ones will be reevaluated + // anyway. + .map(k -> (DirectoryListingStateValue) graphDoneValues.get(k)) + .orElse(null); + + // No listing/we already know it has changed -- nothing to gain from stats anymore. + if (parentListing == null || valuesToInvalidate.contains(parentListingKey.get())) { + if (isInferredDirectory) { + parentState.markInferredDirectory(); + } + valuesToInvalidate.add(key); + parentListingKey.ifPresent(valuesToInvalidate::add); + return; + } + + // We don't take advantage of isInferredDirectory because we set it only in cases of a present + // descendant/done listing which normally cannot exist without having FileStateValue for + // ancestors. + FileStateValue value = getNewFileStateValueFromFileSystem(path); + valuesToInject.put(key, value); + if (isInferredDirectory || value.getType().exists()) { + parentState.markInferredDirectory(); + } + + @Nullable + Dirent dirent = + parentListing.getDirents().maybeGetDirent(path.getRootRelativePath().getBaseName()); + @Nullable Dirent.Type typeInListing = dirent != null ? dirent.getType() : null; + if (!Objects.equals(typeInListing, direntTypeFromFileStateType(value.getType()))) { + valuesToInvalidate.add(parentListingKey.get()); + } + } + + private FileStateValue getNewFileStateValueFromFileSystem(RootedPath path) + throws StatFailedException { + try { + return FileStateValue.create(path, tsgm); + } catch (IOException e) { + throw new StatFailedException(path, e); + } + } + + private boolean listingHasEntriesOutsideOf(RootedPath path, Set allAffectedEntries) { + // TODO(192010830): Try looking up BUILD files if there is no listing -- this is a lookup we + // can speculatively try since those files are often checked against. + @Nullable + DirectoryListingStateValue listing = + (DirectoryListingStateValue) graphDoneValues.get(DirectoryListingStateValue.key(path)); + if (listing == null) { + return false; + } + for (Dirent entry : listing.getDirents()) { + if (!allAffectedEntries.contains(entry.getName())) { + return true; + } + } + return false; + } + + private static Optional parentListingKey(RootedPath path) { + return Optional.ofNullable(path.getParentDirectory()).map(DirectoryListingStateValue::key); + } + + @Nullable + private static Dirent.Type direntTypeFromFileStateType(FileStateType type) { + switch (type) { + case NONEXISTENT: + return null; + case REGULAR_FILE: + return Dirent.Type.FILE; + case SPECIAL_FILE: + return Dirent.Type.UNKNOWN; + case SYMLINK: + return Dirent.Type.SYMLINK; + case DIRECTORY: + return Dirent.Type.DIRECTORY; + } + throw new AssertionError(); + } + + private static class StatFailedException extends Exception { + StatFailedException(RootedPath path, IOException cause) { + super(String.format("Failed to stat: '%s' while computing diff", path.asPath()), cause); + } + } +} 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 fa3bc530f16053..6114056d85d22a 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 @@ -683,9 +683,9 @@ private void add(SkyKey key, @Nullable SkyValue oldValue, @Nullable SkyValue new concurrentDirtyKeysWithoutNewValues.add(key); } else { if (oldValue == null) { - concurrentDirtyKeysWithNewAndOldValues.put(key, new Delta(newValue)); + concurrentDirtyKeysWithNewAndOldValues.put(key, Delta.create(newValue)); } else { - concurrentDirtyKeysWithNewAndOldValues.put(key, new Delta(oldValue, newValue)); + concurrentDirtyKeysWithNewAndOldValues.put(key, Delta.create(oldValue, newValue)); } } } 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 ab3c6f7624c7a9..6a8e420153bc15 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 @@ -436,7 +436,7 @@ private void handleDiffsWithCompleteDiffInformation( Map modifiedFilesByPathEntry, boolean managedDirectoriesChanged, int fsvcThreads) - throws InterruptedException { + throws InterruptedException, AbruptExitException { for (Root pathEntry : ImmutableSet.copyOf(modifiedFilesByPathEntry.keySet())) { DiffAwarenessManager.ProcessableModifiedFileSet processableModifiedFileSet = modifiedFilesByPathEntry.get(pathEntry); @@ -444,7 +444,7 @@ private void handleDiffsWithCompleteDiffInformation( Preconditions.checkState(!modifiedFileSet.treatEverythingAsModified(), pathEntry); handleChangedFiles( ImmutableList.of(pathEntry), - getDiff(tsgm, modifiedFileSet.modifiedSourceFiles(), pathEntry, fsvcThreads), + getDiff(tsgm, modifiedFileSet, pathEntry, fsvcThreads), /*numSourceFilesCheckedIfDiffWasMissing=*/ 0, managedDirectoriesChanged); processableModifiedFileSet.markProcessed(); @@ -689,7 +689,7 @@ public void clearAnalysisCache( @Override protected void invalidateFilesUnderPathForTestingImpl( ExtendedEventHandler eventHandler, ModifiedFileSet modifiedFileSet, Root pathEntry) - throws InterruptedException { + throws InterruptedException, AbruptExitException { TimestampGranularityMonitor tsgm = this.tsgm.get(); Differencer.Diff diff; if (modifiedFileSet.treatEverythingAsModified()) { @@ -697,8 +697,7 @@ protected void invalidateFilesUnderPathForTestingImpl( new FilesystemValueChecker(tsgm, /*lastExecutionTimeRange=*/ null, /*numThreads=*/ 200) .getDirtyKeys(memoizingEvaluator.getValues(), new BasicFilesystemDirtinessChecker()); } else { - diff = - getDiff(tsgm, modifiedFileSet.modifiedSourceFiles(), pathEntry, /* fsvcThreads= */ 200); + diff = getDiff(tsgm, modifiedFileSet, pathEntry, /* fsvcThreads= */ 200); } syscalls.set(getPerBuildSyscallCache(/*globbingThreads=*/ 42)); recordingDiffer.invalidate(diff.changedKeysWithoutNewValues()); diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java index 74521b1f736f53..ac53b509146aac 100644 --- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java +++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java @@ -1312,22 +1312,31 @@ private static boolean compatibleFileTypes(Dirent.Type oldType, FileStateType ne protected Differencer.Diff getDiff( TimestampGranularityMonitor tsgm, - Collection modifiedSourceFiles, + ModifiedFileSet modifiedFileSet, final Root pathEntry, int fsvcThreads) - throws InterruptedException { - if (modifiedSourceFiles.isEmpty()) { + throws InterruptedException, AbruptExitException { + if (modifiedFileSet.modifiedSourceFiles().isEmpty()) { return new ImmutableDiff(ImmutableList.of(), ImmutableMap.of()); } + // TODO(bazel-team): change ModifiedFileSet to work with RootedPaths instead of PathFragments. Collection dirtyFileStateSkyKeys = Collections2.transform( - modifiedSourceFiles, + modifiedFileSet.modifiedSourceFiles(), pathFragment -> { Preconditions.checkState( !pathFragment.isAbsolute(), "found absolute PathFragment: %s", pathFragment); return FileStateValue.key(RootedPath.toRootedPath(pathEntry, pathFragment)); }); + + Map valuesMap = memoizingEvaluator.getValues(); + + if (!modifiedFileSet.includesAncestorDirectories()) { + return FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + tsgm, valuesMap, memoizingEvaluator.getDoneValues(), dirtyFileStateSkyKeys, fsvcThreads); + } + // We only need to invalidate directory values when a file has been created or deleted or // changes type, not when it has merely been modified. Unfortunately we do not have that // information here, so we compute it ourselves. @@ -1338,7 +1347,6 @@ protected Differencer.Diff getDiff( + "changed"); FilesystemValueChecker fsvc = new FilesystemValueChecker(tsgm, /* lastExecutionTimeRange= */ null, fsvcThreads); - Map valuesMap = memoizingEvaluator.getValues(); Differencer.DiffWithDelta diff = fsvc.getNewAndOldValues(valuesMap, dirtyFileStateSkyKeys, new FileDirtinessChecker()); @@ -1349,8 +1357,8 @@ protected Differencer.Diff getDiff( Preconditions.checkState(key.functionName().equals(FileStateValue.FILE_STATE), key); RootedPath rootedPath = (RootedPath) key.argument(); Delta delta = entry.getValue(); - @Nullable FileStateValue oldValue = (FileStateValue) delta.getOldValue(); - FileStateValue newValue = (FileStateValue) delta.getNewValue(); + @Nullable FileStateValue oldValue = (FileStateValue) delta.oldValue(); + FileStateValue newValue = (FileStateValue) delta.newValue(); valuesToInject.put(key, newValue); SkyKey dirListingStateKey = parentDirectoryListingStateKey(rootedPath); // Invalidate the directory listing for the path's parent directory if the change was @@ -2323,7 +2331,7 @@ public ConfiguredTargetAndData getConfiguredTargetAndDataForTesting( @VisibleForTesting public final void invalidateFilesUnderPathForTesting( ExtendedEventHandler eventHandler, ModifiedFileSet modifiedFileSet, Root pathEntry) - throws InterruptedException { + throws InterruptedException, AbruptExitException { if (lastAnalysisDiscarded) { // Values were cleared last build, but they couldn't be deleted because they were needed for // the execution phase. We can delete them now. @@ -2340,7 +2348,7 @@ public final void turnOffSyscallCacheForTesting() { protected abstract void invalidateFilesUnderPathForTestingImpl( ExtendedEventHandler eventHandler, ModifiedFileSet modifiedFileSet, Root pathEntry) - throws InterruptedException; + throws InterruptedException, AbruptExitException; /** Invalidates SkyFrame values that may have failed for transient reasons. */ public abstract void invalidateTransientErrors(); diff --git a/src/main/java/com/google/devtools/build/lib/testing/common/DirectoryListingHelper.java b/src/main/java/com/google/devtools/build/lib/testing/common/DirectoryListingHelper.java index c5e1ce048a7407..4e0184f4908688 100644 --- a/src/main/java/com/google/devtools/build/lib/testing/common/DirectoryListingHelper.java +++ b/src/main/java/com/google/devtools/build/lib/testing/common/DirectoryListingHelper.java @@ -29,6 +29,11 @@ public static Dirent file(String name) { return new Dirent(name, Dirent.Type.FILE); } + /** Shorthand for {@link Dirent} of {@link Dirent.Type#SYMLINK} type with a given name. */ + public static Dirent symlink(String name) { + return new Dirent(name, Dirent.Type.SYMLINK); + } + /** Shorthand for {@link Dirent} of {@link Dirent.Type#DIRECTORY} type with a given name. */ public static Dirent directory(String name) { return new Dirent(name, Dirent.Type.DIRECTORY); diff --git a/src/main/java/com/google/devtools/build/lib/vfs/ModifiedFileSet.java b/src/main/java/com/google/devtools/build/lib/vfs/ModifiedFileSet.java index 241edf680021e7..68ac77788ddb6c 100644 --- a/src/main/java/com/google/devtools/build/lib/vfs/ModifiedFileSet.java +++ b/src/main/java/com/google/devtools/build/lib/vfs/ModifiedFileSet.java @@ -14,9 +14,7 @@ package com.google.devtools.build.lib.vfs; import com.google.common.collect.ImmutableSet; - import java.util.Objects; - import javax.annotation.Nullable; /** @@ -27,11 +25,14 @@ */ public final class ModifiedFileSet { - public static final ModifiedFileSet EVERYTHING_MODIFIED = new ModifiedFileSet(null); - public static final ModifiedFileSet NOTHING_MODIFIED = new ModifiedFileSet( - ImmutableSet.of()); + // When everything is modified that naturally includes all directories. + public static final ModifiedFileSet EVERYTHING_MODIFIED = + new ModifiedFileSet(null, /*includesAncestorDirectories=*/ true); + public static final ModifiedFileSet NOTHING_MODIFIED = + new ModifiedFileSet(ImmutableSet.of(), /*includesAncestorDirectories=*/ true); @Nullable private final ImmutableSet modified; + private final boolean includesAncestorDirectories; /** * Whether all files of interest should be treated as potentially modified. @@ -52,6 +53,14 @@ public ImmutableSet modifiedSourceFiles() { return modified; } + /** + * Returns whether the diff includes all of affected directories or we need to infer those from + * reported items. + */ + public boolean includesAncestorDirectories() { + return includesAncestorDirectories; + } + @Override public boolean equals(Object o) { if (o == this) { @@ -80,20 +89,29 @@ public String toString() { } } - private ModifiedFileSet(ImmutableSet modified) { + private ModifiedFileSet( + ImmutableSet modified, boolean includesAncestorDirectories) { this.modified = modified; + this.includesAncestorDirectories = includesAncestorDirectories; } - /** - * The builder for {@link ModifiedFileSet}. - */ + /** The builder for {@link ModifiedFileSet}. */ public static class Builder { - private final ImmutableSet.Builder setBuilder = - ImmutableSet.builder(); + private final ImmutableSet.Builder setBuilder = ImmutableSet.builder(); + private boolean includesAncestorDirectories = true; public ModifiedFileSet build() { ImmutableSet modified = setBuilder.build(); - return modified.isEmpty() ? NOTHING_MODIFIED : new ModifiedFileSet(modified); + return modified.isEmpty() + // Special case -- if no files were affected, we know the diff is complete even if + // ancestor directories may not be accounted for. + ? NOTHING_MODIFIED + : new ModifiedFileSet(modified, includesAncestorDirectories); + } + + public Builder setIncludesAncestorDirectories(boolean includesAncestorDirectories) { + this.includesAncestorDirectories = includesAncestorDirectories; + return this; } public Builder modify(PathFragment pathFragment) { @@ -124,6 +142,8 @@ public static ModifiedFileSet union(ModifiedFileSet mfs1, ModifiedFileSet mfs2) return ModifiedFileSet.builder() .modifyAll(mfs1.modifiedSourceFiles()) .modifyAll(mfs2.modifiedSourceFiles()) + .setIncludesAncestorDirectories( + mfs1.includesAncestorDirectories() && mfs2.includesAncestorDirectories()) .build(); } } diff --git a/src/main/java/com/google/devtools/build/skyframe/BUILD b/src/main/java/com/google/devtools/build/skyframe/BUILD index 2c0fd1e1584516..1529523409510b 100644 --- a/src/main/java/com/google/devtools/build/skyframe/BUILD +++ b/src/main/java/com/google/devtools/build/skyframe/BUILD @@ -53,6 +53,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/util:string", "//src/main/java/com/google/devtools/common/options", + "//third_party:auto_value", "//third_party:flogger", "//third_party:guava", "//third_party:jsr305", diff --git a/src/main/java/com/google/devtools/build/skyframe/Differencer.java b/src/main/java/com/google/devtools/build/skyframe/Differencer.java index e66e73ea8e3601..1cd77529f65740 100644 --- a/src/main/java/com/google/devtools/build/skyframe/Differencer.java +++ b/src/main/java/com/google/devtools/build/skyframe/Differencer.java @@ -15,6 +15,7 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.auto.value.AutoValue; import com.google.common.collect.Maps; import java.util.Collection; import java.util.Map; @@ -53,32 +54,25 @@ interface DiffWithDelta extends Diff { Map changedKeysWithNewAndOldValues(); /** Represents the delta between two values of the same key. */ - final class Delta { - @Nullable private final SkyValue oldValue; - private final SkyValue newValue; - - public Delta(SkyValue newValue) { - this(null, newValue); - } - - public Delta(SkyValue oldValue, SkyValue newValue) { - this.oldValue = oldValue; - this.newValue = checkNotNull(newValue); - } - + @AutoValue + abstract class Delta { /** Returns the old value, if any. */ @Nullable - public SkyValue getOldValue() { - return oldValue; - } + public abstract SkyValue oldValue(); /** Returns the new value. */ - public SkyValue getNewValue() { - return newValue; + public abstract SkyValue newValue(); + + public static Delta create(SkyValue newValue) { + return create(/*oldValue=*/ null, newValue); + } + + public static Delta create(SkyValue oldValue, SkyValue newValue) { + return new AutoValue_Differencer_DiffWithDelta_Delta(oldValue, checkNotNull(newValue)); } public static Map newValues(Map delta) { - return Maps.transformValues(delta, Delta::getNewValue); + return Maps.transformValues(delta, Delta::newValue); } } } diff --git a/src/main/protobuf/failure_details.proto b/src/main/protobuf/failure_details.proto index 7d474d24dd3c10..15917665b5f057 100644 --- a/src/main/protobuf/failure_details.proto +++ b/src/main/protobuf/failure_details.proto @@ -146,6 +146,7 @@ message FailureDetail { Toolchain toolchain = 177; StarlarkLoading starlark_loading = 179; ExternalDeps external_deps = 181; + DiffAwareness diff_awareness = 182; } reserved 102; // For internal use @@ -1258,3 +1259,12 @@ message ExternalDeps { Code code = 1; } + +message DiffAwareness { + enum Code { + DIFF_AWARENESS_UNKNOWN = 0 [(metadata) = { exit_code: 37 }]; + DIFF_STAT_FAILED = 1 [(metadata) = { exit_code: 36 }]; + } + + Code code = 1; +} diff --git a/src/test/java/com/google/devtools/build/lib/analysis/util/BUILD b/src/test/java/com/google/devtools/build/lib/analysis/util/BUILD index 957f69336d2d7f..ea58cab31d7c95 100644 --- a/src/test/java/com/google/devtools/build/lib/analysis/util/BUILD +++ b/src/test/java/com/google/devtools/build/lib/analysis/util/BUILD @@ -118,6 +118,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib/skyframe:unloaded_toolchain_context", "//src/main/java/com/google/devtools/build/lib/skyframe/packages:PackageFactoryBuilderWithSkyframeForTesting", "//src/main/java/com/google/devtools/build/lib/util", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/util:crash_failure_details", "//src/main/java/com/google/devtools/build/lib/util:filetype", "//src/main/java/com/google/devtools/build/lib/util:string", diff --git a/src/test/java/com/google/devtools/build/lib/analysis/util/BuildViewTestCase.java b/src/test/java/com/google/devtools/build/lib/analysis/util/BuildViewTestCase.java index 104dd7426e899c..bad9bb1b0f8b77 100644 --- a/src/test/java/com/google/devtools/build/lib/analysis/util/BuildViewTestCase.java +++ b/src/test/java/com/google/devtools/build/lib/analysis/util/BuildViewTestCase.java @@ -165,6 +165,7 @@ import com.google.devtools.build.lib.testutil.FoundationTestCase; import com.google.devtools.build.lib.testutil.SkyframeExecutorTestHelper; import com.google.devtools.build.lib.testutil.TestConstants; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.util.StringUtil; import com.google.devtools.build.lib.util.io.TimestampGranularityMonitor; import com.google.devtools.build.lib.vfs.ModifiedFileSet; @@ -551,7 +552,7 @@ protected PackageManager getPackageManager() { return skyframeExecutor.getPackageManager(); } - protected void invalidatePackages() throws InterruptedException { + protected void invalidatePackages() throws InterruptedException, AbruptExitException { invalidatePackages(true); } @@ -560,7 +561,8 @@ protected void invalidatePackages() throws InterruptedException { * *

Tests should invalidate both unless they have specific reason not to. */ - protected void invalidatePackages(boolean alsoConfigs) throws InterruptedException { + protected void invalidatePackages(boolean alsoConfigs) + throws InterruptedException, AbruptExitException { skyframeExecutor.invalidateFilesUnderPathForTesting( reporter, ModifiedFileSet.EVERYTHING_MODIFIED, Root.fromPath(rootDirectory)); if (alsoConfigs) { diff --git a/src/test/java/com/google/devtools/build/lib/bazel/repository/BUILD b/src/test/java/com/google/devtools/build/lib/bazel/repository/BUILD index e4704f93877dc2..677b9736dfad59 100644 --- a/src/test/java/com/google/devtools/build/lib/bazel/repository/BUILD +++ b/src/test/java/com/google/devtools/build/lib/bazel/repository/BUILD @@ -76,6 +76,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib/bazel/repository", "//src/main/java/com/google/devtools/build/lib/cmdline", "//src/main/java/com/google/devtools/build/lib/util", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/util:os", "//src/test/java/com/google/devtools/build/lib/analysis/util", "//third_party:guava", diff --git a/src/test/java/com/google/devtools/build/lib/bazel/repository/LocalConfigPlatformFunctionTest.java b/src/test/java/com/google/devtools/build/lib/bazel/repository/LocalConfigPlatformFunctionTest.java index a3706762b79362..61dda46ad4585c 100644 --- a/src/test/java/com/google/devtools/build/lib/bazel/repository/LocalConfigPlatformFunctionTest.java +++ b/src/test/java/com/google/devtools/build/lib/bazel/repository/LocalConfigPlatformFunctionTest.java @@ -24,6 +24,7 @@ import com.google.devtools.build.lib.analysis.platform.PlatformProviderUtils; import com.google.devtools.build.lib.analysis.util.BuildViewTestCase; import com.google.devtools.build.lib.cmdline.Label; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.util.CPU; import com.google.devtools.build.lib.util.OS; import java.io.IOException; @@ -120,7 +121,8 @@ public static class FunctionTest extends BuildViewTestCase { ConstraintSettingInfo.create(Label.parseAbsoluteUnchecked("@platforms//os:os")); @Before - public void addLocalConfigPlatform() throws InterruptedException, IOException { + public void addLocalConfigPlatform() + throws InterruptedException, IOException, AbruptExitException { scratch.appendFile("WORKSPACE", "local_config_platform(name='local_config_platform_test')"); invalidatePackages(); } diff --git a/src/test/java/com/google/devtools/build/lib/bazel/repository/starlark/BUILD b/src/test/java/com/google/devtools/build/lib/bazel/repository/starlark/BUILD index 56dad967dd5a77..ac60a64711faef 100644 --- a/src/test/java/com/google/devtools/build/lib/bazel/repository/starlark/BUILD +++ b/src/test/java/com/google/devtools/build/lib/bazel/repository/starlark/BUILD @@ -33,6 +33,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib/skyframe:sky_functions", "//src/main/java/com/google/devtools/build/lib/skyframe:skyframe_cluster", "//src/main/java/com/google/devtools/build/lib/starlarkbuildapi/repository", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment", "//src/main/java/com/google/devtools/build/skyframe", @@ -78,6 +79,7 @@ java_test( "//src/main/java/com/google/devtools/build/lib/skyframe:sky_functions", "//src/main/java/com/google/devtools/build/lib/skyframe:skyframe_cluster", "//src/main/java/com/google/devtools/build/lib/starlarkbuildapi/repository", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment", "//src/main/java/com/google/devtools/build/skyframe", diff --git a/src/test/java/com/google/devtools/build/lib/bazel/repository/starlark/StarlarkRepositoryIntegrationTest.java b/src/test/java/com/google/devtools/build/lib/bazel/repository/starlark/StarlarkRepositoryIntegrationTest.java index 2f1305ba4024d2..8968e41573dc03 100644 --- a/src/test/java/com/google/devtools/build/lib/bazel/repository/starlark/StarlarkRepositoryIntegrationTest.java +++ b/src/test/java/com/google/devtools/build/lib/bazel/repository/starlark/StarlarkRepositoryIntegrationTest.java @@ -37,6 +37,7 @@ import com.google.devtools.build.lib.skyframe.SkyFunctions; import com.google.devtools.build.lib.starlarkbuildapi.repository.RepositoryBootstrap; import com.google.devtools.build.lib.testutil.TestRuleClassProvider; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.skyframe.SkyFunction; import com.google.devtools.build.skyframe.SkyFunctionName; import java.io.OutputStream; @@ -107,7 +108,7 @@ protected ConfiguredRuleClassProvider createRuleClassProvider() { } @Override - protected void invalidatePackages() throws InterruptedException { + protected void invalidatePackages() throws InterruptedException, AbruptExitException { // Repository shuffling breaks access to config-needed paths like //tools/jdk:toolchain and // these tests don't do anything interesting with configurations anyway. So exempt them. invalidatePackages(/*alsoConfigs=*/ false); diff --git a/src/test/java/com/google/devtools/build/lib/packages/BUILD b/src/test/java/com/google/devtools/build/lib/packages/BUILD index f6a0b2d904e078..9f3daae177e582 100644 --- a/src/test/java/com/google/devtools/build/lib/packages/BUILD +++ b/src/test/java/com/google/devtools/build/lib/packages/BUILD @@ -147,6 +147,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib/skyframe:skyframe_cluster", "//src/main/java/com/google/devtools/build/lib/skyframe/packages:PackageFactoryBuilderWithSkyframeForTesting", "//src/main/java/com/google/devtools/build/lib/util", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/util/io", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment", diff --git a/src/test/java/com/google/devtools/build/lib/packages/util/PackageLoadingTestCase.java b/src/test/java/com/google/devtools/build/lib/packages/util/PackageLoadingTestCase.java index d1c651ca8ccf93..e5f48c5f4a52fe 100644 --- a/src/test/java/com/google/devtools/build/lib/packages/util/PackageLoadingTestCase.java +++ b/src/test/java/com/google/devtools/build/lib/packages/util/PackageLoadingTestCase.java @@ -44,6 +44,7 @@ import com.google.devtools.build.lib.testutil.FoundationTestCase; import com.google.devtools.build.lib.testutil.SkyframeExecutorTestHelper; import com.google.devtools.build.lib.testutil.TestRuleClassProvider; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.util.io.TimestampGranularityMonitor; import com.google.devtools.build.lib.vfs.ModifiedFileSet; import com.google.devtools.build.lib.vfs.Root; @@ -316,7 +317,7 @@ protected SkyframeExecutor getSkyframeExecutor() { * does not unconditionally invalidate PackageValue nodes; if no file-system nodes have changed, * packages may not be reloaded. */ - protected void invalidatePackages() throws InterruptedException { + protected void invalidatePackages() throws InterruptedException, AbruptExitException { skyframeExecutor.invalidateFilesUnderPathForTesting( reporter, ModifiedFileSet.EVERYTHING_MODIFIED, Root.fromPath(rootDirectory)); } diff --git a/src/test/java/com/google/devtools/build/lib/pkgcache/BUILD b/src/test/java/com/google/devtools/build/lib/pkgcache/BUILD index 500b99dbbdaaf3..7c37fae8a2f925 100644 --- a/src/test/java/com/google/devtools/build/lib/pkgcache/BUILD +++ b/src/test/java/com/google/devtools/build/lib/pkgcache/BUILD @@ -40,6 +40,7 @@ java_test( "//src/main/java/com/google/devtools/build/lib/packages", "//src/main/java/com/google/devtools/build/lib/pkgcache", "//src/main/java/com/google/devtools/build/lib/util", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment", "//third_party:guava", @@ -165,6 +166,7 @@ java_test( "//src/main/java/com/google/devtools/build/lib/rules:repository/repository_function", "//src/main/java/com/google/devtools/build/lib/skyframe:precomputed_value", "//src/main/java/com/google/devtools/build/lib/skyframe:skyframe_cluster", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/util/io", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/common/options", @@ -194,6 +196,7 @@ java_test( "//src/main/java/com/google/devtools/build/lib/rules:repository/repository_function", "//src/main/java/com/google/devtools/build/lib/skyframe:precomputed_value", "//src/main/java/com/google/devtools/build/lib/skyframe:skyframe_cluster", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/util/io", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/build/lib/vfs/inmemoryfs", @@ -244,6 +247,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib/skyframe:precomputed_value", "//src/main/java/com/google/devtools/build/lib/skyframe:skyframe_cluster", "//src/main/java/com/google/devtools/build/lib/skyframe:target_pattern_phase_value", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/util:detailed_exit_code", "//src/main/java/com/google/devtools/build/lib/util:exit_code", "//src/main/java/com/google/devtools/build/lib/util/io", @@ -272,6 +276,7 @@ java_library( "//src/main/java/com/google/devtools/build/lib/pkgcache:QueryTransitivePackagePreloader", "//src/main/java/com/google/devtools/build/lib/skyframe:skyframe_cluster", "//src/main/java/com/google/devtools/build/lib/skyframe:transitive_target_key", + "//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception", "//src/main/java/com/google/devtools/build/lib/vfs", "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment", "//src/main/java/com/google/devtools/build/lib/vfs/inmemoryfs", diff --git a/src/test/java/com/google/devtools/build/lib/pkgcache/BuildFileModificationTest.java b/src/test/java/com/google/devtools/build/lib/pkgcache/BuildFileModificationTest.java index be70df326357a6..26b4aefd2db0e5 100644 --- a/src/test/java/com/google/devtools/build/lib/pkgcache/BuildFileModificationTest.java +++ b/src/test/java/com/google/devtools/build/lib/pkgcache/BuildFileModificationTest.java @@ -35,6 +35,7 @@ import com.google.devtools.build.lib.testutil.FoundationTestCase; import com.google.devtools.build.lib.testutil.ManualClock; import com.google.devtools.build.lib.testutil.SkyframeExecutorTestHelper; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.util.io.TimestampGranularityMonitor; import com.google.devtools.build.lib.vfs.DigestHashFunction; import com.google.devtools.build.lib.vfs.FileSystem; @@ -133,7 +134,7 @@ protected FileSystem createFileSystem() { return new InMemoryFileSystem(clock, DigestHashFunction.SHA256); } - private void invalidatePackages() throws InterruptedException { + private void invalidatePackages() throws InterruptedException, AbruptExitException { skyframeExecutor.invalidateFilesUnderPathForTesting( reporter, ModifiedFileSet.EVERYTHING_MODIFIED, Root.fromPath(rootDirectory)); } diff --git a/src/test/java/com/google/devtools/build/lib/pkgcache/LoadingPhaseRunnerTest.java b/src/test/java/com/google/devtools/build/lib/pkgcache/LoadingPhaseRunnerTest.java index 8f844f4d04349c..11f87ca00e6334 100644 --- a/src/test/java/com/google/devtools/build/lib/pkgcache/LoadingPhaseRunnerTest.java +++ b/src/test/java/com/google/devtools/build/lib/pkgcache/LoadingPhaseRunnerTest.java @@ -57,6 +57,7 @@ import com.google.devtools.build.lib.testutil.ManualClock; import com.google.devtools.build.lib.testutil.MoreAsserts; import com.google.devtools.build.lib.testutil.SkyframeExecutorTestHelper; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.util.DetailedExitCode; import com.google.devtools.build.lib.util.ExitCode; import com.google.devtools.build.lib.util.io.TimestampGranularityMonitor; @@ -1509,7 +1510,7 @@ public void addFile(String fileName, String... content) throws IOException { FileSystemUtils.writeContentAsLatin1(buildFile, Joiner.on('\n').join(content)); } - private void sync() throws InterruptedException { + private void sync() throws InterruptedException, AbruptExitException { clock.advanceMillis(1); ModifiedFileSet.Builder builder = ModifiedFileSet.builder(); for (Path path : changes) { diff --git a/src/test/java/com/google/devtools/build/lib/pkgcache/PackageLoadingTest.java b/src/test/java/com/google/devtools/build/lib/pkgcache/PackageLoadingTest.java index 3798fa9053d174..92022e2d9ca79d 100644 --- a/src/test/java/com/google/devtools/build/lib/pkgcache/PackageLoadingTest.java +++ b/src/test/java/com/google/devtools/build/lib/pkgcache/PackageLoadingTest.java @@ -43,6 +43,7 @@ import com.google.devtools.build.lib.testutil.FoundationTestCase; import com.google.devtools.build.lib.testutil.MoreAsserts; import com.google.devtools.build.lib.testutil.SkyframeExecutorTestHelper; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.util.io.TimestampGranularityMonitor; import com.google.devtools.build.lib.vfs.ModifiedFileSet; import com.google.devtools.build.lib.vfs.Path; @@ -155,7 +156,7 @@ private PackageManager getPackageManager() { return skyframeExecutor.getPackageManager(); } - private void invalidatePackages() throws InterruptedException { + private void invalidatePackages() throws InterruptedException, AbruptExitException { skyframeExecutor.invalidateFilesUnderPathForTesting( reporter, ModifiedFileSet.EVERYTHING_MODIFIED, Root.fromPath(rootDirectory)); } diff --git a/src/test/java/com/google/devtools/build/lib/pkgcache/QueryPreloadingTestCase.java b/src/test/java/com/google/devtools/build/lib/pkgcache/QueryPreloadingTestCase.java index 53f8216f8e6b46..5c7f4e9e7c8b8e 100644 --- a/src/test/java/com/google/devtools/build/lib/pkgcache/QueryPreloadingTestCase.java +++ b/src/test/java/com/google/devtools/build/lib/pkgcache/QueryPreloadingTestCase.java @@ -26,6 +26,7 @@ import com.google.devtools.build.lib.skyframe.SkyframeExecutor; import com.google.devtools.build.lib.skyframe.TransitiveTargetKey; import com.google.devtools.build.lib.testutil.ManualClock; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.vfs.DigestHashFunction; import com.google.devtools.build.lib.vfs.FileStatus; import com.google.devtools.build.lib.vfs.FileSystem; @@ -162,11 +163,12 @@ protected void assertLabelsAreSubsetOfLabelsVisited( .containsAtLeastElementsIn(asLabelSet(expectedLabels)); } - protected void syncPackages() throws InterruptedException { + protected void syncPackages() throws InterruptedException, AbruptExitException { syncPackages(ModifiedFileSet.EVERYTHING_MODIFIED); } - protected void syncPackages(ModifiedFileSet modifiedFileSet) throws InterruptedException { + protected void syncPackages(ModifiedFileSet modifiedFileSet) + throws InterruptedException, AbruptExitException { getSkyframeExecutor() .invalidateFilesUnderPathForTesting( reporter, modifiedFileSet, Root.fromPath(rootDirectory)); diff --git a/src/test/java/com/google/devtools/build/lib/pkgcache/TargetPatternEvaluatorTest.java b/src/test/java/com/google/devtools/build/lib/pkgcache/TargetPatternEvaluatorTest.java index 6db9d71f740614..1c761fd21ef038 100644 --- a/src/test/java/com/google/devtools/build/lib/pkgcache/TargetPatternEvaluatorTest.java +++ b/src/test/java/com/google/devtools/build/lib/pkgcache/TargetPatternEvaluatorTest.java @@ -27,6 +27,7 @@ import com.google.devtools.build.lib.cmdline.TargetParsingException; import com.google.devtools.build.lib.packages.ImplicitOutputsFunction; import com.google.devtools.build.lib.packages.Target; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.util.Pair; import com.google.devtools.build.lib.vfs.ModifiedFileSet; import com.google.devtools.build.lib.vfs.Path; @@ -122,14 +123,15 @@ public final void createFiles() throws Exception { } } - private void invalidate(String file) throws InterruptedException { + private void invalidate(String file) throws InterruptedException, AbruptExitException { skyframeExecutor.invalidateFilesUnderPathForTesting( reporter, ModifiedFileSet.builder().modify(PathFragment.create(file)).build(), Root.fromPath(rootDirectory)); } - private void invalidate(ModifiedFileSet modifiedFileSet) throws InterruptedException { + private void invalidate(ModifiedFileSet modifiedFileSet) + throws InterruptedException, AbruptExitException { skyframeExecutor.invalidateFilesUnderPathForTesting( reporter, modifiedFileSet, Root.fromPath(rootDirectory)); } diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/FileSystemValueCheckerInferringAncestorsTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/FileSystemValueCheckerInferringAncestorsTest.java new file mode 100644 index 00000000000000..d86ba3d3b8bbdb --- /dev/null +++ b/src/test/java/com/google/devtools/build/lib/skyframe/FileSystemValueCheckerInferringAncestorsTest.java @@ -0,0 +1,722 @@ +// Copyright 2021 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.google.devtools.build.lib.skyframe; + +import static com.google.common.truth.Truth.assertThat; +import static com.google.common.truth.Truth.assertWithMessage; +import static com.google.common.truth.Truth8.assertThat; +import static com.google.devtools.build.lib.actions.FileStateValue.DIRECTORY_FILE_STATE_NODE; +import static com.google.devtools.build.lib.actions.FileStateValue.NONEXISTENT_FILE_STATE_NODE; +import static com.google.devtools.build.lib.testing.common.DirectoryListingHelper.file; +import static com.google.devtools.build.lib.testing.common.DirectoryListingHelper.symlink; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.fail; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; +import com.google.common.collect.Streams; +import com.google.devtools.build.lib.actions.FileStateValue; +import com.google.devtools.build.lib.server.FailureDetails.DiffAwareness.Code; +import com.google.devtools.build.lib.testutil.Scratch; +import com.google.devtools.build.lib.util.AbruptExitException; +import com.google.devtools.build.lib.vfs.DelegateFileSystem; +import com.google.devtools.build.lib.vfs.Dirent; +import com.google.devtools.build.lib.vfs.FileStatus; +import com.google.devtools.build.lib.vfs.FileSystem; +import com.google.devtools.build.lib.vfs.Path; +import com.google.devtools.build.lib.vfs.PathFragment; +import com.google.devtools.build.lib.vfs.Root; +import com.google.devtools.build.lib.vfs.RootedPath; +import com.google.devtools.build.skyframe.ImmutableDiff; +import com.google.devtools.build.skyframe.SkyKey; +import com.google.devtools.build.skyframe.SkyValue; +import com.google.testing.junit.testparameterinjector.TestParameter; +import com.google.testing.junit.testparameterinjector.TestParameterInjector; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import javax.annotation.Nullable; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; + +/** Unit tests for {@link FileSystemValueCheckerInferringAncestors}. */ +@RunWith(TestParameterInjector.class) +public final class FileSystemValueCheckerInferringAncestorsTest { + + private final Scratch scratch = new Scratch(); + private final List statedPaths = new ArrayList<>(); + private Root root; + private Root untrackedRoot; + private Exception throwOnStat; + + @TestParameter({"1", "16"}) + private int fsvcThreads; + + @Before + public void createRoot() throws IOException { + Path srcRootPath = scratch.dir("/src"); + PathFragment srcRoot = srcRootPath.asFragment(); + FileSystem trackingFileSystem = + new DelegateFileSystem(scratch.getFileSystem()) { + @Nullable + @Override + public synchronized FileStatus statIfFound(PathFragment path, boolean followSymlinks) + throws IOException { + if (throwOnStat != null) { + Exception toThrow = throwOnStat; + throwOnStat = null; + Throwables.propagateIfPossible(toThrow, IOException.class); + fail("Unexpected exception type"); + } + statedPaths.add(path.relativeTo(srcRoot).toString()); + return super.statIfFound(path, followSymlinks); + } + }; + root = Root.fromPath(trackingFileSystem.getPath(srcRoot)); + scratch.setWorkingDir("/src"); + untrackedRoot = Root.fromPath(srcRootPath); + } + + @After + public void checkExceptionThrown() { + assertThat(throwOnStat).isNull(); + } + + @Test + public void getDiffWithInferredAncestors_unknownFileChanged_returnsFileAndDirs() + throws Exception { + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of(), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(fileStateValueKey("foo/file")), + fsvcThreads); + + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly( + fileStateValueKey(""), + fileStateValueKey("foo"), + fileStateValueKey("foo/file"), + directoryListingStateValueKey(""), + directoryListingStateValueKey("foo")); + assertThat(diff.changedKeysWithNewValues()).isEmpty(); + assertThat(statedPaths).isEmpty(); + } + + @Test + public void getDiffWithInferredAncestors_fileModified_returnsFileWithValues() throws Exception { + scratch.file("file", "hello"); + FileStateValue.Key key = fileStateValueKey("file"); + FileStateValue value = fileStateValue("file"); + scratch.overwriteFile("file", "there"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of(fileStateValueKey("file"), value), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(key), + fsvcThreads); + + FileStateValue newValue = fileStateValue("file"); + assertThat(diff.changedKeysWithNewValues()).containsExactly(key, newValue); + assertThat(diff.changedKeysWithoutNewValues()).isEmpty(); + assertThat(statedPaths).containsExactly("file"); + } + + @Test + public void getDiffWithInferredAncestors_fileAdded_returnsFileAndDirListing() throws Exception { + scratch.file("file"); + FileStateValue.Key key = fileStateValueKey("file"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + key, NONEXISTENT_FILE_STATE_NODE, fileStateValueKey(""), fileStateValue("")), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(key), + fsvcThreads); + + FileStateValue value = fileStateValue("file"); + assertThat(diff.changedKeysWithNewValues()).containsExactly(key, value); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly(directoryListingStateValueKey("")); + assertThat(statedPaths).containsExactly("file"); + } + + @Test + public void getDiffWithInferredAncestors_fileWithDirsAdded_returnsFileAndInjectsDirs() + throws Exception { + scratch.file("a/b/file"); + FileStateValue.Key fileKey = fileStateValueKey("a/b/file"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + fileStateValueKey(""), + fileStateValue(""), + fileStateValueKey("a"), + NONEXISTENT_FILE_STATE_NODE, + fileStateValueKey("a/b"), + NONEXISTENT_FILE_STATE_NODE, + fileKey, + NONEXISTENT_FILE_STATE_NODE), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(fileKey), + fsvcThreads); + + FileStateValue value = fileStateValue("a/b/file"); + assertThat(diff.changedKeysWithNewValues()) + .containsExactly( + fileKey, + value, + fileStateValueKey("a"), + DIRECTORY_FILE_STATE_NODE, + fileStateValueKey("a/b"), + DIRECTORY_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly( + directoryListingStateValueKey(""), + directoryListingStateValueKey("a"), + directoryListingStateValueKey("a/b")); + assertThat(statedPaths).containsExactly("a/b/file"); + } + + @Test + public void getDiffWithInferredAncestors_addedFileWithReportedDirs_returnsFileAndInjectsDirs() + throws Exception { + scratch.file("a/b/file"); + FileStateValue.Key fileKey = fileStateValueKey("a/b/file"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + fileStateValueKey(""), + fileStateValue(""), + fileStateValueKey("a"), + NONEXISTENT_FILE_STATE_NODE, + fileStateValueKey("a/b"), + NONEXISTENT_FILE_STATE_NODE, + fileKey, + NONEXISTENT_FILE_STATE_NODE), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(fileKey, fileStateValueKey("a")), + fsvcThreads); + + FileStateValue newState = fileStateValue("a/b/file"); + assertThat(diff.changedKeysWithNewValues()) + .containsExactly( + fileKey, + newState, + fileStateValueKey("a"), + DIRECTORY_FILE_STATE_NODE, + fileStateValueKey("a/b"), + DIRECTORY_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly( + directoryListingStateValueKey(""), + directoryListingStateValueKey("a"), + directoryListingStateValueKey("a/b")); + assertThat(statedPaths).containsExactly("a/b/file"); + } + + /** + * This is a degenerate case since we normally only know about a file if we checked all parents, + * but that is theoretically possible with this API. + */ + @Test + public void getDiffWithInferredAncestors_fileWithUnknownDirsAdded_returnsFileAndDirs() + throws Exception { + scratch.file("a/b/c/d"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + fileStateValueKey(""), + fileStateValue(""), + fileStateValueKey("a/b/c/d"), + NONEXISTENT_FILE_STATE_NODE), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(fileStateValueKey("a/b/c/d")), + fsvcThreads); + + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly( + fileStateValueKey("a"), + fileStateValueKey("a/b"), + fileStateValueKey("a/b/c"), + directoryListingStateValueKey(""), + directoryListingStateValueKey("a"), + directoryListingStateValueKey("a/b"), + directoryListingStateValueKey("a/b/c")); + assertThat(diff.changedKeysWithNewValues()) + .containsExactly(fileStateValueKey("a/b/c/d"), fileStateValue("a/b/c/d")); + assertThat(statedPaths).containsExactly("a/b/c/d"); + } + + @Test + public void getDiffWithInferredAncestors_addEmptyDir_returnsDirAndParentListing() + throws Exception { + scratch.dir("dir"); + FileStateValue.Key key = fileStateValueKey("dir"); + FileStateValue value = fileStateValue("dir"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + key, NONEXISTENT_FILE_STATE_NODE, fileStateValueKey(""), fileStateValue("")), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(key), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()).containsExactly(key, value); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly(directoryListingStateValueKey("")); + assertThat(statedPaths).containsExactly("dir"); + } + + @Test + public void getDiffWithInferredAncestors_deleteFile_returnsFileParentListing() throws Exception { + Path file = scratch.file("dir/file1"); + scratch.file("dir/file2"); + FileStateValue.Key key = fileStateValueKey("dir/file1"); + FileStateValue oldValue = fileStateValue("dir/file1"); + file.delete(); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + key, oldValue, fileStateValueKey("dir"), fileStateValue("dir")), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(key), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()).containsExactly(key, NONEXISTENT_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly(directoryListingStateValueKey("dir")); + assertThat(statedPaths).containsExactly("dir/file1", "dir"); + } + + @Test + public void getDiffWithInferredAncestors_deleteFileFromDirWithListing_skipsDirStat() + throws Exception { + Path file1 = scratch.file("dir/file1"); + FileStateValue.Key key = fileStateValueKey("dir/file1"); + FileStateValue oldValue = fileStateValue("dir/file1"); + file1.delete(); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + key, oldValue, fileStateValueKey("dir"), fileStateValue("dir")), + /*graphDoneValues=*/ ImmutableMap.of( + directoryListingStateValueKey("dir"), + directoryListingStateValue(file("file1"), file("file2"))), + /*modifiedKeys=*/ ImmutableSet.of(key), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()).containsExactly(key, NONEXISTENT_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly(directoryListingStateValueKey("dir")); + assertThat(statedPaths).containsExactly("dir/file1"); + } + + @Test + public void getDiffWithInferredAncestors_deleteLastFileFromDir_ignoresInvalidatedListing() + throws Exception { + Path file1 = scratch.file("dir/file1"); + FileStateValue.Key key = fileStateValueKey("dir/file1"); + FileStateValue oldValue = fileStateValue("dir/file1"); + file1.delete(); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + key, + oldValue, + fileStateValueKey("dir"), + fileStateValue("dir"), + directoryListingStateValueKey("dir"), + directoryListingStateValue(file("file1"), file("file2"))), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(key), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()).containsExactly(key, NONEXISTENT_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly(directoryListingStateValueKey("dir")); + assertThat(statedPaths).containsExactly("dir/file1", "dir"); + } + + @Test + public void getDiffWithInferredAncestors_modifyAllUnknownEntriesInDirWithListing_skipsDir() + throws Exception { + Path file = scratch.file("dir/file"); + file.getParentDirectory() + .getRelative("symlink") + .createSymbolicLink(PathFragment.create("file")); + FileStateValue.Key fileKey = fileStateValueKey("dir/file"); + FileStateValue.Key symlinkKey = fileStateValueKey("dir/symlink"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of(fileStateValueKey("dir"), fileStateValue("dir")), + /*graphDoneValues=*/ ImmutableMap.of( + directoryListingStateValueKey("dir"), + directoryListingStateValue(file("file"), symlink("symlink"))), + /*modifiedKeys=*/ ImmutableSet.of(fileKey, symlinkKey), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()) + .containsExactly( + fileKey, fileStateValue("dir/file"), symlinkKey, fileStateValue("dir/symlink")); + assertThat(diff.changedKeysWithoutNewValues()).isEmpty(); + assertThat(statedPaths).containsExactly("dir/file", "dir/symlink"); + } + + @Test + public void getDiffWithInferredAncestors_replaceUnknownEntriesInDirWithListing_skipsSiblingStat() + throws Exception { + scratch.dir("dir/file1"); + scratch.dir("dir/file2"); + FileStateValue.Key file1Key = fileStateValueKey("dir/file1"); + FileStateValue.Key file2Key = fileStateValueKey("dir/file2"); + DirectoryListingStateValue.Key dirKey = directoryListingStateValueKey("dir"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of(fileStateValueKey("dir"), fileStateValue("dir")), + /*graphDoneValues=*/ ImmutableMap.of( + dirKey, directoryListingStateValue(file("file1"), file("file2"))), + /*modifiedKeys=*/ ImmutableSet.of(file1Key, file2Key), + fsvcThreads); + + assertIsSubsetOf( + diff.changedKeysWithNewValues().entrySet(), + Maps.immutableEntry(file1Key, fileStateValue("dir/file1")), + Maps.immutableEntry(file2Key, fileStateValue("dir/file2"))); + assertThat(diff.changedKeysWithoutNewValues()).contains(dirKey); + assertThat( + Streams.concat( + diff.changedKeysWithoutNewValues().stream(), + diff.changedKeysWithNewValues().keySet().stream())) + .containsExactly(file1Key, file2Key, dirKey); + assertThat(statedPaths).isNotEmpty(); + assertIsSubsetOf(statedPaths, "dir/file1", "dir/file2"); + if (fsvcThreads == 1) { + // In case of single-threaded execution, we know that once we check dir/file1 or dir/file2, we + // will be able to skip stat on the other one. + assertThat(diff.changedKeysWithNewValues()).hasSize(1); + assertThat(diff.changedKeysWithoutNewValues()).hasSize(2); + assertThat(statedPaths).hasSize(1); + } + } + + @Test + public void getDiffWithInferredAncestors_deleteAllFilesFromDir_returnsFilesAndDirListing() + throws Exception { + Path file1 = scratch.file("dir/file1"); + Path file2 = scratch.file("dir/file2"); + Path file3 = scratch.file("dir/file3"); + FileStateValue.Key key1 = fileStateValueKey("dir/file1"); + FileStateValue oldValue1 = fileStateValue("dir/file1"); + FileStateValue.Key key2 = fileStateValueKey("dir/file2"); + FileStateValue oldValue2 = fileStateValue("dir/file2"); + FileStateValue.Key key3 = fileStateValueKey("dir/file3"); + FileStateValue oldValue3 = fileStateValue("dir/file3"); + file1.delete(); + file2.delete(); + file3.delete(); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + key1, + oldValue1, + key2, + oldValue2, + key3, + oldValue3, + fileStateValueKey("dir"), + fileStateValue("dir")), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(key1, key2, key3), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()) + .containsExactly( + key1, NONEXISTENT_FILE_STATE_NODE, + key2, NONEXISTENT_FILE_STATE_NODE, + key3, NONEXISTENT_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly(directoryListingStateValueKey("dir")); + assertThat(statedPaths).containsExactly("dir", "dir/file1", "dir/file2", "dir/file3"); + } + + @Test + public void getDiffWithInferredAncestors_deleteFileWithDirs_returnsFileAndDirs() + throws Exception { + scratch.file("a/b/c/file"); + FileStateValue.Key abKey = fileStateValueKey("a/b"); + FileStateValue abValue = fileStateValue("a/b"); + FileStateValue.Key abcKey = fileStateValueKey("a/b/c"); + FileStateValue abcValue = fileStateValue("a/b/c"); + FileStateValue.Key abcFileKey = fileStateValueKey("a/b/c/file"); + FileStateValue abcFileValue = fileStateValue("a/b/c/file"); + scratch.dir("a/b").deleteTree(); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + fileStateValueKey("a"), + fileStateValue("a"), + abKey, + abValue, + abcKey, + abcValue, + abcFileKey, + abcFileValue), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(abcFileKey), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()) + .containsExactly( + abKey, NONEXISTENT_FILE_STATE_NODE, + abcKey, NONEXISTENT_FILE_STATE_NODE, + abcFileKey, NONEXISTENT_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly( + directoryListingStateValueKey("a"), + directoryListingStateValueKey("a/b"), + directoryListingStateValueKey("a/b/c")); + assertThat(statedPaths).containsExactly("a", "a/b", "a/b/c", "a/b/c/file"); + } + + @Test + public void getDiffWithInferredAncestors_deleteFileWithReportedDirs_returnsFileAndDirListings() + throws Exception { + scratch.file("a/b/c/file"); + FileStateValue.Key abKey = fileStateValueKey("a/b"); + FileStateValue abValue = fileStateValue("a/b"); + FileStateValue.Key abcKey = fileStateValueKey("a/b/c"); + FileStateValue abcValue = fileStateValue("a/b/c"); + FileStateValue.Key abcFileKey = fileStateValueKey("a/b/c/file"); + FileStateValue abcFileValue = fileStateValue("a/b/c/file"); + scratch.dir("a/b").deleteTree(); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + fileStateValueKey("a"), + fileStateValue("a"), + abKey, + abValue, + abcKey, + abcValue, + abcFileKey, + abcFileValue), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(abcFileKey, abKey), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()) + .containsExactly( + abKey, NONEXISTENT_FILE_STATE_NODE, + abcKey, NONEXISTENT_FILE_STATE_NODE, + abcFileKey, NONEXISTENT_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly( + directoryListingStateValueKey("a"), + directoryListingStateValueKey("a/b"), + directoryListingStateValueKey("a/b/c")); + assertThat(statedPaths).containsExactly("a", "a/b", "a/b/c", "a/b/c/file"); + } + + @Test + public void getDiffWithInferredAncestors_deleteFile_infersDirFromModifiedSibling() + throws Exception { + Path file1 = scratch.file("dir/file1"); + scratch.file("dir/file2", "1"); + FileStateValue.Key file1Key = fileStateValueKey("dir/file1"); + FileStateValue file1Value = fileStateValue("dir/file1"); + FileStateValue.Key file2Key = fileStateValueKey("dir/file2"); + FileStateValue file2Value = fileStateValue("dir/file2"); + file1.delete(); + scratch.overwriteFile("dir/file2", "12"); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + fileStateValueKey("dir"), + fileStateValue("dir"), + file1Key, + file1Value, + file2Key, + file2Value), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(file1Key, file2Key, fileStateValueKey("dir")), + fsvcThreads); + + FileStateValue file2NewValue = fileStateValue("dir/file2"); + assertThat(diff.changedKeysWithNewValues()) + .containsExactly(file1Key, NONEXISTENT_FILE_STATE_NODE, file2Key, file2NewValue); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly(directoryListingStateValueKey("dir")); + assertThat(statedPaths).containsExactly("dir/file1", "dir/file2"); + } + + @Test + public void getDiffWithInferredAncestors_deleteDirReportDirOnly_returnsDir() throws Exception { + Path file1 = scratch.file("dir/file1"); + scratch.file("dir/file2"); + FileStateValue.Key file1Key = fileStateValueKey("dir/file1"); + FileStateValue file1Value = fileStateValue("dir/file1"); + FileStateValue.Key file2Key = fileStateValueKey("dir/file2"); + FileStateValue file2Value = fileStateValue("dir/file2"); + FileStateValue.Key dirKey = fileStateValueKey("dir"); + FileStateValue dirValue = fileStateValue("dir"); + file1.getParentDirectory().deleteTree(); + + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + file1Key, + file1Value, + file2Key, + file2Value, + dirKey, + dirValue, + fileStateValueKey(""), + fileStateValue("")), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(dirKey), + fsvcThreads); + + assertThat(diff.changedKeysWithNewValues()) + .containsExactly(dirKey, NONEXISTENT_FILE_STATE_NODE); + assertThat(diff.changedKeysWithoutNewValues()) + .containsExactly(directoryListingStateValueKey("")); + assertThat(statedPaths).containsExactly("dir", ""); + } + + @Test + public void getDiffWithInferredAncestors_phantomChangeForNonexistentEntry_returnsEmptyDiff() + throws Exception { + ImmutableDiff diff = + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + /*graphValues=*/ ImmutableMap.of( + fileStateValueKey("file"), NONEXISTENT_FILE_STATE_NODE), + /*graphDoneValues=*/ ImmutableMap.of(), + /*modifiedKeys=*/ ImmutableSet.of(fileStateValueKey("file")), + fsvcThreads); + + assertThat(diff.changedKeysWithoutNewValues()).isEmpty(); + assertThat(diff.changedKeysWithNewValues()).isEmpty(); + assertThat(statedPaths).containsExactly("file"); + } + + @Test + public void getDiffWithInferredAncestors_statFails_fails() { + throwOnStat = new IOException("oh no"); + ImmutableMap graphValues = + ImmutableMap.of(fileStateValueKey("file"), NONEXISTENT_FILE_STATE_NODE); + ImmutableSet modifiedKeys = ImmutableSet.of(fileStateValueKey("file")); + + AbruptExitException e = + assertThrows( + AbruptExitException.class, + () -> + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + graphValues, + /*graphDoneValues=*/ ImmutableMap.of(), + modifiedKeys, + fsvcThreads)); + + assertThat(e.getDetailedExitCode().getFailureDetail().hasDiffAwareness()).isTrue(); + assertThat(e.getDetailedExitCode().getFailureDetail().getDiffAwareness().getCode()) + .isEqualTo(Code.DIFF_STAT_FAILED); + assertThat(e).hasMessageThat().isEqualTo("Failed to stat: '/src/file' while computing diff"); + } + + @Test + public void getDiffWithInferredAncestors_statCrashes_fails() { + throwOnStat = new RuntimeException("oh no"); + ImmutableMap graphValues = + ImmutableMap.of(fileStateValueKey("file"), NONEXISTENT_FILE_STATE_NODE); + ImmutableSet modifiedKeys = ImmutableSet.of(fileStateValueKey("file")); + + assertThrows( + IllegalStateException.class, + () -> + FileSystemValueCheckerInferringAncestors.getDiffWithInferredAncestors( + /*tsgm=*/ null, + graphValues, + /*graphDoneValues=*/ ImmutableMap.of(), + modifiedKeys, + fsvcThreads)); + } + + private static void assertIsSubsetOf(Iterable list, T... elements) { + ImmutableSet set = ImmutableSet.copyOf(elements); + assertWithMessage("%s has elements from outside of %s", list, set) + .that(set) + .containsAtLeastElementsIn(list); + } + + private FileStateValue.Key fileStateValueKey(String relativePath) { + return FileStateValue.key( + RootedPath.toRootedPath(root, root.asPath().getRelative(relativePath))); + } + + private DirectoryListingStateValue.Key directoryListingStateValueKey(String relativePath) { + return DirectoryListingStateValue.key( + RootedPath.toRootedPath(root, root.asPath().getRelative(relativePath))); + } + + private static DirectoryListingStateValue directoryListingStateValue(Dirent... dirents) { + return DirectoryListingStateValue.create(ImmutableList.copyOf(dirents)); + } + + private FileStateValue fileStateValue(String relativePath) throws IOException { + return FileStateValue.create( + RootedPath.toRootedPath( + untrackedRoot, untrackedRoot.asPath().asFragment().getRelative(relativePath)), + /*tsgm=*/ null); + } +} diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/RepositoryMappingFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/RepositoryMappingFunctionTest.java index 05b51306909acd..1f65555f09e262 100644 --- a/src/test/java/com/google/devtools/build/lib/skyframe/RepositoryMappingFunctionTest.java +++ b/src/test/java/com/google/devtools/build/lib/skyframe/RepositoryMappingFunctionTest.java @@ -30,6 +30,7 @@ import com.google.devtools.build.lib.packages.NoSuchPackageException; import com.google.devtools.build.lib.skyframe.util.SkyframeExecutorTestUtils; import com.google.devtools.build.lib.testutil.TestConstants; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.vfs.ModifiedFileSet; import com.google.devtools.build.lib.vfs.PathFragment; import com.google.devtools.build.lib.vfs.Root; @@ -46,7 +47,8 @@ public class RepositoryMappingFunctionTest extends BuildViewTestCase { private FakeRegistry registry; - private EvaluationResult eval(SkyKey key) throws InterruptedException { + private EvaluationResult eval(SkyKey key) + throws InterruptedException, AbruptExitException { getSkyframeExecutor() .invalidateFilesUnderPathForTesting( reporter, 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 8c7dad2f8331c0..178140fd01436b 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 @@ -318,7 +318,7 @@ public void getDiff_changedFileStillExists_returnsFile() throws Exception { Diff diff = skyframeExecutor.getDiff( /*tsgm=*/ null, - ImmutableList.of(PathFragment.create("foo/foo.txt")), + ModifiedFileSet.builder().modify(PathFragment.create("foo/foo.txt")).build(), root, /*fsvcThreads=*/ 1); @@ -335,7 +335,7 @@ public void getDiff_newFile_returnsFileAndParentDirectoryListing() throws Except Diff diff = skyframeExecutor.getDiff( /*tsgm=*/ null, - ImmutableList.of(PathFragment.create("foo/foo.txt")), + ModifiedFileSet.builder().modify(PathFragment.create("foo/foo.txt")).build(), root, /*fsvcThreads=*/ 1); @@ -358,7 +358,7 @@ public void getDiff_newFileFailsToStat_returnsFileAndParentDirectoryListing() th Diff diff = skyframeExecutor.getDiff( /*tsgm=*/ null, - ImmutableList.of(PathFragment.create("foo/foo.txt")), + ModifiedFileSet.builder().modify(PathFragment.create("foo/foo.txt")).build(), root, /*fsvcThreads=*/ 1); diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/WorkspaceFileFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/WorkspaceFileFunctionTest.java index 5bacda728dae96..534ff7262832d8 100644 --- a/src/test/java/com/google/devtools/build/lib/skyframe/WorkspaceFileFunctionTest.java +++ b/src/test/java/com/google/devtools/build/lib/skyframe/WorkspaceFileFunctionTest.java @@ -36,6 +36,7 @@ import com.google.devtools.build.lib.rules.repository.ManagedDirectoriesKnowledgeImpl; import com.google.devtools.build.lib.rules.repository.ManagedDirectoriesKnowledgeImpl.ManagedDirectoriesListener; import com.google.devtools.build.lib.skyframe.util.SkyframeExecutorTestUtils; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.vfs.ModifiedFileSet; import com.google.devtools.build.lib.vfs.Path; import com.google.devtools.build.lib.vfs.PathFragment; @@ -111,7 +112,8 @@ public boolean matches(Object item) { public void describeTo(Description description) {} } - private EvaluationResult eval(SkyKey key) throws InterruptedException { + private EvaluationResult eval(SkyKey key) + throws InterruptedException, AbruptExitException { getSkyframeExecutor() .invalidateFilesUnderPathForTesting( reporter, @@ -349,7 +351,7 @@ private void parseWorkspaceFileValueWithError(String expectedError, String... li } private WorkspaceFileValue parseWorkspaceFileValueImpl(String[] lines) - throws IOException, InterruptedException { + throws IOException, InterruptedException, AbruptExitException { RootedPath workspaceFile = createWorkspaceFile(lines); WorkspaceFileKey key = WorkspaceFileValue.key(workspaceFile); EvaluationResult result = eval(key); diff --git a/src/test/java/com/google/devtools/build/lib/skyframe/WorkspaceNameFunctionTest.java b/src/test/java/com/google/devtools/build/lib/skyframe/WorkspaceNameFunctionTest.java index f04433b881ddab..f4c60aea8c7a5c 100644 --- a/src/test/java/com/google/devtools/build/lib/skyframe/WorkspaceNameFunctionTest.java +++ b/src/test/java/com/google/devtools/build/lib/skyframe/WorkspaceNameFunctionTest.java @@ -19,6 +19,7 @@ import com.google.devtools.build.lib.analysis.util.BuildViewTestCase; import com.google.devtools.build.lib.packages.NoSuchPackageException; import com.google.devtools.build.lib.skyframe.util.SkyframeExecutorTestUtils; +import com.google.devtools.build.lib.util.AbruptExitException; import com.google.devtools.build.lib.vfs.FileSystemUtils; import com.google.devtools.build.lib.vfs.ModifiedFileSet; import com.google.devtools.build.lib.vfs.PathFragment; @@ -34,7 +35,8 @@ public class WorkspaceNameFunctionTest extends BuildViewTestCase { private final SkyKey key = WorkspaceNameValue.key(); - private EvaluationResult eval() throws InterruptedException { + private EvaluationResult eval() + throws InterruptedException, AbruptExitException { getSkyframeExecutor() .invalidateFilesUnderPathForTesting( reporter, diff --git a/src/test/java/com/google/devtools/build/lib/vfs/ModifiedFileSetTest.java b/src/test/java/com/google/devtools/build/lib/vfs/ModifiedFileSetTest.java index f3c5eb63e7520f..5f42deb53b4311 100644 --- a/src/test/java/com/google/devtools/build/lib/vfs/ModifiedFileSetTest.java +++ b/src/test/java/com/google/devtools/build/lib/vfs/ModifiedFileSetTest.java @@ -13,17 +13,17 @@ // limitations under the License. package com.google.devtools.build.lib.vfs; +import static com.google.common.truth.Truth.assertThat; + import com.google.common.collect.ImmutableList; import com.google.common.testing.EqualsTester; - +import com.google.testing.junit.testparameterinjector.TestParameterInjector; +import com.google.testing.junit.testparameterinjector.TestParameters; import org.junit.Test; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -/** - * Tests for {@link ModifiedFileSet}. - */ -@RunWith(JUnit4.class) +/** Tests for {@link ModifiedFileSet}. */ +@RunWith(TestParameterInjector.class) public class ModifiedFileSetTest { @Test @@ -51,4 +51,31 @@ public void testHashCodeAndEqualsContract() throws Exception { .addEqualityGroup(everythingModified) .testEquals(); } + + @TestParameters({ + "{set1: false, set2: false, union: false}", + "{set1: true, set2: false, union: false}", + "{set1: false, set2: true, union: false}", + "{set1: true, set2: true, union: true}" + }) + @Test + public void union_returnsConjunctionOfIncludesAncestorDirectories( + boolean set1, boolean set2, boolean union) { + ModifiedFileSet fileSet1 = + ModifiedFileSet.builder() + .modify(PathFragment.create("a")) + .setIncludesAncestorDirectories(set1) + .build(); + ModifiedFileSet fileSet2 = + ModifiedFileSet.builder() + .modify(PathFragment.create("b")) + .setIncludesAncestorDirectories(set2) + .build(); + + ModifiedFileSet unionSet = ModifiedFileSet.union(fileSet1, fileSet2); + + assertThat(unionSet.modifiedSourceFiles()) + .containsExactly(PathFragment.create("a"), PathFragment.create("b")); + assertThat(unionSet.includesAncestorDirectories()).isEqualTo(union); + } }