Skip to content

Commit

Permalink
Fix the case where if all strategies for one branch of dynamic exec…
Browse files Browse the repository at this point in the history
…ution fail to accept (that is, refuse to even take) the action given, the whole action fails. Instead of seeing whether the other branch can run and the action that that it succeeded.

NOTE: This is _not_ about the local/remote execution branch trying to run it and failing. Obviously that should fail the whole dynamic execution. This is about none of the strategies of one branch even stating that they can run it (based on what they return from `canExec`)

If the local execution branch requires a specific architecture/OS for some action, and users' bazel is being run a machine that isn't that combination (assuming the local strategy given is already setup to return `false` in `canExec` in this case).

Previously the whole execution of the dynamic execution would fail without even trying to see if remote execution succeeded. Now it will gracefully fallback to just waiting on remote strategy.

To conditionally use workers for supported actions but still simultaneously kick off a dynamic run, the options`--stragegy=TaskMnemonic=dynamic --dynamic_local_strategy=TaskMnemonic=worker --dynamic_remote_strategy=TaskMnemonic=remote` can be given.
Then any action with with the mnemonic `TaskMnemonic` that can't support `worker` execution (xor `remote` execution) will wait for the other execution branch to complete. If neither set of strategies can run the action, then the task fails.

Previously, this would have failed if, for example, the `worker` strategy cannot handle the action, even if `remote` could have.

This at first glance seems silly as if you know TaskMnemonic doesn't have a worker enabled implementation, why specify `worker` as the local strategy?
But keep in mind any action can declare most any mnemonic. In this example, say that first rule doing a TaskMnemonic is worker enabled, so you add the flags. But then someone makes a bazel/starlark rule with the same mnemonic but different implementation (for example, a "mimic" rule), and that new one doesn't support worker. Then this becomes a case of "partial" worker support for a mnemonic.

