Make files and directories under managed directories be correctly processed by SequencedSkyframeExecutor.

TL;DR - two main changes: how to invalidate changed managed directories files, and how to force owning external repository to be evaluated before managed directories files.

- In ExternalFilesHelper, introduce one more file type - EXTERNAL_REPO_IN_USER_DIRECTORY, for the files under managed directories.
- For files under managed directories, require owning RepositoryDirectoryValue to be evaluated first.
- For correct dirtying of files under managed directories, both with watchfs flag and without, the following should be taken into account: not only that new values for external repositories and managed directories files can not be injected at the stage of dirtying, but also files that used to be under managed directories on the previous evaluator invocation.
The latter are still cached in evaluator, and the fact that they used to depend on their RepositoryDirectory values would prevent injection of the new values for them.
To meet those conditions, in SequencedSkyframeExecutor.handleChangedFiles() filtering of the going-to-be-injected files is added.
(The change can not be done inside ExternalDirtinessChecker only, as then it does not affect watchfs=true case).
- ManagedDirectoriesBlackBoxTest added to demonstrate and validate the functionality of managed directories.

PiperOrigin-RevId: 246496823
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/DirectoryListingStateFunction.java b/src/main/java/com/google/devtools/build/lib/skyframe/DirectoryListingStateFunction.java
index c3d3ce9..8e8b5bb 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/DirectoryListingStateFunction.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/DirectoryListingStateFunction.java
@@ -13,6 +13,7 @@
 // limitations under the License.
 package com.google.devtools.build.lib.skyframe;
 
+import com.google.devtools.build.lib.skyframe.ExternalFilesHelper.FileType;
 import com.google.devtools.build.lib.vfs.RootedPath;
 import com.google.devtools.build.lib.vfs.Symlinks;
 import com.google.devtools.build.lib.vfs.UnixGlob.FilesystemCalls;
@@ -51,10 +52,17 @@
     RootedPath dirRootedPath = (RootedPath) skyKey.argument();
 
     try {
-      externalFilesHelper.maybeHandleExternalFile(dirRootedPath, true, env);
+      FileType fileType = externalFilesHelper.maybeHandleExternalFile(dirRootedPath, true, env);
       if (env.valuesMissing()) {
         return null;
       }
+      if (fileType == FileType.EXTERNAL_REPO
+          || fileType == FileType.EXTERNAL_IN_MANAGED_DIRECTORY) {
+        // Do not use syscallCache as files under repositories get generated during the build,
+        // while syscallCache is used independently from Skyframe and generally assumes
+        // the file system is frozen at the beginning of the build command.
+        return DirectoryListingStateValue.create(dirRootedPath);
+      }
       return DirectoryListingStateValue.create(
           syscallCache.get().readdir(dirRootedPath.asPath(), Symlinks.NOFOLLOW));
     } catch (ExternalFilesHelper.NonexistentImmutableExternalFileException e) {
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/DirtinessCheckerUtils.java b/src/main/java/com/google/devtools/build/lib/skyframe/DirtinessCheckerUtils.java
index ce4de19..860f437 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/DirtinessCheckerUtils.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/DirtinessCheckerUtils.java
@@ -106,7 +106,11 @@
     }
   }
 
