Skip to content

Commit

Permalink
Automated rollback of commit c004ff1.
Browse files Browse the repository at this point in the history
*** Reason for rollback ***

Second attempt; rollout with a flag this time.

The original change should have been a no-op, but twerth@ swears that
his binary search narrowed down to it, so we're being super extra
careful this time, having been unable to find the root cause to the
issue that caused the rollback.

*** Original change description ***

Remove PerActionFileCache

Instead, make ActionMetadataHandler implement the MetadataProvider interface.

This fixes an issue where an action that runs two spawns where one depends on
an output of the other was unable to get the metadata for the intermediate
output.

We don't currently have actions that do this, but we will have in a future
change (which will also implicitly act as a regression test).

PiperOrigin-RevId: 214919699
  • Loading branch information
ulfjack authored and Copybara-Service committed Sep 28, 2018
1 parent 030a2f1 commit f2b260c
Show file tree
Hide file tree
Showing 8 changed files with 230 additions and 40 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.google.devtools.build.lib.actions.Artifact;
import com.google.devtools.build.lib.actions.Artifact.TreeFileArtifact;
import com.google.devtools.build.lib.actions.FileArtifactValue;
import com.google.devtools.build.lib.actions.MetadataProvider;
import com.google.devtools.build.lib.vfs.FileStatus;
import java.io.IOException;

