Skip to content

Commit

Permalink
Fix a race condition in IncrementalPackageRoots.
Browse files Browse the repository at this point in the history
Consider the following scenario:

- Top level targets `A` and `B`
- To execute actions in `A`, we need to plant symlinks for `NestedSet<Package>: [A1, C1, [D1]]`,
- To execute actions in `B`, we need to plant symlinks for `NestedSet<Package>: [B1, C1, [D1]]`

In the end, we expect to see symlinks to `A1`, `B1`, `C1` and `D1`.

**What went wrong**

With the current code, there are 2 possible race conditions:

1. Transitive NestedSet level:
- Start to plant symlinks for `A`
- `NestedSet [D1]` added to `handledPackageNestedSets`. _No symlink planted yet_.
- Start to plant symlinks for `B`
- `NestedSet [D1]` seen as "handled" and immediately skipped. Planted `B1` and `C1`. `B` moves on to execution.
=> actions from `B` that requires `D1` would fail (no such file or directory).

2. Individual symlink level:
- Start to plant symlinks for `A`
- `C1` added to `lazilyPlantedSymlinks`. _No symlink planted yet_.
- Start to plant symlinks for `B`
- `C1` already seen in `lazilyPlantedSymlinks` and immediately skipped. Planted `B1` and `D1`. `B` moves on to execution.
=> actions from `B` that requires `C1` would fail (no such file or directory).

**The Solution**

In order to prevent this race condition, we can plant the symlinks for top level targets `A` and `B` sequentially. This gives us the guarantee that: for an action `foo` under a top level target `A`, `foo` is only executed when all the necessary symlinks for `A` are already planted.

The above scenario would look like:
- Start to plant symlinks for `A`
- The `TopLevelTargetReadyForSymlinkPlanting` event for `B` arrived and is held in the sequential event queue
- Plant all symlinks. `lazilyPlantedSymlinks: [A1, C1, D1]`. `A` moves on to execution.
- Start to plant symlinks for `B`
- `NestedSet [D1]` already seen in `handledPackageNestedSets` and immediately skipped.
- `C1` already seen in `lazilyPlantedSymlinks` and immediately skipped.
- Planted `B1`. `B` moves on to execution.

As an (hopefully not premature) optimization, the symlinks under a single top level target are planted in parallel.

Fixes #22073

Verified locally with something similar to the repro in #22073 (comment).

PiperOrigin-RevId: 628080361
Change-Id: Ic6c1a6606d26400c46aa98bfeddc844abd075d0a
  • Loading branch information