-  /** Checks files outside of the package roots for changes. */
+  /**
+   * Serves for tracking whether there are external and output files {@see ExternalFilesKnowledge}.
+   * Filtering of files, for which the new values should not be injected into evaluator, is done in
+   * SequencedSkyframeExecutor.handleChangedFiles().
+   */
   static final class ExternalDirtinessChecker extends BasicFilesystemDirtinessChecker {
     private final ExternalFilesHelper externalFilesHelper;
     private final EnumSet<FileType> fileTypesToCheck;
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/ExternalFilesHelper.java b/src/main/java/com/google/devtools/build/lib/skyframe/ExternalFilesHelper.java
index abc7577..1fcbc4e 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/ExternalFilesHelper.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/ExternalFilesHelper.java
@@ -14,12 +14,17 @@
 package com.google.devtools.build.lib.skyframe;
 
 import com.google.common.base.Preconditions;
+import com.google.devtools.build.lib.actions.FileStateValue;
+import com.google.devtools.build.lib.actions.FileValue;
 import com.google.devtools.build.lib.analysis.BlazeDirectories;
 import com.google.devtools.build.lib.cmdline.LabelConstants;
+import com.google.devtools.build.lib.cmdline.RepositoryName;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadCompatible;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
 import com.google.devtools.build.lib.pkgcache.PathPackageLocator;
+import com.google.devtools.build.lib.rules.repository.ManagedDirectoriesKnowledge;
 import com.google.devtools.build.lib.rules.repository.RepositoryFunction;
+import com.google.devtools.build.lib.util.Pair;
 import com.google.devtools.build.lib.vfs.Path;
 import com.google.devtools.build.lib.vfs.RootedPath;
 import com.google.devtools.build.skyframe.SkyFunction;
@@ -45,40 +50,59 @@
   private boolean anyOutputFilesSeen = false;
   private boolean anyNonOutputExternalFilesSeen = false;
 
+  private final ManagedDirectoriesKnowledge managedDirectoriesKnowledge;
+
   private ExternalFilesHelper(
       AtomicReference<PathPackageLocator> pkgLocator,
       ExternalFileAction externalFileAction,
       BlazeDirectories directories,
-      int maxNumExternalFilesToLog) {
+      int maxNumExternalFilesToLog,
+      ManagedDirectoriesKnowledge managedDirectoriesKnowledge) {
     this.pkgLocator = pkgLocator;
     this.externalFileAction = externalFileAction;
     this.directories = directories;
     this.maxNumExternalFilesToLog = maxNumExternalFilesToLog;
+    this.managedDirectoriesKnowledge = managedDirectoriesKnowledge;
   }
 
   public static ExternalFilesHelper create(
       AtomicReference<PathPackageLocator> pkgLocator,
       ExternalFileAction externalFileAction,
-      BlazeDirectories directories) {
+      BlazeDirectories directories,
+      ManagedDirectoriesKnowledge managedDirectoriesKnowledge) {
     return IN_TEST
-        ? createForTesting(pkgLocator, externalFileAction, directories)
+        ? createForTesting(pkgLocator, externalFileAction, directories, managedDirectoriesKnowledge)
         : new ExternalFilesHelper(
             pkgLocator,
             externalFileAction,
             directories,
-            /*maxNumExternalFilesToLog=*/ 100);
+            /*maxNumExternalFilesToLog=*/ 100,
+            managedDirectoriesKnowledge);
   }
 
   public static ExternalFilesHelper createForTesting(
       AtomicReference<PathPackageLocator> pkgLocator,
       ExternalFileAction externalFileAction,
       BlazeDirectories directories) {
+    return createForTesting(
+        pkgLocator,
+        externalFileAction,
+        directories,
+        ManagedDirectoriesKnowledge.NO_MANAGED_DIRECTORIES);
+  }
+
+  private static ExternalFilesHelper createForTesting(
+      AtomicReference<PathPackageLocator> pkgLocator,
+      ExternalFileAction externalFileAction,
+      BlazeDirectories directories,
+      ManagedDirectoriesKnowledge managedDirectoriesKnowledge) {
     return new ExternalFilesHelper(
         pkgLocator,
         externalFileAction,
         directories,
         // These log lines are mostly spam during unit and integration tests.
-        /*maxNumExternalFilesToLog=*/ 0);
+        /*maxNumExternalFilesToLog=*/ 0,
+        managedDirectoriesKnowledge);
   }
 
 
@@ -105,12 +129,12 @@
 
   /** Classification of a path encountered by Bazel. */
   public enum FileType {
-    /** A path inside the package roots or in an external repository. */
+    /** A path inside the package roots. */
     INTERNAL,
 
     /**
-     * A non {@link #EXTERNAL_REPO} path outside the package roots about which we may make no other
-     * assumptions.
+     * A non {@link #EXTERNAL_REPO} or {@link #EXTERNAL_IN_MANAGED_DIRECTORY} path outside the
+     * package roots about which we may make no other assumptions.
      */
     EXTERNAL,
 
@@ -132,9 +156,22 @@
 
     /**
      * A path in the part of Bazel's output tree that contains (/ symlinks to) to external
-     * repositories.
+     * repositories. Every such path under the external repository is generated by the execution of
+     * the corresponding repository rule, so these paths should not be cached by Skyframe before the
+     * RepositoryDirectoryValue is computed.
      */
     EXTERNAL_REPO,
+
+    /**
+     * A path is under one of the managed directories. Managed directories are user-owned
+     * directories, which can be incrementally updated by repository rules, so that the updated
+     * files are visible for the actions in the same build.
+     *
+     * <p>Every such path under the managed directory is generated or updated by the execution of
+     * the corresponding repository rule, so these paths should not be cached by Skyframe before the
+     * RepositoryDirectoryValue is computed. {@link ManagedDirectoriesKnowledge}
+     */
+    EXTERNAL_IN_MANAGED_DIRECTORY,
   }
 
   /**
@@ -168,10 +205,35 @@
 
   ExternalFilesHelper cloneWithFreshExternalFilesKnowledge() {
     return new ExternalFilesHelper(
-        pkgLocator, externalFileAction, directories, maxNumExternalFilesToLog);
+        pkgLocator,
+        externalFileAction,
+        directories,
+        maxNumExternalFilesToLog,
+        managedDirectoriesKnowledge);
   }
 
-  FileType getAndNoteFileType(RootedPath rootedPath) {
+  public FileType getAndNoteFileType(RootedPath rootedPath) {
+    return getFileTypeAndRepository(rootedPath).getFirst();
+  }
+
+  private Pair<FileType, RepositoryName> getFileTypeAndRepository(RootedPath rootedPath) {
+    RepositoryName repositoryName =
+        managedDirectoriesKnowledge.getOwnerRepository(rootedPath.getRootRelativePath());
+    if (repositoryName != null) {
+      anyNonOutputExternalFilesSeen = true;
+      return Pair.of(FileType.EXTERNAL_IN_MANAGED_DIRECTORY, repositoryName);
+    }
+    FileType fileType = detectFileType(rootedPath);
+    if (FileType.EXTERNAL == fileType || FileType.EXTERNAL_REPO == fileType) {
+      anyNonOutputExternalFilesSeen = true;
+    }
+    if (FileType.OUTPUT == fileType) {
+      anyOutputFilesSeen = true;
+    }
+    return Pair.of(fileType, null);
+  }
+
+  private FileType detectFileType(RootedPath rootedPath) {
     PathPackageLocator packageLocator = pkgLocator.get();
     if (packageLocator.getPathEntries().contains(rootedPath.getRoot())) {
       return FileType.INTERNAL;
@@ -204,27 +266,39 @@
    * a {@link NonexistentImmutableExternalFileException} instead.
    */
   @ThreadSafe