RELNOTES: If all strategies of one branch (the local or remote execution branch) of the `dynamic` strategy fail to even accept (via the response they give from `canExec`) the action, `dynamic` will now try to see if the other branch can accept it. (Trying to run it and it failing will still cause a failure if it was the first result, this is about strategies claiming they can't even try the action)
PiperOrigin-RevId: 374265582
  • Loading branch information
Googler authored and larsrc-google committed Jul 30, 2021
1 parent 687fd52 commit 1962a59
Show file tree
Hide file tree
Showing 5 changed files with 405 additions and 48 deletions.
3 changes: 2 additions & 1 deletion CONTRIBUTORS
Original file line number Diff line number Diff line change
Expand Up @@ -108,4 +108,5 @@ Jonathan Dierksen <dierksen@google.com>
Tony Aiuto <aiuto@google.com>
Andy Scott <andy.g.scott@gmail.com>
Jamie Snape <jamie.snape@kitware.com>
Irina Chernushina <ichern@google.com>
Irina Chernushina <ichern@google.com>
C. Sean Young <csyoung@google.com>
Original file line number Diff line number Diff line change
Expand Up @@ -35,14 +35,17 @@
import com.google.devtools.build.lib.actions.SpawnResult;
import com.google.devtools.build.lib.actions.SpawnResult.Status;
import com.google.devtools.build.lib.actions.SpawnStrategy;
import com.google.devtools.build.lib.actions.UserExecException;
import com.google.devtools.build.lib.events.Event;
import com.google.devtools.build.lib.exec.ExecutionPolicy;
import com.google.devtools.build.lib.server.FailureDetails;
import com.google.devtools.build.lib.server.FailureDetails.DynamicExecution;
import com.google.devtools.build.lib.server.FailureDetails.DynamicExecution.Code;
import com.google.devtools.build.lib.server.FailureDetails.FailureDetail;
import com.google.devtools.build.lib.util.io.FileOutErr;
import com.google.devtools.build.lib.vfs.Path;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException;
Expand Down Expand Up @@ -265,12 +268,17 @@ private static ImmutableList<SpawnResult> waitBranches(
ImmutableList<SpawnResult> remoteResult = waitBranch(remoteBranch);

if (remoteResult != null && localResult != null) {
throw new AssertionError("One branch did not cancel the other one");
} else if (remoteResult != null) {
return remoteResult;
throw new AssertionError(
String.format(
"Neither branch of %s cancelled the other one.",
spawn.getResourceOwner().getPrimaryOutput().prettyPrint()));
} else if (localResult != null) {
return localResult;
} else if (remoteResult != null) {
return remoteResult;
} else {
// TODO(b/173153395): Sometimes gets thrown for currently unknown reasons.
// (sometimes happens in relation to the whole dynamic execution being cancelled)
throw new AssertionError(
"Neither branch completed. Local was "
+ (localBranch.isCancelled() ? "" : "not ")
Expand Down Expand Up @@ -321,18 +329,142 @@ static void verifyAvailabilityInfo(DynamicExecutionOptions options, Spawn spawn)
}
}

private static boolean canExecLocalSpawn(
Spawn spawn,
ExecutionPolicy executionPolicy,
ActionContext.ActionContextRegistry actionContextRegistry,
DynamicStrategyRegistry dynamicStrategyRegistry) {
if (!executionPolicy.canRunLocally()) {
return false;
}
List<SandboxedSpawnStrategy> localStrategies =
dynamicStrategyRegistry.getDynamicSpawnActionContexts(
spawn, DynamicStrategyRegistry.DynamicMode.LOCAL);
return localStrategies.stream()
.anyMatch(
s ->
(s.canExec(spawn, actionContextRegistry)
|| s.canExecWithLegacyFallback(spawn, actionContextRegistry)));
}

private boolean canExecLocal(
Spawn spawn,
ExecutionPolicy mainSpawnExecutionPolicy,
ActionContext.ActionContextRegistry actionContextRegistry,
DynamicStrategyRegistry dynamicStrategyRegistry) {
if (!canExecLocalSpawn(
spawn, mainSpawnExecutionPolicy, actionContextRegistry, dynamicStrategyRegistry)) {
return false;
}
// Present if there is a extra local spawn. Unset if not.
Optional<Boolean> canLocalSpawn =
getExtraSpawnForLocalExecution
.apply(spawn)
.map(
extraSpawn ->
canExecLocalSpawn(
extraSpawn,
getExecutionPolicy.apply(extraSpawn),
actionContextRegistry,
dynamicStrategyRegistry));
return canLocalSpawn.orElse(true);
}

private static boolean canExecRemote(
Spawn spawn,
ExecutionPolicy executionPolicy,
ActionContext.ActionContextRegistry actionContextRegistry,
DynamicStrategyRegistry dynamicStrategyRegistry) {
if (!executionPolicy.canRunRemotely()) {
return false;
}
List<SandboxedSpawnStrategy> remoteStrategies =
dynamicStrategyRegistry.getDynamicSpawnActionContexts(
spawn, DynamicStrategyRegistry.DynamicMode.REMOTE);
return remoteStrategies.stream().anyMatch(s -> s.canExec(spawn, actionContextRegistry));
}

@Override
public boolean canExec(Spawn spawn, ActionContext.ActionContextRegistry actionContextRegistry) {
ExecutionPolicy executionPolicy = getExecutionPolicy.apply(spawn);
DynamicStrategyRegistry dynamicStrategyRegistry =
actionContextRegistry.getContext(DynamicStrategyRegistry.class);

return canExecLocal(spawn, executionPolicy, actionContextRegistry, dynamicStrategyRegistry)
|| canExecRemote(spawn, executionPolicy, actionContextRegistry, dynamicStrategyRegistry);
}

/**
* Returns an error string for being unable to execute locally and/or remotely the given execution
* state.
*
* <p>Usage note, this method is only to be called after an impossible condition is already
* detected by the caller, as all this does is give an error string to put in the exception.
*
* @param spawn The action that needs to be executed
* @param localAllowedBySpawnExecutionPolicy whether the execution policy for this spawn allows
* trying local execution.
* @param remoteAllowedBySpawnExecutionPolicy whether the execution policy for this spawn allows
* trying remote execution.
*/
private static String getNoCanExecFailureMessage(
Spawn spawn,
boolean localAllowedBySpawnExecutionPolicy,
boolean remoteAllowedBySpawnExecutionPolicy) {
// TODO(b/188387840): Can't use Spawn.toString() here because tests report FakeOwner instances
// as the resource owner, and those cause toStrings to throw if no primary output.
// TODO(b/188402092): Even if the above is fixed, we still don't want to use Spawn.toString()
// until the mnemonic is included in the output unconditionally. Too useful for the error
// message.
if (!localAllowedBySpawnExecutionPolicy && !remoteAllowedBySpawnExecutionPolicy) {
return "Neither local nor remote execution allowed for action " + spawn.getMnemonic();
} else if (!remoteAllowedBySpawnExecutionPolicy) {
return "No usable dynamic_local_strategy found (and remote execution disabled) for action "
+ spawn.getMnemonic();
} else if (!localAllowedBySpawnExecutionPolicy) {
return "No usable dynamic_remote_strategy found (and local execution disabled) for action "
+ spawn.getMnemonic();
} else {
return "No usable dynamic_local_strategy or dynamic_remote_strategy found for action "
+ spawn.getMnemonic();
}
}

@Override
public ImmutableList<SpawnResult> exec(
final Spawn spawn, final ActionExecutionContext actionExecutionContext)
throws ExecException, InterruptedException {
DynamicSpawnStrategy.verifyAvailabilityInfo(options, spawn);
ExecutionPolicy executionPolicy = getExecutionPolicy.apply(spawn);
if (executionPolicy.canRunLocallyOnly()) {
return runLocally(spawn, actionExecutionContext, null);
}
if (executionPolicy.canRunRemotelyOnly()) {

DynamicStrategyRegistry dynamicStrategyRegistry =
actionExecutionContext.getContext(DynamicStrategyRegistry.class);
boolean localCanExec =
canExecLocal(spawn, executionPolicy, actionExecutionContext, dynamicStrategyRegistry);

boolean remoteCanExec =
canExecRemote(spawn, executionPolicy, actionExecutionContext, dynamicStrategyRegistry);

if (!localCanExec && !remoteCanExec) {
FailureDetail failure =
FailureDetail.newBuilder()
.setMessage(
getNoCanExecFailureMessage(
spawn, executionPolicy.canRunLocally(), executionPolicy.canRunRemotely()))
.setDynamicExecution(
DynamicExecution.newBuilder().setCode(Code.NO_USABLE_STRATEGY_FOUND).build())
.setSpawn(
FailureDetails.Spawn.newBuilder()
.setCode(FailureDetails.Spawn.Code.NO_USABLE_STRATEGY_FOUND)
.build())
.build();
throw new UserExecException(failure);
} else if (!localCanExec && remoteCanExec) {
return runRemotely(spawn, actionExecutionContext, null);
} else if (localCanExec && !remoteCanExec) {
return runLocally(spawn, actionExecutionContext, null);
}
// else both can exec. Fallthrough to below.

// Semaphores to track termination of each branch. These are necessary to wait for the branch to
// finish its own cleanup (e.g. terminating subprocesses) once it has been cancelled.
Expand Down Expand Up @@ -448,28 +580,6 @@ public ImmutableList<SpawnResult> callImpl(ActionExecutionContext context)
}
}

@Override
public boolean canExec(Spawn spawn, ActionContext.ActionContextRegistry actionContextRegistry) {
DynamicStrategyRegistry dynamicStrategyRegistry =
actionContextRegistry.getContext(DynamicStrategyRegistry.class);
for (SandboxedSpawnStrategy strategy :
dynamicStrategyRegistry.getDynamicSpawnActionContexts(
spawn, DynamicStrategyRegistry.DynamicMode.LOCAL)) {
if (strategy.canExec(spawn, actionContextRegistry)
|| strategy.canExecWithLegacyFallback(spawn, actionContextRegistry)) {
return true;
}
}
for (SandboxedSpawnStrategy strategy :
dynamicStrategyRegistry.getDynamicSpawnActionContexts(
spawn, DynamicStrategyRegistry.DynamicMode.REMOTE)) {
if (strategy.canExec(spawn, actionContextRegistry)) {
return true;
}
}
return false;
}

@Override
public void usedContext(ActionContext.ActionContextRegistry actionContextRegistry) {
actionContextRegistry
Expand All @@ -486,6 +596,12 @@ private static FileOutErr getSuffixedFileOutErr(FileOutErr fileOutErr, String su
outDir.getChild(outBaseName + suffix), errDir.getChild(errBaseName + suffix));
}

/**
* Try to run the given spawn locally.
*
* <p>Precondition: At least one {@code dynamic_local_strategy} returns {@code true} from its
* {@link SpawnStrategy#canExec canExec} method for the given {@code spawn}.
*/
private ImmutableList<SpawnResult> runLocally(
Spawn spawn,
ActionExecutionContext actionExecutionContext,
Expand Down Expand Up @@ -529,12 +645,15 @@ private static ImmutableList<SpawnResult> runSpawnLocally(
return strategy.exec(spawn, actionExecutionContext, stopConcurrentSpawns);
}
}
throw new RuntimeException(
String.format(
"executorCreated not yet called or no default dynamic_local_strategy set for %s",
spawn.getMnemonic()));
throw new AssertionError("canExec passed but no usable local strategy for action " + spawn);
}

/**
* Try to run the given spawn locally.
*
* <p>Precondition: At least one {@code dynamic_remote_strategy} returns {@code true} from its
* {@link SpawnStrategy#canExec canExec} method for the given {@code spawn}.
*/
private static ImmutableList<SpawnResult> runRemotely(
Spawn spawn,
ActionExecutionContext actionExecutionContext,
Expand All @@ -550,10 +669,7 @@ private static ImmutableList<SpawnResult> runRemotely(
return strategy.exec(spawn, actionExecutionContext, stopConcurrentSpawns);
}
}
throw new RuntimeException(
String.format(
"executorCreated not yet called or no default dynamic_remote_strategy set for %s",
spawn.getMnemonic()));
throw new AssertionError("canExec passed but no usable remote strategy for action " + spawn);
}

