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/pkgcache/RecursivePackageProvider.java b/src/main/java/com/google/devtools/build/lib/pkgcache/RecursivePackageProvider.java
index 7dd322c..a395124 100644
--- a/src/main/java/com/google/devtools/build/lib/pkgcache/RecursivePackageProvider.java
+++ b/src/main/java/com/google/devtools/build/lib/pkgcache/RecursivePackageProvider.java
@@ -25,6 +25,7 @@
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.lib.vfs.RootedPath;
import java.util.Map;
+import java.util.function.Consumer;
/**
* Support for resolving {@code package/...} target patterns.
@@ -32,11 +33,14 @@
public interface RecursivePackageProvider extends PackageProvider {
/**
- * Returns the names of all the packages under a given directory.
+ * Calls the supplied callback with the name of each package under a given directory, as soon as
+ * that package is identified.
*
- * <p>Packages returned by this method and passed into {@link #bulkGetPackages(Iterable)} are
+ * <p>Packages yielded by this method and passed into {@link #bulkGetPackages(Iterable)} are
* expected to return successful {@link Package} values.
*
+ * @param results callback invoked <em>from a single thread</em> for every eligible, loaded
+ * package as it is discovered
* @param eventHandler any errors emitted during package lookup and loading for {@code directory}
* and non-excluded directories beneath it will be reported here
* @param directory a {@link RootedPath} specifying the directory to search
@@ -45,7 +49,8 @@
* @param excludedSubdirectories a set of {@link PathFragment}s specifying transitive
* subdirectories to exclude
*/
- Iterable<PathFragment> getPackagesUnderDirectory(
+ void streamPackagesUnderDirectory(
+ Consumer<PackageIdentifier> results,
ExtendedEventHandler eventHandler,
RepositoryName repository,
PathFragment directory,
@@ -72,7 +77,7 @@
/**
* A {@link RecursivePackageProvider} in terms of a map of pre-fetched packages.
*
- * <p>Note that this class implements neither {@link #getPackagesUnderDirectory} nor {@link
+ * <p>Note that this class implements neither {@link #streamPackagesUnderDirectory} nor {@link
* #bulkGetPackages}, so it can only be used for use cases that do not call either of these
* methods. When used for target pattern resolution, it can be used to resolve SINGLE_TARGET and
* TARGETS_IN_PACKAGE patterns by pre-fetching the corresponding packages. It can also be used to
@@ -110,7 +115,8 @@
}
@Override
- public Iterable<PathFragment> getPackagesUnderDirectory(
+ public void streamPackagesUnderDirectory(
+ Consumer<PackageIdentifier> results,
ExtendedEventHandler eventHandler,
RepositoryName repository,
PathFragment directory,
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);
}
}