-  void maybeHandleExternalFile(
+  FileType maybeHandleExternalFile(
       RootedPath rootedPath, boolean isDirectory, SkyFunction.Environment env)
       throws NonexistentImmutableExternalFileException, IOException, InterruptedException {
-    FileType fileType = getAndNoteFileType(rootedPath);
-    if (fileType == FileType.INTERNAL) {
-      return;
+    Pair<FileType, RepositoryName> pair = getFileTypeAndRepository(rootedPath);
+
+    FileType fileType = Preconditions.checkNotNull(pair.getFirst());
+    switch (fileType) {
+      case EXTERNAL_IN_MANAGED_DIRECTORY:
+        Preconditions.checkState(
+            externalFileAction == ExternalFileAction.DEPEND_ON_EXTERNAL_PKG_FOR_EXTERNAL_REPO_PATHS,
+            externalFileAction);
+        RepositoryFunction.addManagedDirectoryDependencies(pair.getSecond(), env);
+        break;
+      case INTERNAL:
+        break;
+      case EXTERNAL:
+        if (numExternalFilesLogged.incrementAndGet() < maxNumExternalFilesToLog) {
+          logger.info("Encountered an external path " + rootedPath);
+        }
+        // fall through
+      case OUTPUT:
+        if (externalFileAction
+            == ExternalFileAction.ASSUME_NON_EXISTENT_AND_IMMUTABLE_FOR_EXTERNAL_PATHS) {
+          throw new NonexistentImmutableExternalFileException();
+        }
+        break;
+      case EXTERNAL_REPO:
+        Preconditions.checkState(
+            externalFileAction == ExternalFileAction.DEPEND_ON_EXTERNAL_PKG_FOR_EXTERNAL_REPO_PATHS,
+            externalFileAction);
+        RepositoryFunction.addExternalFilesDependencies(rootedPath, isDirectory, directories, env);
+        break;
     }
-    if (fileType == FileType.OUTPUT || fileType == FileType.EXTERNAL) {
-      if (externalFileAction
-          == ExternalFileAction.ASSUME_NON_EXISTENT_AND_IMMUTABLE_FOR_EXTERNAL_PATHS) {
-        throw new NonexistentImmutableExternalFileException();
-      }
-      if (fileType == FileType.EXTERNAL
-          && numExternalFilesLogged.incrementAndGet() < maxNumExternalFilesToLog) {
-        logger.info("Encountered an external path " + rootedPath);
-      }
-      return;
-    }
-    Preconditions.checkState(
-        externalFileAction == ExternalFileAction.DEPEND_ON_EXTERNAL_PKG_FOR_EXTERNAL_REPO_PATHS,
-        externalFileAction);
-    RepositoryFunction.addExternalFilesDependencies(rootedPath, isDirectory, directories, env);
+    return fileType;
   }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/FileStateFunction.java b/src/main/java/com/google/devtools/build/lib/skyframe/FileStateFunction.java
index 289c943..8ece6ee 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/FileStateFunction.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/FileStateFunction.java
@@ -14,6 +14,7 @@
 package com.google.devtools.build.lib.skyframe;
 
 import com.google.devtools.build.lib.actions.FileStateValue;
+import com.google.devtools.build.lib.skyframe.ExternalFilesHelper.FileType;
 import com.google.devtools.build.lib.util.io.TimestampGranularityMonitor;
 import com.google.devtools.build.lib.vfs.RootedPath;
 import com.google.devtools.build.lib.vfs.UnixGlob.FilesystemCalls;
@@ -50,10 +51,15 @@
     RootedPath rootedPath = (RootedPath) skyKey.argument();
 
     try {
-      externalFilesHelper.maybeHandleExternalFile(rootedPath, false, env);
+      FileType fileType = externalFilesHelper.maybeHandleExternalFile(rootedPath, false, env);
       if (env.valuesMissing()) {
         return null;
       }
+      if (fileType == FileType.EXTERNAL_REPO
+          || fileType == FileType.EXTERNAL_IN_MANAGED_DIRECTORY) {
+        // do not use syscallCache as files under repositories get generated during the build
+        return FileStateValue.create(rootedPath, tsgm.get());
+      }
       return FileStateValue.create(rootedPath, syscallCache.get(), tsgm.get());
     } catch (ExternalFilesHelper.NonexistentImmutableExternalFileException e) {
       return FileStateValue.NONEXISTENT_FILE_STATE_NODE;
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 08f5fa1..8130c4d 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
@@ -62,7 +62,9 @@
 import com.google.devtools.build.lib.profiler.ProfilerTask;
 import com.google.devtools.build.lib.profiler.SilentCloseable;
 import com.google.devtools.build.lib.query2.AqueryActionFilter;
+import com.google.devtools.build.lib.rules.repository.ManagedDirectoriesKnowledge;
 import com.google.devtools.build.lib.skyframe.AspectValue.AspectKey;
+import com.google.devtools.build.lib.skyframe.DiffAwarenessManager.ProcessableModifiedFileSet;
 import com.google.devtools.build.lib.skyframe.DirtinessCheckerUtils.BasicFilesystemDirtinessChecker;
 import com.google.devtools.build.lib.skyframe.DirtinessCheckerUtils.ExternalDirtinessChecker;
 import com.google.devtools.build.lib.skyframe.DirtinessCheckerUtils.MissingDiffDirtinessChecker;
@@ -86,6 +88,7 @@
 import com.google.devtools.build.lib.vfs.RootedPath;
 import com.google.devtools.build.skyframe.BuildDriver;
 import com.google.devtools.build.skyframe.Differencer;
+import com.google.devtools.build.skyframe.Differencer.Diff;
 import com.google.devtools.build.skyframe.EvaluationContext;
 import com.google.devtools.build.skyframe.GraphInconsistencyReceiver;
 import com.google.devtools.build.skyframe.InMemoryMemoizingEvaluator;
@@ -109,7 +112,6 @@
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Callable;
@@ -149,7 +151,8 @@
   private Duration sourceDiffCheckingDuration = Duration.ofSeconds(-1L);
   private Duration outputTreeDiffCheckingDuration = Duration.ofSeconds(-1L);
 
-  @Nullable private final WorkspaceFileHeaderListener workspaceFileHeaderListener;
+  // If this is null then workspace header pre-calculation won't happen.
+  @Nullable private final ManagedDirectoriesKnowledge managedDirectoriesKnowledge;
 
   private SequencedSkyframeExecutor(
       Consumer<SkyframeExecutor> skyframeExecutorConsumerOnInit,
@@ -170,7 +173,7 @@
       ActionOnIOExceptionReadingBuildFile actionOnIOExceptionReadingBuildFile,
       BuildOptions defaultBuildOptions,
       MutableArtifactFactorySupplier mutableArtifactFactorySupplier,
-      @Nullable WorkspaceFileHeaderListener workspaceFileHeaderListener) {
+      @Nullable ManagedDirectoriesKnowledge managedDirectoriesKnowledge) {
     super(
         skyframeExecutorConsumerOnInit,
         evaluatorSupplier,
@@ -193,10 +196,11 @@
         new PackageProgressReceiver(),
         mutableArtifactFactorySupplier,
         new ConfiguredTargetProgressReceiver(),
-        /*nonexistentFileReceiver=*/ null);
+        /*nonexistentFileReceiver=*/ null,
+        managedDirectoriesKnowledge);
     this.diffAwarenessManager = new DiffAwarenessManager(diffAwarenessFactories);
     this.customDirtinessCheckers = customDirtinessCheckers;
-    this.workspaceFileHeaderListener = workspaceFileHeaderListener;
+    this.managedDirectoriesKnowledge = managedDirectoriesKnowledge;
   }
 
   @Override
@@ -330,8 +334,10 @@
     TimestampGranularityMonitor tsgm = this.tsgm.get();
     modifiedFiles = 0;
 
-    if (workspaceFileHeaderListener != null) {
-      refreshWorkspaceHeader(eventHandler);
+    boolean managedDirectoriesChanged =
+        managedDirectoriesKnowledge != null && refreshWorkspaceHeader(eventHandler);
+    if (managedDirectoriesChanged) {
+      invalidateCachedWorkspacePathsStates();
     }
 
     Map<Root, DiffAwarenessManager.ProcessableModifiedFileSet> modifiedFilesByPathEntry =
@@ -347,12 +353,47 @@
         modifiedFilesByPathEntry.put(pathEntry, modifiedFileSet);
       }
     }
-    handleDiffsWithCompleteDiffInformation(tsgm, modifiedFilesByPathEntry);
-    handleDiffsWithMissingDiffInformation(eventHandler, tsgm, pathEntriesWithoutDiffInformation,
-        checkOutputFiles);
+    handleDiffsWithCompleteDiffInformation(
+        tsgm, modifiedFilesByPathEntry, managedDirectoriesChanged);
+    handleDiffsWithMissingDiffInformation(
+        eventHandler,
+        tsgm,
+        pathEntriesWithoutDiffInformation,
+        checkOutputFiles,
+        managedDirectoriesChanged);
     handleClientEnvironmentChanges();
   }
 