/**
Expand Down
1 change: 1 addition & 0 deletions src/main/protobuf/failure_details.proto
Original file line number Diff line number Diff line change
Expand Up @@ -1036,6 +1036,7 @@ message DynamicExecution {
XCODE_RELATED_PREREQ_UNMET = 1 [(metadata) = { exit_code: 36 }];
ACTION_LOG_MOVE_FAILURE = 2 [(metadata) = { exit_code: 1 }];
RUN_FAILURE = 3 [(metadata) = { exit_code: 1 }];
NO_USABLE_STRATEGY_FOUND = 4 [(metadata) = { exit_code: 2 }];
}

Code code = 1;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,14 +137,22 @@ private class MockSpawnStrategy implements SandboxedSpawnStrategy {

private final DoExec doExecAfterStop;

private final boolean canExec;

MockSpawnStrategy(String name) {
this(name, DoExec.NOTHING, DoExec.NOTHING);
}

MockSpawnStrategy(String name, DoExec doExecBeforeStop, DoExec doExecAfterStop) {
this(name, doExecBeforeStop, doExecAfterStop, true);
}

MockSpawnStrategy(
String name, DoExec doExecBeforeStop, DoExec doExecAfterStop, boolean canExec) {
this.name = name;
this.doExecBeforeStop = doExecBeforeStop;
this.doExecAfterStop = doExecAfterStop;
this.canExec = canExec;
}

/** Helper to record an execution failure from within {@link #doExecBeforeStop}. */
Expand Down Expand Up @@ -201,7 +209,7 @@ public ImmutableList<SpawnResult> exec(

@Override
public boolean canExec(Spawn spawn, ActionContext.ActionContextRegistry actionContextRegistry) {
return true;
return canExec;
}

@Nullable
Expand Down Expand Up @@ -561,6 +569,48 @@ public void actionSucceedsIfRemoteExecutionSucceedsEvenIfLocalFailsLater() throw
assertThat(outErr.outAsLatin1()).doesNotContain("MockLocalSpawnStrategy");
}

