RecursivePackageProviderBackedTargetPatternResolver uses streamPackagesUnderDirectory.

The RootPackageExtractor and RecursivePackageProvider interfaces now only have
"streaming" interfaces, in which callers must provide a callback that is invoked as
each package is identified.

This allows us to begin resolving targets as soon as a single batch of
PackagesIdentifiers have been discovered.

RELNOTES:
None.
PiperOrigin-RevId: 260497403
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/CollectPackagesUnderDirectoryValue.java b/src/main/java/com/google/devtools/build/lib/skyframe/CollectPackagesUnderDirectoryValue.java
index a742bb6..1eb90b8 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/CollectPackagesUnderDirectoryValue.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/CollectPackagesUnderDirectoryValue.java
@@ -34,10 +34,10 @@
  * The value computed by {@link CollectPackagesUnderDirectoryFunction}. Contains a mapping for all
  * its non-excluded directories to whether there are packages or error messages beneath them.
  *
- * <p>This value is used by {@link GraphBackedRecursivePackageProvider#getPackagesUnderDirectory} to
- * help it traverse the graph and find the set of packages under a directory, recursively by {@link
- * CollectPackagesUnderDirectoryFunction} which computes a value for a directory by aggregating
- * results calculated from its subdirectories, and by {@link
+ * <p>This value is used by {@link GraphBackedRecursivePackageProvider#streamPackagesUnderDirectory}
+ * to help it traverse the graph and find the set of packages under a directory, recursively by
+ * {@link CollectPackagesUnderDirectoryFunction} which computes a value for a directory by
+ * aggregating results calculated from its subdirectories, and by {@link
  * PrepareDepsOfTargetsUnderDirectoryFunction} which uses this value to find transitive targets to
  * load.
  *
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/EnvironmentBackedRecursivePackageProvider.java b/src/main/java/com/google/devtools/build/lib/skyframe/EnvironmentBackedRecursivePackageProvider.java
index d0c5f62..8b06189 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/EnvironmentBackedRecursivePackageProvider.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/EnvironmentBackedRecursivePackageProvider.java
@@ -14,7 +14,6 @@
 package com.google.devtools.build.lib.skyframe;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
@@ -36,10 +35,10 @@
 import com.google.devtools.build.skyframe.SkyFunction.Environment;
 import com.google.devtools.build.skyframe.SkyKey;
 import java.util.ArrayList;
-import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
 
 /**
  * A {@link RecursivePackageProvider} backed by an {@link Environment}. Its methods may throw {@link
@@ -61,7 +60,7 @@
 
   /**
    * Whether any of the calls to {@link #getPackage}, {@link #getTarget}, {@link #bulkGetPackages},
-   * or {@link #getPackagesUnderDirectory} encountered a package in error.
+   * or {@link #streamPackagesUnderDirectory} encountered a package in error.
    *
    * <p>The client of {@link EnvironmentBackedRecursivePackageProvider} may want to check this. See
    * comments in {@link #getPackage} for details.
@@ -122,8 +121,11 @@
     SkyKey packageLookupKey = PackageLookupValue.key(packageId);
     try {
       PackageLookupValue packageLookupValue =
-          (PackageLookupValue) env.getValueOrThrow(packageLookupKey, NoSuchPackageException.class,
-              InconsistentFilesystemException.class);
+          (PackageLookupValue)
+              env.getValueOrThrow(
+                  packageLookupKey,
+                  NoSuchPackageException.class,
+                  InconsistentFilesystemException.class);
       if (packageLookupValue == null) {
         throw new MissingDepException();
       }
@@ -136,7 +138,8 @@
   }
 
   @Override
-  public Iterable<PathFragment> getPackagesUnderDirectory(
+  public void streamPackagesUnderDirectory(
+      Consumer<PackageIdentifier> results,
       ExtendedEventHandler eventHandler,
       RepositoryName repository,
       PathFragment directory,
@@ -160,13 +163,13 @@
 
       if (!repositoryValue.repositoryExists()) {
         eventHandler.handle(Event.error(String.format("No such repository '%s'", repository)));
-        return ImmutableList.of();
+        return;
       }
       roots.add(Root.fromPath(repositoryValue.getPath()));
     }
 
     if (blacklistedSubdirectories.contains(directory)) {
-      return ImmutableList.of();
+      return;
     }
     ImmutableSet<PathFragment> filteredBlacklistedSubdirectories =
         ImmutableSet.copyOf(
@@ -174,7 +177,6 @@
                 blacklistedSubdirectories,
                 path -> !path.equals(directory) && path.startsWith(directory)));
 
-    LinkedHashSet<PathFragment> packageNames = new LinkedHashSet<>();
     for (Root root : roots) {
       RecursivePkgValue lookup =
           (RecursivePkgValue)
@@ -204,11 +206,9 @@
         if (!Iterables.any(
             excludedSubdirectories,
             excludedSubdirectory -> packageNamePathFragment.startsWith(excludedSubdirectory))) {
-          packageNames.add(packageNamePathFragment);
+          results.accept(PackageIdentifier.create(repository, packageNamePathFragment));
         }
       }
     }
-
-    return packageNames;
   }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/GraphBackedRecursivePackageProvider.java b/src/main/java/com/google/devtools/build/lib/skyframe/GraphBackedRecursivePackageProvider.java
index 8c82a17..668bf0c 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/GraphBackedRecursivePackageProvider.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/GraphBackedRecursivePackageProvider.java
@@ -45,6 +45,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Consumer;
 import java.util.logging.Logger;
 
 /**
@@ -207,7 +208,8 @@
   }
 
   @Override
-  public Iterable<PathFragment> getPackagesUnderDirectory(
+  public void streamPackagesUnderDirectory(
+      Consumer<PackageIdentifier> results,
       ExtendedEventHandler eventHandler,
       RepositoryName repository,
       PathFragment directory,
@@ -218,7 +220,8 @@
         checkValidDirectoryAndGetRoots(
             repository, directory, blacklistedSubdirectories, excludedSubdirectories);
 
-    return rootPackageExtractor.getPackagesFromRoots(
+    rootPackageExtractor.streamPackagesFromRoots(
+        path -> results.accept(PackageIdentifier.create(repository, path)),
         graph,
         roots,
         eventHandler,
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/PackageIdentifierBatchingCallback.java b/src/main/java/com/google/devtools/build/lib/skyframe/PackageIdentifierBatchingCallback.java
new file mode 100644
index 0000000..ecabd39
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/PackageIdentifierBatchingCallback.java
@@ -0,0 +1,68 @@
+// Copyright 2015 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 com.google.common.collect.ImmutableList;
+import com.google.devtools.build.lib.cmdline.PackageIdentifier;
+import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadCompatible;
+import com.google.devtools.build.lib.pkgcache.RecursivePackageProvider;
+import java.util.function.Consumer;
+
+/**
+ * A callback for {@link RecursivePackageProvider#streamPackagesUnderDirectory} that buffers the
+ * PackageIdentifiers it receives into batches that it delivers to a supplied {@code
+ * Consumer<ImmutableList<PackageIdentifier>>}.
+ */
+@ThreadCompatible
+public class PackageIdentifierBatchingCallback
+    implements Consumer<PackageIdentifier>, AutoCloseable {
+
+  private final Consumer<ImmutableList<PackageIdentifier>> batchResults;
+  private final int batchSize;
+  private ImmutableList.Builder<PackageIdentifier> packageIdentifiers;
+  private int bufferedPackageIds;
+
+  public PackageIdentifierBatchingCallback(
+      Consumer<ImmutableList<PackageIdentifier>> batchResults, int batchSize) {
+    this.batchResults = batchResults;
+    this.batchSize = batchSize;
+    reset();
+  }
+
+  @Override
+  public void accept(PackageIdentifier path) {
+    packageIdentifiers.add(path);
+    bufferedPackageIds++;
+    if (bufferedPackageIds >= this.batchSize) {
+      flush();
+    }
+  }
+
+  @Override
+  public void close() {
+    flush();
+  }
+
+  private void flush() {
+    if (bufferedPackageIds > 0) {
+      batchResults.accept(packageIdentifiers.build());
+      reset();
+    }
+  }
+
+  private void reset() {
+    packageIdentifiers = ImmutableList.builderWithExpectedSize(batchSize);
+    bufferedPackageIds = 0;
+  }
+}
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/RecursivePackageProviderBackedTargetPatternResolver.java b/src/main/java/com/google/devtools/build/lib/skyframe/RecursivePackageProviderBackedTargetPatternResolver.java
index d63b8eb..1d18260 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/RecursivePackageProviderBackedTargetPatternResolver.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/RecursivePackageProviderBackedTargetPatternResolver.java
@@ -50,7 +50,9 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
 
 /**
  * A {@link TargetPatternResolver} backed by a {@link RecursivePackageProvider}.
@@ -180,7 +182,7 @@
       implements ThreadSafeBatchCallback<T, E> {
     private final BatchCallback<T, E> delegate;
 
-    public SynchronizedBatchCallback(BatchCallback<T, E> delegate) {
+    SynchronizedBatchCallback(BatchCallback<T, E> delegate) {
       this.delegate = delegate;
     }
 
@@ -241,76 +243,95 @@
   }
 
   private <E extends Exception> ListenableFuture<Void> findTargetsBeneathDirectoryAsyncImpl(
-      final RepositoryName repository,
-      final String originalPattern,
+      RepositoryName repository,
+      String pattern,
       String directory,
       boolean rulesOnly,
       ImmutableSet<PathFragment> blacklistedSubdirectories,
       ImmutableSet<PathFragment> excludedSubdirectories,
-      final ThreadSafeBatchCallback<Target, E> callback,
+      ThreadSafeBatchCallback<Target, E> callback,
       ListeningExecutorService executor) {
-    final FilteringPolicy actualPolicy = rulesOnly
-        ? FilteringPolicies.and(FilteringPolicies.RULES_ONLY, policy)
-        : policy;
-    final PathFragment pathFragment;
-    Iterable<PathFragment> packagesUnderDirectory;
-    try {
+    FilteringPolicy actualPolicy =
+        rulesOnly ? FilteringPolicies.and(FilteringPolicies.RULES_ONLY, policy) : policy;
+    PathFragment pathFragment;
+    ArrayList<ListenableFuture<Void>> futures = new ArrayList<>();
+    Consumer<ImmutableList<PackageIdentifier>> startGettingTargetsCallback =
+        (pkgIdBatch) ->
+            futures.add(
+                executor.submit(
+                    new GetTargetsInPackagesTask<>(pkgIdBatch, pattern, actualPolicy, callback)));
+    try (PackageIdentifierBatchingCallback pkgIdBatchProducer =
+        new PackageIdentifierBatchingCallback(startGettingTargetsCallback, MAX_PACKAGES_BULK_GET)) {
       pathFragment = TargetPatternResolverUtil.getPathFragment(directory);
-      packagesUnderDirectory =
-          recursivePackageProvider.getPackagesUnderDirectory(
-              eventHandler,
-              repository,
-              pathFragment,
-              blacklistedSubdirectories,
-              excludedSubdirectories);
+      recursivePackageProvider.streamPackagesUnderDirectory(
+          pkgIdBatchProducer,
+          eventHandler,
+          repository,
+          pathFragment,
+          blacklistedSubdirectories,
+          excludedSubdirectories);
     } catch (TargetParsingException e) {
       return Futures.immediateFailedFuture(e);
     } catch (InterruptedException e) {
       return Futures.immediateCancelledFuture();
     }
 
-    if (Iterables.isEmpty(packagesUnderDirectory)) {
+    if (futures.isEmpty()) {
       return Futures.immediateFailedFuture(
           new TargetParsingException("no targets found beneath '" + pathFragment + "'"));
     }
 
-    Iterable<PackageIdentifier> pkgIds =
-        Iterables.transform(
-            packagesUnderDirectory, path -> PackageIdentifier.create(repository, path));
-
-    // For very large sets of packages, we may not want to process all of them at once, so we split
-    // into batches.
-    List<List<PackageIdentifier>> partitions =
-        ImmutableList.copyOf(Iterables.partition(pkgIds, MAX_PACKAGES_BULK_GET));
-    ArrayList<ListenableFuture<Void>> futures = new ArrayList<>(partitions.size());
-    for (final Iterable<PackageIdentifier> pkgIdBatch : partitions) {
-      futures.add(
-          executor.submit(
-              () -> {
-                ImmutableSet<PackageIdentifier> pkgIdBatchSet = ImmutableSet.copyOf(pkgIdBatch);
-                packageSemaphore.acquireAll(pkgIdBatchSet);
-                try {
-                  Iterable<Collection<Target>> resolvedTargets =
-                      bulkGetTargetsInPackage(originalPattern, pkgIdBatch, actualPolicy).values();
-                  List<Target> filteredTargets = new ArrayList<>(calculateSize(resolvedTargets));
-                  for (Collection<Target> targets : resolvedTargets) {
-                    filteredTargets.addAll(targets);
-                  }
-                  // TODO(bazel-core): Invoking the callback while holding onto the package
-                  // semaphore can lead to deadlocks. Also, if the semaphore has a small count,
-                  // acquireAll can also lead to problems if we don't batch appropriately.
-                  // Although we default to an unbounded semaphore for SkyQuery and this is an
-                  // unreported issue, consider refactoring so that the code is strictly correct.
-                  callback.process(filteredTargets);
-                } finally {
-                  packageSemaphore.releaseAll(pkgIdBatchSet);
-                }
-                return null;
-              }));
-    }
     return Futures.whenAllSucceed(futures).call(() -> null, directExecutor());
   }
 
+  /**
+   * Task to get all matching targets in the given packages, filter them, and pass them to the
+   * target batch callback.
+   */
+  private class GetTargetsInPackagesTask<E extends Exception> implements Callable<Void> {
+
+    private final Iterable<PackageIdentifier> packageIdentifiers;
+    private final String originalPattern;
+    private final FilteringPolicy actualPolicy;
+    private final ThreadSafeBatchCallback<Target, E> callback;
+
+    GetTargetsInPackagesTask(
+        Iterable<PackageIdentifier> packageIdentifiers,
+        String originalPattern,
+        FilteringPolicy actualPolicy,
+        ThreadSafeBatchCallback<Target, E> callback) {
+      this.packageIdentifiers = packageIdentifiers;
+      this.originalPattern = originalPattern;
+      this.actualPolicy = actualPolicy;
+      this.callback = callback;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      ImmutableSet<PackageIdentifier> pkgIdBatchSet = ImmutableSet.copyOf(packageIdentifiers);
+      packageSemaphore.acquireAll(pkgIdBatchSet);
+      try {
+        Iterable<Collection<Target>> resolvedTargets =
+            RecursivePackageProviderBackedTargetPatternResolver.this
+                .bulkGetTargetsInPackage(originalPattern, packageIdentifiers, actualPolicy)
+                .values();
+        List<Target> filteredTargets = new ArrayList<>(calculateSize(resolvedTargets));
+        for (Collection<Target> targets : resolvedTargets) {
+          filteredTargets.addAll(targets);
+        }
+        // TODO(bazel-core): Invoking the callback while holding onto the package
+        // semaphore can lead to deadlocks. Also, if the semaphore has a small count,
+        // acquireAll can also lead to problems if we don't batch appropriately.
+        // Although we default to an unbounded semaphore for SkyQuery and this is an
+        // unreported issue, consider refactoring so that the code is strictly correct.
+        callback.process(filteredTargets);
+      } finally {
+        packageSemaphore.releaseAll(pkgIdBatchSet);
+      }
+      return null;
+    }
+  }
+
   private static <T> int calculateSize(Iterable<Collection<T>> resolvedTargets) {
     int size = 0;
     for (Collection<T> targets : resolvedTargets) {
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/RecursivePkgValueRootPackageExtractor.java b/src/main/java/com/google/devtools/build/lib/skyframe/RecursivePkgValueRootPackageExtractor.java
index df3e53e..18579ae 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/RecursivePkgValueRootPackageExtractor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/RecursivePkgValueRootPackageExtractor.java
@@ -22,13 +22,15 @@
 import com.google.devtools.build.lib.vfs.Root;
 import com.google.devtools.build.lib.vfs.RootedPath;
 import com.google.devtools.build.skyframe.WalkableGraph;
-import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.function.Consumer;
 
 /** Looks up {@link RecursivePkgValue}s of given roots in a {@link WalkableGraph}. */
 public class RecursivePkgValueRootPackageExtractor implements RootPackageExtractor {
 
-  public Iterable<PathFragment> getPackagesFromRoots(
+  @Override
+  public void streamPackagesFromRoots(
+      Consumer<PathFragment> results,
       WalkableGraph graph,
       List<Root> roots,
       ExtendedEventHandler eventHandler,
@@ -37,14 +39,12 @@
       ImmutableSet<PathFragment> blacklistedSubdirectories,
       ImmutableSet<PathFragment> excludedSubdirectories)
       throws InterruptedException {
-
     ImmutableSet filteredBlacklistedSubdirectories =
         ImmutableSet.copyOf(
             Iterables.filter(
                 blacklistedSubdirectories,
                 path -> !path.equals(directory) && path.startsWith(directory)));
 
-    LinkedHashSet<PathFragment> packageNames = new LinkedHashSet<>();
     for (Root root : roots) {
       // Note: no need to check if lookup == null because it will never be null.
       // {@link RecursivePkgFunction} handles all errors in a keep_going build.
@@ -68,11 +68,9 @@
         if (!Iterables.any(
             excludedSubdirectories,
             excludedSubdirectory -> packageNamePathFragment.startsWith(excludedSubdirectory))) {
-          packageNames.add(packageNamePathFragment);
+          results.accept(packageNamePathFragment);
         }
       }
     }
-
-    return packageNames;
   }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/RootPackageExtractor.java b/src/main/java/com/google/devtools/build/lib/skyframe/RootPackageExtractor.java
index 553262e..f6eeb6a 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/RootPackageExtractor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/RootPackageExtractor.java
@@ -20,10 +20,30 @@
 import com.google.devtools.build.lib.vfs.Root;
 import com.google.devtools.build.skyframe.WalkableGraph;
 import java.util.List;
+import java.util.function.Consumer;
 
-/** An interface for returning recursive packages under a given set of roots. */
+/** A streaming interface for recursively searching for all packages under a given set of roots. */
 public interface RootPackageExtractor {
-  Iterable<PathFragment> getPackagesFromRoots(
+
+  /**
+   * Recursively search each of the given roots in a repository for packages (while respecting
+   * blacklists and exclusions), calling the {@code results} callback as each package is discovered.
+   *
+   * @param results callback invoked once for each package as it is discovered under a root
+   * @param graph skyframe graph used for retrieving the directories under each root
+   * @param roots all the filesystem roots to search for packages
+   * @param eventHandler receives package-loading errors for any packages loaded by graph queries
+   * @param repository the repository under which the roots can be found
+   * @param directory starting directory under which to find packages, relative to the roots
+   * @param blacklistedSubdirectories directories that will not be searched by policy, relative to
+   *     the roots
+   * @param excludedSubdirectories directories the user requests not be searched, relative to the
+   *     roots
+   * @throws InterruptedException if a graph query is interrupted before all roots have been
+   *     searched exhaustively
+   */
+  void streamPackagesFromRoots(
+      Consumer<PathFragment> results,
       WalkableGraph graph,
       List<Root> roots,
       ExtendedEventHandler eventHandler,
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/TraversalInfoRootPackageExtractor.java b/src/main/java/com/google/devtools/build/lib/skyframe/TraversalInfoRootPackageExtractor.java
index 3a0624f..8fba861 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/TraversalInfoRootPackageExtractor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/TraversalInfoRootPackageExtractor.java
@@ -16,7 +16,6 @@
 import static com.google.common.collect.ImmutableSet.toImmutableSet;
 
 import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
@@ -33,6 +32,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.TreeSet;
+import java.util.function.Consumer;
 
 /** Looks up values under {@link TraversalInfo}s of given roots in a {@link WalkableGraph}. */
 public class TraversalInfoRootPackageExtractor implements RootPackageExtractor {
@@ -41,7 +41,8 @@
       Comparator.comparing(ti -> ti.rootedDir.getRootRelativePath());
 
   @Override
-  public Iterable<PathFragment> getPackagesFromRoots(
+  public void streamPackagesFromRoots(
+      Consumer<PathFragment> results,
       WalkableGraph graph,
       List<Root> roots,
       ExtendedEventHandler eventHandler,
@@ -50,27 +51,22 @@
       ImmutableSet<PathFragment> blacklistedSubdirectories,
       ImmutableSet<PathFragment> excludedSubdirectories)
       throws InterruptedException {
-    // If we found a TargetsBelowDirectory pattern in the universe that contains this directory,
-    // then we can look for packages in and under it in the graph. If we didn't find one, then the
-    // directory wasn't in the universe, so return an empty list.
-    ImmutableList.Builder<PathFragment> builder = ImmutableList.builder();
     for (Root root : roots) {
       RootedPath rootedDir = RootedPath.toRootedPath(root, directory);
       TraversalInfo info =
           new TraversalInfo(rootedDir, blacklistedSubdirectories, excludedSubdirectories);
       TreeSet<TraversalInfo> dirsToCheckForPackages = new TreeSet<>(TRAVERSAL_INFO_COMPARATOR);
       dirsToCheckForPackages.add(info);
-      collectPackagesUnder(graph, eventHandler, repository, dirsToCheckForPackages, builder);
+      collectPackagesUnder(results, graph, eventHandler, repository, dirsToCheckForPackages);
     }
-    return builder.build();
   }
 
   private void collectPackagesUnder(
+      Consumer<PathFragment> results,
       WalkableGraph graph,
       ExtendedEventHandler eventHandler,
       final RepositoryName repository,
-      TreeSet<TraversalInfo> dirsToCheckForPackages,
-      ImmutableList.Builder<PathFragment> resultsBuilder)
+      TreeSet<TraversalInfo> dirsToCheckForPackages)
       throws InterruptedException {
     // NOTE: Maps.asMap returns a Map<T> view whose entrySet() order matches the underlying Set<T>.
     Map<TraversalInfo, SkyKey> traversalToKeyMap =
@@ -91,7 +87,7 @@
           (CollectPackagesUnderDirectoryValue) val;
       if (collectPackagesValue != null) {
         if (collectPackagesValue.isDirectoryPackage()) {
-          resultsBuilder.add(info.rootedDir.getRootRelativePath());
+          results.accept(info.rootedDir.getRootRelativePath());
         }
 
         if (collectPackagesValue.getErrorMessage() != null) {
@@ -126,8 +122,7 @@
     }
 
     if (!subdirsToCheckForPackages.isEmpty()) {
-      collectPackagesUnder(
-          graph, eventHandler, repository, subdirsToCheckForPackages, resultsBuilder);
+      collectPackagesUnder(results, graph, eventHandler, repository, subdirsToCheckForPackages);
     }
   }