+  /**
+   * Skyframe caches the states of files (FileStateValue) and directory listings
+   * (DirectoryListingStateValue). In the case when the files/directories are under a managed
+   * directory or inside an external repository, evaluation of file/directory listing states
+   * requires that RepositoryDirectoryValue of the owning external repository is evaluated
+   * beforehand. (So that the repository rule generates the files.) So there is a dependency on
+   * RepositoryDirectoryValue for files under managed directories and external repositories. This
+   * dependency is recorded by Skyframe.
+   *
+   * <p>From the other side, by default Skyframe injects the new values of changed files already at
+   * the stage of checking what files have changed. Only values without any dependencies can be
+   * injected into Skyframe.
+   *
+   * <p>When the values of managed directories change, whether a file is under a managed directory
+   * can change. This implies that corresponding file/directory listing states gain the dependency
+   * (RepositoryDirectoryValue) or they lose this dependency. In both cases, we should prevent
+   * Skyframe from injecting those new values of file/directory listing states at the stage of
+   * checking changed files, because the files have not been generated yet.
+   *
+   * <p>The selected approach is to invalidate PACKAGE_LOCATOR_DEPENDENT_VALUES, which includes
+   * invalidating all cached file/directory listing state values. Additionally, no new
+   * FileStateValues and DirectoryStateValues should be injected.
+   */
+  private void invalidateCachedWorkspacePathsStates() {
+    logger.info(
+        "Invalidating cached packages and paths states"
+            + " because managed directories configuration has changed.");
+    invalidate(SkyFunctionName.functionIsIn(PACKAGE_LOCATOR_DEPENDENT_VALUES));
+  }
+
   /** Invalidates entries in the client environment. */
   private void handleClientEnvironmentChanges() {
     // Remove deleted client environmental variables.
@@ -382,15 +423,18 @@
    */
   private void handleDiffsWithCompleteDiffInformation(
       TimestampGranularityMonitor tsgm,
-      Map<Root, DiffAwarenessManager.ProcessableModifiedFileSet> modifiedFilesByPathEntry)
+      Map<Root, ProcessableModifiedFileSet> modifiedFilesByPathEntry,
+      boolean managedDirectoriesChanged)
       throws InterruptedException {
     for (Root pathEntry : ImmutableSet.copyOf(modifiedFilesByPathEntry.keySet())) {
       DiffAwarenessManager.ProcessableModifiedFileSet processableModifiedFileSet =
           modifiedFilesByPathEntry.get(pathEntry);
       ModifiedFileSet modifiedFileSet = processableModifiedFileSet.getModifiedFileSet();
       Preconditions.checkState(!modifiedFileSet.treatEverythingAsModified(), pathEntry);
-      handleChangedFiles(ImmutableList.of(pathEntry),
-          getDiff(tsgm, modifiedFileSet.modifiedSourceFiles(), pathEntry));
+      handleChangedFiles(
+          ImmutableList.of(pathEntry),
+          getDiff(tsgm, modifiedFileSet.modifiedSourceFiles(), pathEntry),
+          managedDirectoriesChanged);
       processableModifiedFileSet.markProcessed();
     }
   }