@Test
public void actionSucceedsIfLocalExecutionSucceedsEvenIfRemoteRunsNothing() throws Exception {
MockSpawnStrategy localStrategy = new MockSpawnStrategy("MockLocalSpawnStrategy");

MockSpawnStrategy remoteStrategy =
new MockSpawnStrategy("MockRemoteSpawnStrategy", DoExec.NOTHING, DoExec.NOTHING, false);

StrategyAndContext strategyAndContext = createSpawnStrategy(localStrategy, remoteStrategy);

Spawn spawn = newDynamicSpawn();
strategyAndContext.exec(spawn);

assertThat(localStrategy.getExecutedSpawn()).isEqualTo(spawn);
assertThat(localStrategy.succeeded()).isTrue();
assertThat(remoteStrategy.getExecutedSpawn()).isNull();
assertThat(remoteStrategy.succeeded()).isFalse();

assertThat(outErr.outAsLatin1()).contains("output files written with MockLocalSpawnStrategy");
assertThat(outErr.outAsLatin1()).doesNotContain("MockRemoteSpawnStrategy");
}

@Test
public void actionSucceedsIfRemoteExecutionSucceedsEvenIfLocalRunsNothing() throws Exception {
MockSpawnStrategy localStrategy =
new MockSpawnStrategy("MockLocalSpawnStrategy", DoExec.NOTHING, DoExec.NOTHING, false);

MockSpawnStrategy remoteStrategy = new MockSpawnStrategy("MockRemoteSpawnStrategy");

StrategyAndContext strategyAndContext = createSpawnStrategy(localStrategy, remoteStrategy);

Spawn spawn = newDynamicSpawn();
strategyAndContext.exec(spawn);

assertThat(localStrategy.getExecutedSpawn()).isNull();
assertThat(localStrategy.succeeded()).isFalse();
assertThat(remoteStrategy.getExecutedSpawn()).isEqualTo(spawn);
assertThat(remoteStrategy.succeeded()).isTrue();

assertThat(outErr.outAsLatin1()).contains("output files written with MockRemoteSpawnStrategy");
assertThat(outErr.outAsLatin1()).doesNotContain("MockLocalSpawnStrategy");
}

