Skip to content

Commit

Permalink
Implement GlobsFunction and GlobsProducer
Browse files Browse the repository at this point in the history
This change implements the main logic of `GlobsFunction`, which creates `GlobsProducer` to start the `StateMachine`.

The `GlobsProducer` will query the IgnoredPackagePrefixed (IPP) patterns and create `GlobComputationProducer`s for every `glob` or `subpackages` call defined in the package's `BUILD` file. Computation of each glob is done under these `GlobComputationProducer`s as what is done in 605a05a.

I also refactored existing `GlobFunctionTest` into:

* `GlobTestBase`: Holds all test cases used for both `GlobFunctionTest` and `GlobsFunctionTest`.
* `GlobFunctionTest`: Holds utility code necessary to test `GlobFunction`.
* `GlobFunctionTest`: Holds utility code necessary to test `GlobsFunction`. Contains some additional test to cover when `GlobsValue.Key` has more than one `GlobRequest`.

PiperOrigin-RevId: 595383193
Change-Id: I7f36ba6f765ce981e145d5f2fd53103c68df54d8
  • Loading branch information
yuyue730 authored and copybara-github committed Jan 3, 2024
1 parent 84d1a72 commit ffc8c15
Show file tree
Hide file tree
Showing 11 changed files with 1,561 additions and 1,150 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ java_library(
"//src/main/java/com/google/devtools/build/lib/packages:globber",
"//src/main/java/com/google/devtools/build/lib/skyframe:directory_listing_value",
"//src/main/java/com/google/devtools/build/lib/skyframe:glob_descriptor",
"//src/main/java/com/google/devtools/build/lib/skyframe:globs_value",
"//src/main/java/com/google/devtools/build/lib/skyframe:ignored_package_prefixes_value",
"//src/main/java/com/google/devtools/build/lib/skyframe:package_lookup_value",
"//src/main/java/com/google/devtools/build/lib/util:pair",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,14 +36,25 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
import java.util.regex.Pattern;
import javax.annotation.Nullable;

/**
* Serves as the entrance {@link StateMachine} for {@link
* com.google.devtools.build.lib.skyframe.GlobFunctionWithMultipleRecursiveFunctions}.
* Serves as the entrance {@link StateMachine} to compute a single glob. There are two ways to
* create {@link GlobComputationProducer}:
*
* <p>Gets all ignored package prefix patterns. Starts the {@link FragmentProducer}s chain which
* recursively process each glob pattern fragment. Accepts and aggregates globbing matching {@link
* PathFragment}s result.
* <ul>
* <li>When each glob within a package is represented as an individual GLOB node, {@link
* com.google.devtools.build.lib.skyframe.GlobFunctionWithRecursionInSingleFunction} creates
* {@link GlobComputationProducer} to start the computation of each GLOB node.
* <li>Glob computations within in a package are aggregated into a single GLOBS node. {@link
* com.google.devtools.build.lib.skyframe.GlobsFunction} creates {@link GlobsProducer}, which
* further creates {@link GlobComputationProducer} to compute each glob.
* </ul>
*
* <p>Ignored package prefix (IPP) patterns can be optionally passed in. {@link
* GlobComputationProducer} will query the IPP patterns in Skyframe if not provided. Then it starts
* the {@link FragmentProducer}s chain which recursively process each glob pattern fragment. Accepts
* and aggregates globbing matching {@link PathFragment}s result.
*/
public class GlobComputationProducer
implements StateMachine, Consumer<SkyValue>, FragmentProducer.ResultSink {
Expand All @@ -52,15 +63,15 @@ public class GlobComputationProducer
* Propagates all glob matching {@link PathFragment}s or any {@link Exception}.
*
* <p>If any {@link GlobError} is accepted, the already discovered path fragments are still
* reported. However, {@link
* com.google.devtools.build.lib.skyframe.GlobFunctionWithRecursionInSingleFunction} throws the
* first discovered {@link GlobError} wrapped in a {@link
* com.google.devtools.build.lib.skyframe.GlobFunction.GlobFunctionException}.
* reported. However, {@link com.google.devtools.build.lib.skyframe.GlobFunction} and {@link
* com.google.devtools.build.lib.skyframe.GlobsFunction} throw the first discovered {@link
* GlobError} wrapped in a {@link com.google.devtools.build.lib.skyframe.GlobException}.
*
* <p>The already discovered path fragments should be considered as undefined. Since: (1) there is
* no skyframe restart after glob computation throws an exception, so the discovered path
* fragments can miss some matchings; (2) these discovered path fragments are not used to
* construct a {@link com.google.devtools.build.lib.skyframe.GlobValue}.
* construct a {@link com.google.devtools.build.lib.skyframe.GlobValue} or {@link
* com.google.devtools.build.lib.skyframe.GlobsValue}.
*/
public interface ResultSink {
void acceptPathFragmentsWithoutPackageFragment(ImmutableSet<PathFragment> pathFragments);
Expand All @@ -74,33 +85,38 @@ public interface ResultSink {

// -------------------- Internal State --------------------
private final ImmutableSet.Builder<PathFragment> pathFragmentsWithPackageFragment;
private ImmutableSet<PathFragment> ignorePackagePrefixPatterns = null;
private ImmutableSet<PathFragment> ignoredPackagePrefixPatterns;
private final ConcurrentHashMap<String, Pattern> regexPatternCache;

public GlobComputationProducer(
GlobDescriptor globDescriptor,
@Nullable ImmutableSet<PathFragment> ignoredPackagePrefixPatterns,
ConcurrentHashMap<String, Pattern> regexPatternCache,
ResultSink resultSink) {
this.globDescriptor = globDescriptor;
this.ignoredPackagePrefixPatterns = ignoredPackagePrefixPatterns;
this.regexPatternCache = regexPatternCache;
this.resultSink = resultSink;
this.pathFragmentsWithPackageFragment = ImmutableSet.builder();
}

@Override
public StateMachine step(Tasks tasks) throws InterruptedException {
RepositoryName repositoryName = globDescriptor.getPackageId().getRepository();
tasks.lookUp(IgnoredPackagePrefixesValue.key(repositoryName), (Consumer<SkyValue>) this);
if (ignoredPackagePrefixPatterns == null) {
// Query ignorePatterPrefixPatterns in Skyframe if not provided.
RepositoryName repositoryName = globDescriptor.getPackageId().getRepository();
tasks.lookUp(IgnoredPackagePrefixesValue.key(repositoryName), (Consumer<SkyValue>) this);
}
return this::createFragmentProducer;
}

@Override
public void accept(SkyValue skyValue) {
this.ignorePackagePrefixPatterns = ((IgnoredPackagePrefixesValue) skyValue).getPatterns();
this.ignoredPackagePrefixPatterns = ((IgnoredPackagePrefixesValue) skyValue).getPatterns();
}

private StateMachine createFragmentProducer(Tasks tasks) {
Preconditions.checkNotNull(ignorePackagePrefixPatterns);
Preconditions.checkNotNull(ignoredPackagePrefixPatterns);
ImmutableList<String> patterns =
ImmutableList.copyOf(Splitter.on('/').split(globDescriptor.getPattern()));
GlobDetail globDetail =
Expand All @@ -109,7 +125,7 @@ private StateMachine createFragmentProducer(Tasks tasks) {
globDescriptor.getPackageRoot(),
patterns,
/* containsMultipleDoubleStars= */ Collections.frequency(patterns, "**") > 1,
ignorePackagePrefixPatterns,
ignoredPackagePrefixPatterns,
regexPatternCache,
globDescriptor.globberOperation());
Set<Pair<PathFragment, Integer>> visitedGlobSubTasks = null;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
// Copyright 2024 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.packages.producers;

import com.google.common.collect.ImmutableSet;
import com.google.devtools.build.lib.cmdline.RepositoryName;
import com.google.devtools.build.lib.skyframe.GlobDescriptor;
import com.google.devtools.build.lib.skyframe.GlobsValue;
import com.google.devtools.build.lib.skyframe.GlobsValue.GlobRequest;
import com.google.devtools.build.lib.skyframe.IgnoredPackagePrefixesValue;
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.skyframe.SkyValue;
import com.google.devtools.build.skyframe.state.StateMachine;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
import java.util.regex.Pattern;

/**
* Serves as the entrance {@link StateMachine} for {@link
* com.google.devtools.build.lib.skyframe.GlobsFunction}.
*
* <p>{@link GlobsValue.Key} which includes all globs within in a package is provided. {@link
* GlobsProducer} queries the {@link #ignoredPackagePrefixPatterns} before creating {@link
* GlobComputationProducer}s for each glob.
*
* <p>{@link GlobsProducer} is expected to make glob computations within a package have some
* structured logical concurrency and reduce the number of Skyframe restarts.
*/
public class GlobsProducer
implements StateMachine, Consumer<SkyValue>, GlobComputationProducer.ResultSink {

/**
* Propagates all glob matching {@link PathFragment}s or any {@link Exception}.
*
* <p>See {@link GlobComputationProducer.ResultSink} for more details.
*/
public interface ResultSink {
void acceptAggregateMatchingPaths(ImmutableSet<PathFragment> globsMatchingResult);

void acceptGlobError(GlobError globError);
}

// -------------------- Input --------------------
private final GlobsValue.Key globsKey;
private final ResultSink resultSink;

// -------------------- Internal State --------------------
private final ImmutableSet.Builder<PathFragment> aggregateMatchingPathsBuilder =
ImmutableSet.builder();
private ImmutableSet<PathFragment> ignoredPackagePrefixPatterns = null;
private final ConcurrentHashMap<String, Pattern> regexPatternCache;

public GlobsProducer(
GlobsValue.Key globsKey,
ConcurrentHashMap<String, Pattern> regexPatternCache,
ResultSink resultSink) {
this.globsKey = globsKey;
this.regexPatternCache = regexPatternCache;
this.resultSink = resultSink;
}

@Override
public StateMachine step(Tasks tasks) throws InterruptedException {
RepositoryName repositoryName = globsKey.getPackageIdentifier().getRepository();
tasks.lookUp(IgnoredPackagePrefixesValue.key(repositoryName), (Consumer<SkyValue>) this);
return this::createGlobComputationProducers;
}

@Override
public void accept(SkyValue skyValue) {
this.ignoredPackagePrefixPatterns = ((IgnoredPackagePrefixesValue) skyValue).getPatterns();
}

public StateMachine createGlobComputationProducers(Tasks tasks) {
if (ignoredPackagePrefixPatterns == null) {
return DONE;
}

for (GlobRequest globRequest : globsKey.getGlobRequests()) {
GlobDescriptor globDescriptor =
GlobDescriptor.create(
globsKey.getPackageIdentifier(),
globsKey.getPackageRoot(),
PathFragment.EMPTY_FRAGMENT,
globRequest.getPattern(),
globRequest.getGlobOeration());
tasks.enqueue(
new GlobComputationProducer(
globDescriptor,
ignoredPackagePrefixPatterns,
regexPatternCache,
(GlobComputationProducer.ResultSink) this));
}

return this::aggregateResults;
}

@Override
public void acceptPathFragmentsWithoutPackageFragment(ImmutableSet<PathFragment> pathFragments) {
aggregateMatchingPathsBuilder.addAll(pathFragments);
}

@Override
public void acceptGlobError(GlobError error) {
resultSink.acceptGlobError(error);
}

public StateMachine aggregateResults(Tasks tasks) {
resultSink.acceptAggregateMatchingPaths(aggregateMatchingPathsBuilder.build());
return DONE;
}
}
6 changes: 6 additions & 0 deletions src/main/java/com/google/devtools/build/lib/skyframe/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -1468,8 +1468,14 @@ java_library(
name = "globs_function",
srcs = ["GlobsFunction.java"],
deps = [
":glob_exception",
":globs_value",
"//src/main/java/com/google/devtools/build/lib/packages/producers",
"//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",
"//third_party:guava",
"//third_party:jsr305",
],
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public final class GlobDescriptor implements SkyKey {
* @param pattern a valid glob pattern
* @param globberOperation type of Globber operation being tracked.
*/
static GlobDescriptor create(
public static GlobDescriptor create(
PackageIdentifier packageId,
Root packageRoot,
PathFragment subdir,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,9 @@ public SkyValue compute(SkyKey skyKey, Environment env)

if (state.globComputationDriver == null) {
state.globComputationDriver =
new Driver(new GlobComputationProducer(glob, regexPatternCache, state));
new Driver(
new GlobComputationProducer(
glob, /* ignoredPackagePrefixPatterns= */ null, regexPatternCache, state));
}

if (!state.globComputationDriver.drive(env)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,16 +13,80 @@
// limitations under the License.
package com.google.devtools.build.lib.skyframe;

import com.google.common.collect.ImmutableSet;
import com.google.devtools.build.lib.packages.producers.GlobError;
import com.google.devtools.build.lib.packages.producers.GlobsProducer;
import com.google.devtools.build.lib.vfs.PathFragment;
import com.google.devtools.build.skyframe.SkyFunction;
import com.google.devtools.build.skyframe.SkyFunction.Environment.SkyKeyComputeState;
import com.google.devtools.build.skyframe.SkyFunctionException;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.SkyValue;
import com.google.devtools.build.skyframe.state.Driver;
import java.util.concurrent.ConcurrentHashMap;
import java.util.regex.Pattern;
import javax.annotation.Nullable;

/** A SkyFunction for {@link GlobsValue}s. */
// TODO(b/290998109): Implement GlobsFunction with StateMachine mechanism.
/**
* A {@link SkyFunction} for {@link GlobsValue}, which drives the glob matching process for all
* globs within a package.
*
* <p>{@link GlobsFunction} creates a {@link GlobsProducer} which takes in all package's {@link
* com.google.devtools.build.lib.skyframe.GlobsValue.GlobRequest}s. The {@link GlobsProducer} then
* creates {@link com.google.devtools.build.lib.packages.producers.GlobComputationProducer} for each
* {@link com.google.devtools.build.lib.skyframe.GlobsValue.GlobRequest} in the package and collects
* matching paths or the first discovered error.
*
* <p>{@link GlobsFunction} has two benefits over {@link GlobFunction}:
*
* <ul>
* <li>The multiple GLOB nodes rdeping on the package are aggregated into a single one. This
* reduces some memory overhead, especially when number of globs defined in the BUILD file is
* very large.
* <li>Evaluating all globs within a package starts to have some structured logical concurrency,
* thus reducing the number of Skyframe restarts.
* </ul>
*/
public class GlobsFunction implements SkyFunction {

protected ConcurrentHashMap<String, Pattern> regexPatternCache = new ConcurrentHashMap<>();

private static class State implements SkyKeyComputeState, GlobsProducer.ResultSink {
@Nullable private Driver globsDriver;

private ImmutableSet<PathFragment> globsMatchingResult;
private GlobError error;

@Override
public void acceptAggregateMatchingPaths(ImmutableSet<PathFragment> globsMatchingResult) {
this.globsMatchingResult = globsMatchingResult;
}

@Override
public void acceptGlobError(GlobError error) {
this.error = error;
}
}

@Nullable
@Override
public SkyValue compute(SkyKey skyKey, Environment env) {
throw new UnsupportedOperationException();
public SkyValue compute(SkyKey skyKey, Environment env)
throws SkyFunctionException, InterruptedException {
GlobsValue.Key globsKey = (GlobsValue.Key) skyKey;
State state = env.getState(State::new);

if (state.globsDriver == null) {
state.globsDriver =
new Driver(
new GlobsProducer(globsKey, regexPatternCache, (GlobsProducer.ResultSink) state));
}

if (!state.globsDriver.drive(env)) {
GlobException.handleExceptions(state.error);
return null;
}

GlobException.handleExceptions(state.error);
return new GlobsValue(state.globsMatchingResult);
}
}
1 change: 1 addition & 0 deletions src/test/java/com/google/devtools/build/lib/skyframe/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -229,6 +229,7 @@ java_test(
"//src/main/java/com/google/devtools/build/lib/skyframe:glob_descriptor",
"//src/main/java/com/google/devtools/build/lib/skyframe:glob_function",
"//src/main/java/com/google/devtools/build/lib/skyframe:glob_value",
"//src/main/java/com/google/devtools/build/lib/skyframe:globs_function",
"//src/main/java/com/google/devtools/build/lib/skyframe:globs_value",
"//src/main/java/com/google/devtools/build/lib/skyframe:ignored_package_prefixes_function",
"//src/main/java/com/google/devtools/build/lib/skyframe:incompatible_view_exception",
Expand Down
Loading

0 comments on commit ffc8c15

Please sign in to comment.