@@ -402,9 +446,9 @@
   private void handleDiffsWithMissingDiffInformation(
       ExtendedEventHandler eventHandler,
       TimestampGranularityMonitor tsgm,
-      Set<Pair<Root, DiffAwarenessManager.ProcessableModifiedFileSet>>
-          pathEntriesWithoutDiffInformation,
-      boolean checkOutputFiles)
+      Set<Pair<Root, ProcessableModifiedFileSet>> pathEntriesWithoutDiffInformation,
+      boolean checkOutputFiles,
+      boolean managedDirectoriesChanged)
       throws InterruptedException {
     ExternalFilesKnowledge externalFilesKnowledge =
         externalFilesHelper.getExternalFilesKnowledge();
@@ -464,7 +508,7 @@
                           new ExternalDirtinessChecker(tmpExternalFilesHelper, fileTypesToCheck),
                           new MissingDiffDirtinessChecker(diffPackageRootsUnderWhichToCheck)))));
     }
-    handleChangedFiles(diffPackageRootsUnderWhichToCheck, diff);
+    handleChangedFiles(diffPackageRootsUnderWhichToCheck, diff, managedDirectoriesChanged);
 
     for (Pair<Root, DiffAwarenessManager.ProcessableModifiedFileSet> pair :
         pathEntriesWithoutDiffInformation) {
@@ -478,18 +522,33 @@
   }
 
   private void handleChangedFiles(
-      Collection<Root> diffPackageRootsUnderWhichToCheck, Differencer.Diff diff) {
-    Collection<SkyKey> changedKeysWithoutNewValues = diff.changedKeysWithoutNewValues();
+      Collection<Root> diffPackageRootsUnderWhichToCheck,
+      Diff diff,
+      boolean managedDirectoriesChanged) {
+    int numWithoutNewValues = diff.changedKeysWithoutNewValues().size();
+    Iterable<SkyKey> keysToBeChangedLaterInThisBuild = diff.changedKeysWithoutNewValues();
     Map<SkyKey, SkyValue> changedKeysWithNewValues = diff.changedKeysWithNewValues();
 
-    logDiffInfo(diffPackageRootsUnderWhichToCheck, changedKeysWithoutNewValues,
+    // If managed directories settings changed, do not inject any new values, just invalidate
+    // keys of the changed values. {@link #invalidateCachedWorkspacePathsStates()}
+    if (managedDirectoriesChanged) {
+      numWithoutNewValues += changedKeysWithNewValues.size();
+      keysToBeChangedLaterInThisBuild =
+          Iterables.concat(keysToBeChangedLaterInThisBuild, changedKeysWithNewValues.keySet());
+      changedKeysWithNewValues = ImmutableMap.of();
+    }
+
+    logDiffInfo(
+        diffPackageRootsUnderWhichToCheck,
+        keysToBeChangedLaterInThisBuild,
+        numWithoutNewValues,
         changedKeysWithNewValues);
 
-    recordingDiffer.invalidate(changedKeysWithoutNewValues);
+    recordingDiffer.invalidate(keysToBeChangedLaterInThisBuild);
     recordingDiffer.inject(changedKeysWithNewValues);
-    modifiedFiles += getNumberOfModifiedFiles(changedKeysWithoutNewValues);
+    modifiedFiles += getNumberOfModifiedFiles(keysToBeChangedLaterInThisBuild);
     modifiedFiles += getNumberOfModifiedFiles(changedKeysWithNewValues.keySet());
-    incrementalBuildMonitor.accrue(changedKeysWithoutNewValues);
+    incrementalBuildMonitor.accrue(keysToBeChangedLaterInThisBuild);
     incrementalBuildMonitor.accrue(changedKeysWithNewValues.keySet());
   }
 
@@ -497,9 +556,10 @@
 
   private static void logDiffInfo(
       Iterable<Root> pathEntries,
-      Collection<SkyKey> changedWithoutNewValue,
+      Iterable<SkyKey> changedWithoutNewValue,
+      int numWithoutNewValues,
       Map<SkyKey, ? extends SkyValue> changedWithNewValue) {
-    int numModified = changedWithNewValue.size() + changedWithoutNewValue.size();
+    int numModified = changedWithNewValue.size() + numWithoutNewValues;
     StringBuilder result = new StringBuilder("DiffAwareness found ")
         .append(numModified)
         .append(" modified source files and directory listings");
@@ -826,10 +886,8 @@
    * and call the listener, if the value has changed. Needed for incremental update of user-owned
    * directories by repository rules.
    */
-  private void refreshWorkspaceHeader(ExtendedEventHandler eventHandler)
+  private boolean refreshWorkspaceHeader(ExtendedEventHandler eventHandler)
       throws InterruptedException, AbruptExitException {
-    Preconditions.checkNotNull(workspaceFileHeaderListener);
-
     Root workspaceRoot = Root.fromPath(directories.getWorkspace());
     RootedPath workspacePath =
         RootedPath.toRootedPath(workspaceRoot, LabelConstants.WORKSPACE_FILE_NAME);
@@ -840,9 +898,7 @@
     maybeInvalidateWorkspaceFileStateValue(workspacePath);
     WorkspaceFileValue newValue =
         (WorkspaceFileValue) evaluateSingleValue(workspaceFileKey, eventHandler);
-    if (!Objects.equals(newValue, oldValue)) {
-      workspaceFileHeaderListener.workspaceHeaderChanged(newValue);
-    }
+    return managedDirectoriesKnowledge.workspaceHeaderReloaded(oldValue, newValue);
   }
 
   // We only check the FileStateValue of the WORKSPACE file; we do not support the case
@@ -901,13 +957,12 @@
     protected ActionKeyContext actionKeyContext;
     protected ImmutableList<BuildInfoFactory> buildInfoFactories;
     protected BuildOptions defaultBuildOptions;
-
     private ImmutableSet<PathFragment> hardcodedBlacklistedPackagePrefixes;
     private PathFragment additionalBlacklistedPackagePrefixesFile;
     private CrossRepositoryLabelViolationStrategy crossRepositoryLabelViolationStrategy;
     private List<BuildFileName> buildFilesByPriority;
     private ActionOnIOExceptionReadingBuildFile actionOnIOExceptionReadingBuildFile;
-    private WorkspaceFileHeaderListener workspaceFileHeaderListener;
+    @Nullable private ManagedDirectoriesKnowledge managedDirectoriesKnowledge;
 
     // Fields with default values.
     private ImmutableMap<SkyFunctionName, SkyFunction> extraSkyFunctions = ImmutableMap.of();
@@ -954,16 +1009,11 @@
               actionOnIOExceptionReadingBuildFile,
               defaultBuildOptions,
               mutableArtifactFactorySupplier,
-              workspaceFileHeaderListener);
+              managedDirectoriesKnowledge);
       skyframeExecutor.init();
       return skyframeExecutor;
     }
 