@Test
public void actionFailsIfLocalFailsImmediatelyEvenIfRemoteSucceedsLater() throws Exception {
CountDownLatch countDownLatch = new CountDownLatch(2);
Expand Down Expand Up @@ -671,6 +721,34 @@ public void actionFailsIfLocalAndRemoteFail() throws Exception {
assertThat(remoteStrategy.succeeded()).isFalse();
}

@Test
public void actionFailsIfLocalAndRemoteRunNothing() throws Exception {
MockSpawnStrategy localStrategy =
new MockSpawnStrategy("MockLocalSpawnStrategy", DoExec.NOTHING, DoExec.NOTHING, false);

MockSpawnStrategy remoteStrategy =
new MockSpawnStrategy("MockRemoteSpawnStrategy", DoExec.NOTHING, DoExec.NOTHING, false);

StrategyAndContext strategyAndContext = createSpawnStrategy(localStrategy, remoteStrategy);

Spawn spawn = newDynamicSpawn();
ExecException e = assertThrows(UserExecException.class, () -> strategyAndContext.exec(spawn));

// Has "No usable", followed by both dynamic_local_strategy and dynamic_remote_strategy in,
// followed by the action's mnemonic.
String regexMatch =
"[nN]o usable\\b.*\\bdynamic_local_strategy\\b.*\\bdynamic_remote_strategy\\b.*\\b"
+ spawn.getMnemonic()
+ "\\b";

assertThat(e).hasMessageThat().containsMatch(regexMatch);

assertThat(localStrategy.getExecutedSpawn()).isNull();
assertThat(localStrategy.succeeded()).isFalse();
assertThat(remoteStrategy.getExecutedSpawn()).isNull();
assertThat(remoteStrategy.succeeded()).isFalse();
}

@Test
public void stopConcurrentSpawnsWaitForCompletion() throws Exception {
if (legacyBehavior) {
Expand Down
Loading

0 comments on commit 1962a59

Please sign in to comment.