joeleba authored and copybara-github committed Apr 25, 2024
1 parent 83cfbe8 commit 52adf0b
Show file tree
Hide file tree
Showing 2 changed files with 127 additions and 64 deletions.
1 change: 1 addition & 0 deletions src/main/java/com/google/devtools/build/lib/skyframe/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -1617,6 +1617,7 @@ java_library(
"//src/main/java/com/google/devtools/build/lib/analysis:analysis_phase_complete_event",
"//src/main/java/com/google/devtools/build/lib/cmdline",
"//src/main/java/com/google/devtools/build/lib/collect/nestedset",
"//src/main/java/com/google/devtools/build/lib/concurrent",
"//src/main/java/com/google/devtools/build/lib/packages",
"//src/main/java/com/google/devtools/build/lib/util:abrupt_exit_exception",
"//src/main/java/com/google/devtools/build/lib/util:detailed_exit_code",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,21 +13,29 @@
// limitations under the License.
package com.google.devtools.build.lib.skyframe;

import static com.google.common.util.concurrent.MoreExecutors.directExecutor;

import com.google.common.base.Ascii;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.eventbus.AllowConcurrentEvents;
import com.google.common.eventbus.EventBus;
import com.google.common.eventbus.Subscribe;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.devtools.build.lib.actions.PackageRoots;
import com.google.devtools.build.lib.analysis.AnalysisPhaseCompleteEvent;
import com.google.devtools.build.lib.buildtool.SymlinkForest;
import com.google.devtools.build.lib.buildtool.SymlinkForest.SymlinkPlantingException;
import com.google.devtools.build.lib.cmdline.PackageIdentifier;
import com.google.devtools.build.lib.collect.nestedset.NestedSet;
import com.google.devtools.build.lib.collect.nestedset.NestedSet.Node;
import com.google.devtools.build.lib.concurrent.ExecutorUtil;
import com.google.devtools.build.lib.packages.Package;
import com.google.devtools.build.lib.server.FailureDetails;
import com.google.devtools.build.lib.server.FailureDetails.FailureDetail;
Expand All @@ -37,9 +45,13 @@
import com.google.devtools.build.lib.vfs.Path;
import com.google.devtools.build.lib.vfs.Root;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;

Expand All @@ -56,13 +68,14 @@ public class IncrementalPackageRoots implements PackageRoots {

@GuardedBy("stateLock")
@Nullable
private Set<NestedSet.Node> handledPackageNestedSets = Sets.newConcurrentHashSet();
private Set<NestedSet.Node> donePackages = Sets.newConcurrentHashSet();

// Only tracks the symlinks lazily planted after the first eager planting wave.
@GuardedBy("stateLock")
@Nullable
private Set<Path> lazilyPlantedSymlinks = Sets.newConcurrentHashSet();

private final ListeningExecutorService symlinkPlantingPool;
private final Object stateLock = new Object();
private final Path execroot;
private final Root singleSourceRoot;
Expand Down Expand Up @@ -93,6 +106,11 @@ private IncrementalPackageRoots(
this.eventBus = eventBus;
this.useSiblingRepositoryLayout = useSiblingRepositoryLayout;
this.allowExternalRepositories = allowExternalRepositories;
this.symlinkPlantingPool =
MoreExecutors.listeningDecorator(
Executors.newFixedThreadPool(
Runtime.getRuntime().availableProcessors(),
new ThreadFactoryBuilder().setNameFormat("Non-eager Symlink planter %d").build()));
}

public static IncrementalPackageRoots createAndRegisterToEventBus(
Expand Down Expand Up @@ -171,12 +189,27 @@ public PackageRootLookup getPackageRootLookup() {
: threadSafeExternalRepoPackageRootsMap.get(packageId);
}

@AllowConcurrentEvents
// Intentionally don't allow concurrent events here to prevent a race condition between planting
// a symlink and starting an action that requires that symlink. This race condition is possible
// because of the various memoizations we use to avoid repeated work.
@Subscribe
public void topLevelTargetReadyForSymlinkPlanting(TopLevelTargetReadyForSymlinkPlanting event)
throws AbruptExitException {
if (allowExternalRepositories || !maybeConflictingBaseNamesLowercase.isEmpty()) {
registerAndPlantMissingSymlinks(event.transitivePackagesForSymlinkPlanting());
Set<NestedSet.Node> donePackagesLocalRef;
Set<Path> lazilyPlantedSymlinksLocalRef;
// May still race with analysisFinished, hence the synchronization.
synchronized (stateLock) {
if (donePackages == null || lazilyPlantedSymlinks == null) {
return;
}
donePackagesLocalRef = donePackages;
lazilyPlantedSymlinksLocalRef = lazilyPlantedSymlinks;
}
registerAndPlantMissingSymlinks(
event.transitivePackagesForSymlinkPlanting(),
donePackagesLocalRef,
lazilyPlantedSymlinksLocalRef);
}
}

Expand All @@ -191,83 +224,105 @@ public void analysisFinished(AnalysisPhaseCompleteEvent unused) {
* <p>There are 2 possibilities: either we're planting symlinks to the external repos, or there's
* potentially conflicting symlinks detected.
*/
private void registerAndPlantMissingSymlinks(NestedSet<Package> packages)
private void registerAndPlantMissingSymlinks(
NestedSet<Package> packages, Set<Node> donePackagesRef, Set<Path> lazilyPlantedSymlinksRef)
throws AbruptExitException {
Set<Path> lazilyPlantedSymlinksLocalRef;
synchronized (stateLock) {
if (handledPackageNestedSets == null || !handledPackageNestedSets.add(packages.toNode())) {
// Optimization to prune subsequent traversals.
// A false negative does not affect correctness.
if (donePackagesRef.contains(packages.toNode())) {
return;
}

List<ListenableFuture<Void>> futures = new ArrayList<>(packages.getLeaves().size());
synchronized (symlinkPlantingPool) {
// Some other thread shut down the executor, exit now.
if (symlinkPlantingPool.isShutdown()) {
return;
}
lazilyPlantedSymlinksLocalRef = lazilyPlantedSymlinks;
if (lazilyPlantedSymlinksLocalRef == null) {
return;
for (Package pkg : packages.getLeaves()) {
futures.add(
symlinkPlantingPool.submit(
() -> plantSingleSymlinkForPackage(pkg, lazilyPlantedSymlinksRef)));
}
}
for (NestedSet<Package> transitive : packages.getNonLeaves()) {
registerAndPlantMissingSymlinks(transitive, donePackagesRef, lazilyPlantedSymlinksRef);
}
// Now wait on the futures.
try {
Futures.whenAllSucceed(futures).call(() -> null, directExecutor()).get();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return; // Bail
} catch (ExecutionException e) {
if (e.getCause() instanceof AbruptExitException) {
throw (AbruptExitException) e.getCause();
}
throw new IllegalStateException("Unexpected exception", e);
}
// Only update the memoization set now, after the symlinks are confirmed planted.
donePackagesRef.add(packages.toNode());
}

// To reach this point, this has to be the first and only time we plant the symlinks for this
// NestedSet<Package>. That means it's not possible to reach this after analysis has ended.
private Void plantSingleSymlinkForPackage(Package pkg, Set<Path> lazilyPlantedSymlinksRef)
throws AbruptExitException {
try {
for (Package pkg : packages.getLeaves()) {
PackageIdentifier pkgId = pkg.getPackageIdentifier();
if (isExternalRepository(pkgId) && pkg.getSourceRoot().isPresent()) {
threadSafeExternalRepoPackageRootsMap.put(
pkg.getPackageIdentifier(), pkg.getSourceRoot().get());
SymlinkForest.plantSingleSymlinkForExternalRepo(
pkgId.getRepository(),
pkg.getSourceRoot().get().asPath(),
execroot,
useSiblingRepositoryLayout,
lazilyPlantedSymlinksLocalRef);
} else if (!maybeConflictingBaseNamesLowercase.isEmpty()) {
String originalBaseName = pkgId.getTopLevelDir();
String baseNameLowercase = Ascii.toLowerCase(originalBaseName);
PackageIdentifier pkgId = pkg.getPackageIdentifier();
if (isExternalRepository(pkgId) && pkg.getSourceRoot().isPresent()) {
threadSafeExternalRepoPackageRootsMap.putIfAbsent(
pkg.getPackageIdentifier(), pkg.getSourceRoot().get());
SymlinkForest.plantSingleSymlinkForExternalRepo(
pkgId.getRepository(),
pkg.getSourceRoot().get().asPath(),
execroot,
useSiblingRepositoryLayout,
lazilyPlantedSymlinksRef);
} else if (!maybeConflictingBaseNamesLowercase.isEmpty()) {
String originalBaseName = pkgId.getTopLevelDir();
String baseNameLowercase = Ascii.toLowerCase(originalBaseName);

// As Skymeld only supports single package path at the moment, we only seek to symlink to
// the top-level dir i.e. what's directly under the source root.
Path link = execroot.getRelative(originalBaseName);
Path target = singleSourceRoot.getRelative(originalBaseName);
// As Skymeld only supports single package path at the moment, we only seek to symlink to
// the top-level dir i.e. what's directly under the source root.
Path link = execroot.getRelative(originalBaseName);
Path target = singleSourceRoot.getRelative(originalBaseName);

if (originalBaseName.isEmpty()
|| !maybeConflictingBaseNamesLowercase.contains(baseNameLowercase)
|| !SymlinkForest.symlinkShouldBePlanted(
prefix, ignoredPaths, useSiblingRepositoryLayout, originalBaseName, target)) {
// We should have already eagerly planted a symlink for this, or there's nothing to do.
continue;
}
if (originalBaseName.isEmpty()
|| !maybeConflictingBaseNamesLowercase.contains(baseNameLowercase)
|| !SymlinkForest.symlinkShouldBePlanted(
prefix, ignoredPaths, useSiblingRepositoryLayout, originalBaseName, target)) {
// We should have already eagerly planted a symlink for this, or there's nothing to do.
return null;
}

if (lazilyPlantedSymlinksLocalRef.add(link)) {
try {
link.createSymbolicLink(target);
} catch (IOException e) {
StringBuilder errorMessage =
new StringBuilder(
String.format("Failed to plant a symlink: %s -> %s", link, target));
if (link.exists() && link.isSymbolicLink()) {
// If the link already exists, it must mean that we're planting from a
// case-insensitive file system and this is a legitimate conflict.
// TODO(b/295300378) We technically can go deeper here and try to create the subdirs
// to try to resolve the conflict, but the complexity isn't worth it at the moment
// and the non-skymeld code path isn't doing any better. Revisit if necessary.
Path existingTarget = link.resolveSymbolicLinks();
if (!existingTarget.equals(target)) {
errorMessage.append(
String.format(
". Found an existing conflicting symlink: %s -> %s",
link, existingTarget));
}
if (lazilyPlantedSymlinksRef.add(link)) {
try {
link.createSymbolicLink(target);
} catch (IOException e) {
StringBuilder errorMessage =
new StringBuilder(
String.format("Failed to plant a symlink: %s -> %s", link, target));
if (link.exists() && link.isSymbolicLink()) {
// If the link already exists, it must mean that we're planting from a
// case-insensitive file system and this is a legitimate conflict.
// TODO(b/295300378) We technically can go deeper here and try to create the subdirs
// to try to resolve the conflict, but the complexity isn't worth it at the moment
// and the non-skymeld code path isn't doing any better. Revisit if necessary.
Path existingTarget = link.resolveSymbolicLinks();
if (!existingTarget.equals(target)) {
errorMessage.append(
String.format(
". Found an existing conflicting symlink: %s -> %s", link, existingTarget));
}

throw new SymlinkPlantingException(errorMessage.toString(), e);
}

throw new SymlinkPlantingException(errorMessage.toString(), e);
}
}
}
} catch (IOException | SymlinkPlantingException e) {
throwAbruptExitException(e);
}
for (NestedSet<Package> transitive : packages.getNonLeaves()) {
registerAndPlantMissingSymlinks(transitive);
}
return null;
}

private static void throwAbruptExitException(Exception e) throws AbruptExitException {
Expand Down Expand Up @@ -299,9 +354,16 @@ private void dropIntermediateStatesAndUnregisterFromEventBus() {
eventBus = null;

synchronized (stateLock) {
handledPackageNestedSets = null;
donePackages = null;
lazilyPlantedSymlinks = null;
maybeConflictingBaseNamesLowercase = ImmutableSet.of();
}
synchronized (symlinkPlantingPool) {
if (!symlinkPlantingPool.isShutdown()
&& ExecutorUtil.interruptibleShutdown(symlinkPlantingPool)) {
// Preserve the interrupt status.
Thread.currentThread().interrupt();
}
}
}
}

0 comments on commit 52adf0b

Please sign in to comment.