-    public Builder modify(Consumer<Builder> consumer) {
-      consumer.accept(this);
-      return this;
-    }
-
     public Builder setPkgFactory(PackageFactory pkgFactory) {
       this.pkgFactory = pkgFactory;
       return this;
@@ -1051,27 +1101,28 @@
       this.mutableArtifactFactorySupplier = mutableArtifactFactorySupplier;
       return this;
     }
-
     public Builder setSkyframeExecutorConsumerOnInit(
         Consumer<SkyframeExecutor> skyframeExecutorConsumerOnInit) {
       this.skyframeExecutorConsumerOnInit = skyframeExecutorConsumerOnInit;
       return this;
     }
 
-    public Builder setWorkspaceFileHeaderListener(
-        WorkspaceFileHeaderListener workspaceFileHeaderListener) {
-      this.workspaceFileHeaderListener = workspaceFileHeaderListener;
+    public Builder setManagedDirectoriesKnowledge(
+        @Nullable ManagedDirectoriesKnowledge managedDirectoriesKnowledge) {
+      this.managedDirectoriesKnowledge = managedDirectoriesKnowledge;
       return this;
     }
   }
 
   /**
-   * Listener class to subscribe for WORKSPACE file header changes.
+   * Listener class to subscribe for WORKSPACE file header refreshes.
    *
    * <p>Changes to WORKSPACE file header are computed before the files difference is computed in
    * {@link #handleDiffs(ExtendedEventHandler, boolean, OptionsProvider)}
    */
   public interface WorkspaceFileHeaderListener {
-    void workspaceHeaderChanged(@Nullable WorkspaceFileValue newValue);
+    boolean workspaceHeaderReloaded(
+        @Nullable WorkspaceFileValue oldValue, @Nullable WorkspaceFileValue newValue)
+        throws AbruptExitException;
   }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutorFactory.java b/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutorFactory.java