Expand All @@ -30,15 +31,9 @@
* <p>Note that implementations of this interface call chmod on output files if {@link
* #discardOutputMetadata} has been called.
*/
public interface MetadataHandler {
/**
* Returns metadata for the given artifact or throws an exception if the metadata could not be
* obtained.
*
* @return metadata instance
* @throws IOException if metadata could not be obtained.
*/
FileArtifactValue getMetadata(Artifact artifact) throws IOException;
public interface MetadataHandler extends MetadataProvider {
@Override
FileArtifactValue getMetadata(ActionInput actionInput) throws IOException;

/** Sets digest for virtual artifacts (e.g. middlemen). {@code md5Digest} must not be null. */
void setDigestForVirtualArtifact(Artifact artifact, Md5Digest md5Digest);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -406,6 +406,20 @@ public boolean useTopLevelTargetsForSymlinks() {
help = "This option is deprecated and has no effect.")
public boolean discardActionsAfterExecution;

@Option(
name = "incompatible_use_per_action_file_cache",
defaultValue = "true",
documentationCategory = OptionDocumentationCategory.UNDOCUMENTED,
effectTags = {OptionEffectTag.BAZEL_INTERNAL_CONFIGURATION},
metadataTags = {
OptionMetadataTag.INCOMPATIBLE_CHANGE,
OptionMetadataTag.TRIGGERED_BY_ALL_INCOMPATIBLE_CHANGES
},
help = "Whether to use the per action file cache. We saw issues with a previous rollout "
+ "attempt (which we could not track down to a root cause), so we are extra careful now "
+ "and use a flag to enable the new code path.")
public boolean usePerActionFileCache;

/** Converter for jobs: [0, MAX_JOBS] or "auto". */
public static class JobsConverter extends RangeConverter {
/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import com.google.devtools.build.lib.actions.ArtifactSkyKey;
import com.google.devtools.build.lib.actions.FileArtifactValue;
import com.google.devtools.build.lib.actions.FilesetOutputSymlink;
import com.google.devtools.build.lib.actions.MetadataProvider;
import com.google.devtools.build.lib.actions.MissingDepException;
import com.google.devtools.build.lib.actions.MissingInputFileException;
import com.google.devtools.build.lib.actions.NotifyOnActionCacheHit;
Expand Down Expand Up @@ -67,7 +68,6 @@
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import javax.annotation.Nullable;

/**
Expand Down Expand Up @@ -383,15 +383,14 @@ private ActionExecutionValue checkCacheAndExecuteIfNeeded(
// The metadataHandler may be recreated (via the supplier) if we discover inputs.
ArtifactPathResolver pathResolver = ArtifactPathResolver.createPathResolver(
state.actionFileSystem, skyframeActionExecutor.getExecRoot());
Supplier<ActionMetadataHandler> metadataHandlerSupplier =
() ->
new ActionMetadataHandler(
state.inputArtifactData,
action.getOutputs(),
tsgm.get(),
pathResolver,
state.actionFileSystem == null ? new OutputStore() : new MinimalOutputStore());
ActionMetadataHandler metadataHandler = metadataHandlerSupplier.get();
ActionMetadataHandler metadataHandler =
new ActionMetadataHandler(
state.inputArtifactData,
/* missingArtifactsAllowed= */ action.discoversInputs(),
action.getOutputs(),
tsgm.get(),
pathResolver,
state.actionFileSystem == null ? new OutputStore() : new MinimalOutputStore());
long actionStartTime = BlazeClock.nanoTime();
// We only need to check the action cache if we haven't done it on a previous run.
if (!state.hasCheckedActionCache()) {
Expand Down Expand Up @@ -427,9 +426,10 @@ private ActionExecutionValue checkCacheAndExecuteIfNeeded(
// This may be recreated if we discover inputs.
// TODO(shahan): this isn't used when using ActionFileSystem so we can avoid creating some
// unused objects.
PerActionFileCache perActionFileCache =
new PerActionFileCache(
state.inputArtifactData, /*missingArtifactsAllowed=*/ action.discoversInputs());
MetadataProvider perActionFileCache = skyframeActionExecutor.usePerFileActionCache()
? new PerActionFileCache(
state.inputArtifactData, /*missingArtifactsAllowed=*/ action.discoversInputs())
: metadataHandler;
if (action.discoversInputs()) {
if (state.discoveredInputs == null) {
try {
Expand Down Expand Up @@ -459,12 +459,20 @@ private ActionExecutionValue checkCacheAndExecuteIfNeeded(
if (env.valuesMissing()) {
return null;
}
perActionFileCache =
new PerActionFileCache(state.inputArtifactData, /*missingArtifactsAllowed=*/ false);

metadataHandler = metadataHandlerSupplier.get();
metadataHandler =
new ActionMetadataHandler(
state.inputArtifactData,
/* missingArtifactsAllowed= */ false,
action.getOutputs(),
tsgm.get(),
pathResolver,
state.actionFileSystem == null ? new OutputStore() : new MinimalOutputStore());
// Set the MetadataHandler to accept output information.
metadataHandler.discardOutputMetadata();

perActionFileCache = skyframeActionExecutor.usePerFileActionCache()
? new PerActionFileCache(state.inputArtifactData, /*missingArtifactsAllowed=*/ false)
: metadataHandler;
}

// Make sure this is a regular HashMap rather than ImmutableMapBuilder so that we are safe
Expand Down Expand Up @@ -542,7 +550,14 @@ private ActionExecutionValue checkCacheAndExecuteIfNeeded(
// the documentation on MetadataHandler.artifactOmitted. This works by accident because
// markOmitted is only called for remote execution, and this code only gets executed for
// local execution.
metadataHandler = metadataHandlerSupplier.get();
metadataHandler =
new ActionMetadataHandler(
state.inputArtifactData,
/*missingArtifactsAllowed=*/ false,
action.getOutputs(),
tsgm.get(),
pathResolver,
state.actionFileSystem == null ? new OutputStore() : new MinimalOutputStore());
}
}
Preconditions.checkState(!env.valuesMissing(), action);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ public final class ActionMetadataHandler implements MetadataHandler {
* <p>This should never be read directly. Use {@link #getInputFileArtifactValue} instead.
*/
private final ActionInputMap inputArtifactData;
private final boolean missingArtifactsAllowed;

/** Outputs that are to be omitted. */
private final Set<Artifact> omittedOutputs = Sets.newConcurrentHashSet();
Expand All @@ -90,6 +91,7 @@ public final class ActionMetadataHandler implements MetadataHandler {
* The timestamp granularity monitor for this build.
* Use {@link #getTimestampGranularityMonitor(Artifact)} to fetch this member.
*/
@Nullable
private final TimestampGranularityMonitor tsgm;
private final ArtifactPathResolver artifactPathResolver;

Expand All @@ -104,11 +106,13 @@ public final class ActionMetadataHandler implements MetadataHandler {
@VisibleForTesting
public ActionMetadataHandler(
ActionInputMap inputArtifactData,
boolean missingArtifactsAllowed,
Iterable<Artifact> outputs,
TimestampGranularityMonitor tsgm,
@Nullable TimestampGranularityMonitor tsgm,
ArtifactPathResolver artifactPathResolver,
OutputStore store) {
this.inputArtifactData = Preconditions.checkNotNull(inputArtifactData);
this.missingArtifactsAllowed = missingArtifactsAllowed;
this.outputs = ImmutableSet.copyOf(outputs);
this.tsgm = tsgm;
this.artifactPathResolver = artifactPathResolver;
Expand All @@ -124,6 +128,7 @@ public ActionMetadataHandler(
* @param artifact the artifact for which to fetch the timestamp granularity monitor
* @return the timestamp granularity monitor to use, which may be null
*/
@Nullable
private TimestampGranularityMonitor getTimestampGranularityMonitor(Artifact artifact) {
return artifact.isConstantMetadata() ? null : tsgm;
}
Expand Down Expand Up @@ -151,15 +156,25 @@ private FileArtifactValue getInputFileArtifactValue(Artifact input) {
}

@Override
public FileArtifactValue getMetadata(Artifact artifact) throws IOException {
public FileArtifactValue getMetadata(ActionInput actionInput) throws IOException {
// TODO(shahan): is this bypass needed?
if (!(actionInput instanceof Artifact)) {
return null;
}

Artifact artifact = (Artifact) actionInput;
FileArtifactValue value = getInputFileArtifactValue(artifact);
if (value != null) {
return metadataFromValue(value);
}

if (artifact.isSourceArtifact()) {
// A discovered input we didn't have data for.
// TODO(bazel-team): Change this to an assertion once Skyframe has native input discovery, so
// all inputs will already have metadata known.
if (!missingArtifactsAllowed) {
throw new IllegalStateException(String.format("null for %s", artifact));
}
return null;
} else if (artifact.isMiddlemanArtifact()) {
// A middleman artifact's data was either already injected from the action cache checker using
Expand All @@ -180,7 +195,11 @@ public FileArtifactValue getMetadata(Artifact artifact) throws IOException {
// Calling code depends on this particular exception.
throw new FileNotFoundException(artifact + " not found");
}
// It's an ordinary artifact.
// Fallthrough: the artifact must be a non-tree, non-middleman output artifact.

// Check for existing metadata. It may have been injected. In either case, this method is called
// from SkyframeActionExecutor to make sure that we have metadata for all action outputs, as the
// results are then stored in Skyframe (and the action cache).
ArtifactFileMetadata fileMetadata = store.getArtifactData(artifact);
if (fileMetadata != null) {
// Non-middleman artifacts should only have additionalOutputData if they have
Expand All @@ -197,14 +216,30 @@ public FileArtifactValue getMetadata(Artifact artifact) throws IOException {
}
return FileArtifactValue.createNormalFile(fileMetadata);
}
// We do not cache exceptions besides nonexistence here, because it is unlikely that the file
// will be requested from this cache too many times.

// No existing metadata; this can happen if the output metadata is not injected after a spawn
// is executed. SkyframeActionExecutor.checkOutputs calls this method for every output file of
// the action, which hits this code path. Another possibility is that an action runs multiple
// spawns, and a subsequent spawn requests the metadata of an output of a previous spawn.
//
// Stat the file. All output artifacts of an action are deleted before execution, so if a file
// exists, it was most likely created by the current action. There is a race condition here if
// an external process creates (or modifies) the file between the deletion and this stat, which
// we cannot solve.
//
// We only cache nonexistence here, not file system errors. It is unlikely that the file will be
// requested from this cache too many times.
fileMetadata = constructArtifactFileMetadata(artifact, /*statNoFollow=*/ null);
return maybeStoreAdditionalData(artifact, fileMetadata, null);
}

@Override
public ActionInput getInput(String execPath) {
return inputArtifactData.getInput(execPath);
}

/**
* See {@link Outputstore#getAllAdditionalOutputData} for why we sometimes need to store
* See {@link OutputStore#getAllAdditionalOutputData} for why we sometimes need to store
* additional data, even for normal (non-middleman) artifacts.
*/
@Nullable
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,6 +155,7 @@ public final class SkyframeActionExecutor {
// findAndStoreArtifactConflicts, and is preserved across builds otherwise.
private ImmutableMap<ActionAnalysisMetadata, ConflictException> badActionMap = ImmutableMap.of();
private OptionsProvider options;
private boolean usePerFileActionCache;
private boolean hadExecutionError;
private MetadataProvider perBuildFileCache;
private ActionInputPrefetcher actionInputPrefetcher;
Expand Down Expand Up @@ -358,6 +359,8 @@ void prepareForExecution(
this.actionCacheChecker = Preconditions.checkNotNull(actionCacheChecker);
// Don't cache possibly stale data from the last build.
this.options = options;
this.usePerFileActionCache =
options.getOptions(BuildRequestOptions.class).usePerActionFileCache;
// Cache the finalizeActions value for performance, since we consult it on every action.
this.finalizeActions = options.getOptions(BuildRequestOptions.class).finalizeActions;
this.outputService = outputService;
Expand All @@ -373,6 +376,10 @@ public void setClientEnv(Map<String, String> clientEnv) {
this.clientEnv = ImmutableMap.copyOf(clientEnv);
}

boolean usePerFileActionCache() {
return usePerFileActionCache;
}

boolean usesActionFileSystem() {
return outputService != null && outputService.supportsActionFileSystem();
}
Expand Down Expand Up @@ -509,7 +516,7 @@ ActionExecutionValue executeAction(
* tasks related to that action.
*/
public ActionExecutionContext getContext(
MetadataProvider graphFileCache,
MetadataProvider perActionFileCache,
MetadataHandler metadataHandler,
Map<Artifact, Collection<Artifact>> expandedInputs,
Map<Artifact, ImmutableList<FilesetOutputSymlink>> expandedFilesets,
Expand All @@ -520,7 +527,7 @@ public ActionExecutionContext getContext(
ArtifactPathResolver.createPathResolver(actionFileSystem, executorEngine.getExecRoot()));
return new ActionExecutionContext(
executorEngine,
createFileCache(graphFileCache, actionFileSystem),
createFileCache(perActionFileCache, actionFileSystem),
actionInputPrefetcher,
actionKeyContext,
metadataHandler,
Expand Down Expand Up @@ -643,15 +650,15 @@ Iterable<Artifact> getActionCachedInputs(Action action, PackageRootResolver reso
*/
Iterable<Artifact> discoverInputs(
Action action,
PerActionFileCache graphFileCache,
MetadataProvider perActionFileCache,
MetadataHandler metadataHandler,
Environment env,
@Nullable FileSystem actionFileSystem)
throws ActionExecutionException, InterruptedException {
ActionExecutionContext actionExecutionContext =
ActionExecutionContext.forInputDiscovery(
executorEngine,
createFileCache(graphFileCache, actionFileSystem),
createFileCache(perActionFileCache, actionFileSystem),
actionInputPrefetcher,
actionKeyContext,
metadataHandler,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -333,8 +333,11 @@ public MiddlemanType getActionType() {
/** A fake metadata handler that is able to obtain metadata from the file system. */
private static class FakeMetadataHandler extends FakeMetadataHandlerBase {
@Override
public FileArtifactValue getMetadata(Artifact artifact) throws IOException {
return FileArtifactValue.create(artifact);
public FileArtifactValue getMetadata(ActionInput input) throws IOException {
if (!(input instanceof Artifact)) {
return null;
}
return FileArtifactValue.create((Artifact) input);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@
import com.google.devtools.build.skyframe.BuildDriver;
import com.google.devtools.build.skyframe.ErrorInfo;
import com.google.devtools.build.skyframe.EvaluationResult;
import com.google.devtools.build.skyframe.SkyFunction;
import com.google.devtools.build.skyframe.SkyKey;
import com.google.devtools.build.skyframe.SkyValue;
import com.google.devtools.build.skyframe.ValueOrUntypedException;
Expand Down Expand Up @@ -712,7 +713,12 @@ public Path getPathFromSourceExecPath(PathFragment execPath) {
*/
public static class FakeMetadataHandlerBase implements MetadataHandler {
@Override
public FileArtifactValue getMetadata(Artifact artifact) throws IOException {
public FileArtifactValue getMetadata(ActionInput input) throws IOException {
throw new UnsupportedOperationException();
}

@Override
public ActionInput getInput(String execPath) {
throw new UnsupportedOperationException();
}

Expand Down
Loading

0 comments on commit f2b260c

Please sign in to comment.