index ca668ea..33f5a61 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutorFactory.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SequencedSkyframeExecutorFactory.java
@@ -21,7 +21,7 @@
 import com.google.devtools.build.lib.analysis.buildinfo.BuildInfoFactory;
 import com.google.devtools.build.lib.analysis.config.BuildOptions;
 import com.google.devtools.build.lib.packages.PackageFactory;
-import com.google.devtools.build.lib.skyframe.SequencedSkyframeExecutor.WorkspaceFileHeaderListener;
+import com.google.devtools.build.lib.rules.repository.ManagedDirectoriesKnowledge;
 import com.google.devtools.build.lib.vfs.FileSystem;
 import com.google.devtools.build.skyframe.SkyFunction;
 import com.google.devtools.build.skyframe.SkyFunctionName;
@@ -49,7 +49,7 @@
       Iterable<? extends DiffAwareness.Factory> diffAwarenessFactories,
       ImmutableMap<SkyFunctionName, SkyFunction> extraSkyFunctions,
       Iterable<SkyValueDirtinessChecker> customDirtinessCheckers,
-      @Nullable WorkspaceFileHeaderListener workspaceFileHeaderListener) {
+      @Nullable ManagedDirectoriesKnowledge managedDirectoriesKnowledge) {
     return BazelSkyframeExecutorConstants.newBazelSkyframeExecutorBuilder()
         .setPkgFactory(pkgFactory)
         .setFileSystem(fileSystem)
@@ -61,7 +61,7 @@
         .setDiffAwarenessFactories(diffAwarenessFactories)
         .setExtraSkyFunctions(extraSkyFunctions)
         .setCustomDirtinessCheckers(customDirtinessCheckers)
-        .setWorkspaceFileHeaderListener(workspaceFileHeaderListener)
+        .setManagedDirectoriesKnowledge(managedDirectoriesKnowledge)
         .build();
   }
 }
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 e89873b..ced09be 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
@@ -70,6 +70,7 @@
 import com.google.devtools.build.lib.analysis.PlatformOptions;
 import com.google.devtools.build.lib.analysis.TopLevelArtifactContext;
 import com.google.devtools.build.lib.analysis.WorkspaceStatusAction;
+import com.google.devtools.build.lib.analysis.WorkspaceStatusAction.Factory;
 import com.google.devtools.build.lib.analysis.buildinfo.BuildInfoFactory;
 import com.google.devtools.build.lib.analysis.buildinfo.BuildInfoFactory.BuildInfoKey;
 import com.google.devtools.build.lib.analysis.config.BuildConfiguration;
@@ -128,6 +129,7 @@
 import com.google.devtools.build.lib.profiler.SilentCloseable;
 import com.google.devtools.build.lib.remote.options.RemoteOptions;
 import com.google.devtools.build.lib.remote.options.RemoteOutputsMode;
+import com.google.devtools.build.lib.rules.repository.ManagedDirectoriesKnowledge;
 import com.google.devtools.build.lib.rules.repository.ResolvedFileFunction;
 import com.google.devtools.build.lib.rules.repository.ResolvedHashesFunction;
 import com.google.devtools.build.lib.runtime.KeepGoingOption;
@@ -381,7 +383,7 @@
       FileSystem fileSystem,
       BlazeDirectories directories,
       ActionKeyContext actionKeyContext,
-      WorkspaceStatusAction.Factory workspaceStatusActionFactory,
+      Factory workspaceStatusActionFactory,
       ImmutableList<BuildInfoFactory> buildInfoFactories,
       ImmutableMap<SkyFunctionName, SkyFunction> extraSkyFunctions,
       ExternalFileAction externalFileAction,
@@ -396,7 +398,8 @@
       @Nullable PackageProgressReceiver packageProgress,
       MutableArtifactFactorySupplier artifactResolverSupplier,
       @Nullable ConfiguredTargetProgressReceiver configuredTargetProgress,
-      @Nullable NonexistentFileReceiver nonexistentFileReceiver) {
+      @Nullable NonexistentFileReceiver nonexistentFileReceiver,
+      @Nullable ManagedDirectoriesKnowledge managedDirectoriesKnowledge) {
     // Strictly speaking, these arguments are not required for initialization, but all current
     // callsites have them at hand, so we might as well set them during construction.
     this.skyframeExecutorConsumerOnInit = skyframeExecutorConsumerOnInit;
@@ -443,7 +446,13 @@
     this.artifactFactory = artifactResolverSupplier;
     this.artifactFactory.set(skyframeBuildView.getArtifactFactory());
     this.externalFilesHelper =
-        ExternalFilesHelper.create(pkgLocator, externalFileAction, directories);
+        ExternalFilesHelper.create(
+            pkgLocator,
+            externalFileAction,
+            directories,
+            managedDirectoriesKnowledge != null
+                ? managedDirectoriesKnowledge
+                : ManagedDirectoriesKnowledge.NO_MANAGED_DIRECTORIES);
     this.crossRepositoryLabelViolationStrategy = crossRepositoryLabelViolationStrategy;
     this.buildFilesByPriority = buildFilesByPriority;
     this.actionOnIOExceptionReadingBuildFile = actionOnIOExceptionReadingBuildFile;
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutorFactory.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutorFactory.java
index 504e4a4..e7c7106 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutorFactory.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutorFactory.java
@@ -20,7 +20,7 @@
 import com.google.devtools.build.lib.analysis.WorkspaceStatusAction.Factory;
 import com.google.devtools.build.lib.analysis.buildinfo.BuildInfoFactory;
 import com.google.devtools.build.lib.packages.PackageFactory;
-import com.google.devtools.build.lib.skyframe.SequencedSkyframeExecutor.WorkspaceFileHeaderListener;
+import com.google.devtools.build.lib.rules.repository.ManagedDirectoriesKnowledge;
 import com.google.devtools.build.lib.util.AbruptExitException;
 import com.google.devtools.build.lib.vfs.FileSystem;
 import com.google.devtools.build.skyframe.SkyFunction;
@@ -52,6 +52,6 @@
       Iterable<? extends DiffAwareness.Factory> diffAwarenessFactories,
       ImmutableMap<SkyFunctionName, SkyFunction> extraSkyFunctions,
       Iterable<SkyValueDirtinessChecker> customDirtinessCheckers,
-      WorkspaceFileHeaderListener workspaceFileHeaderListener)
+      ManagedDirectoriesKnowledge managedDirectoriesKnowledge)
       throws AbruptExitException;
 }
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java b/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
index 240eeec..69072d4 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/packages/AbstractPackageLoader.java
@@ -43,6 +43,7 @@
 import com.google.devtools.build.lib.packages.PackageFactory.EnvironmentExtension;
 import com.google.devtools.build.lib.packages.WorkspaceFileValue;
 import com.google.devtools.build.lib.pkgcache.PathPackageLocator;
+import com.google.devtools.build.lib.rules.repository.ManagedDirectoriesKnowledge;
 import com.google.devtools.build.lib.skyframe.ASTFileLookupFunction;
 import com.google.devtools.build.lib.skyframe.BlacklistedPackagePrefixesFunction;
 import com.google.devtools.build.lib.skyframe.ContainingPackageLookupFunction;
@@ -232,7 +233,11 @@
     public final PackageLoader build() {
       validate();
       externalFilesHelper =
-          ExternalFilesHelper.create(pkgLocatorRef, externalFileAction, directories);
+          ExternalFilesHelper.create(
+              pkgLocatorRef,
+              externalFileAction,
+              directories,
+              ManagedDirectoriesKnowledge.NO_MANAGED_DIRECTORIES);
       return buildImpl();
     }