Automated rollback of commit a6255612e4892729d3758775c76085b26b9bc584.

*** Reason for rollback ***

Breaks FDO builds, see http://b/112112125

*** 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: 207084179
diff --git a/src/main/java/com/google/devtools/build/lib/actions/cache/MetadataHandler.java b/src/main/java/com/google/devtools/build/lib/actions/cache/MetadataHandler.java
index fc61397..62d03fd 100644
--- a/src/main/java/com/google/devtools/build/lib/actions/cache/MetadataHandler.java
+++ b/src/main/java/com/google/devtools/build/lib/actions/cache/MetadataHandler.java
@@ -17,7 +17,6 @@
 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;
 
@@ -31,9 +30,15 @@
  * <p>Note that implementations of this interface call chmod on output files if {@link
  * #discardOutputMetadata} has been called.
  */
-public interface MetadataHandler extends MetadataProvider {
-  @Override
-  FileArtifactValue getMetadata(ActionInput actionInput) throws IOException;
+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;
 
   /** Sets digest for virtual artifacts (e.g. middlemen). {@code md5Digest} must not be null. */
   void setDigestForVirtualArtifact(Artifact artifact, Md5Digest md5Digest);
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/ActionExecutionFunction.java b/src/main/java/com/google/devtools/build/lib/skyframe/ActionExecutionFunction.java
index 4e61f8c..6d8a90d 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/ActionExecutionFunction.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/ActionExecutionFunction.java
@@ -385,13 +385,9 @@
           actionLookupData,
           /* inputDiscoveryRan= */ false);
     }
-    // The metadata handler is recreated below if the action performs input discovery.
-    ActionMetadataHandler metadataHandler = new ActionMetadataHandler(
-        state.inputArtifactData,
-        /* missingArtifactsAllowed= */ action.discoversInputs(),
-        action.getOutputs(),
-        tsgm.get(),
-        pathResolver(state.actionFileSystem));
+    // This may be recreated if we discover inputs.
+    ActionMetadataHandler metadataHandler = new ActionMetadataHandler(state.inputArtifactData,
+        action.getOutputs(), tsgm.get(), pathResolver(state.actionFileSystem));
     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()) {
@@ -426,15 +422,19 @@
     // Delete the metadataHandler's cache of the action's outputs, since they are being deleted.
     metadataHandler.discardOutputMetadata();
 
+    // 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());
     if (action.discoversInputs()) {
       if (state.discoveredInputs == null) {
         try {
           state.updateFileSystemContext(skyframeActionExecutor, env, metadataHandler);
           state.discoveredInputs =
               skyframeActionExecutor.discoverInputs(
-                  action, metadataHandler, env, state.actionFileSystem);
+                  action, perActionFileCache, metadataHandler, env, state.actionFileSystem);
           Preconditions.checkState(
               env.valuesMissing() == (state.discoveredInputs == null),
               "discoverInputs() must return null iff requesting more dependencies.");
@@ -451,13 +451,11 @@
       if (env.valuesMissing()) {
         return null;
       }
+      perActionFileCache =
+          new PerActionFileCache(state.inputArtifactData, /*missingArtifactsAllowed=*/ false);
 
       metadataHandler =
-          new ActionMetadataHandler(
-              state.inputArtifactData,
-              /* missingArtifactsAllowed= */ false,
-              action.getOutputs(),
-              tsgm.get(),
+          new ActionMetadataHandler(state.inputArtifactData, action.getOutputs(), tsgm.get(),
               pathResolver(state.actionFileSystem));
       // Set the MetadataHandler to accept output information.
       metadataHandler.discardOutputMetadata();
@@ -489,6 +487,7 @@
     state.updateFileSystemContext(skyframeActionExecutor, env, metadataHandler);
     try (ActionExecutionContext actionExecutionContext =
         skyframeActionExecutor.getContext(
+            perActionFileCache,
             metadataHandler,
             Collections.unmodifiableMap(state.expandedArtifacts),
             filesetMappings.build(),
@@ -530,11 +529,7 @@
         // markOmitted is only called for remote execution, and this code only gets executed for
         // local execution.
         metadataHandler =
-            new ActionMetadataHandler(
-                state.inputArtifactData,
-                /*missingArtifactsAllowed=*/ false,
-                action.getOutputs(),
-                tsgm.get(),
+            new ActionMetadataHandler(state.inputArtifactData, action.getOutputs(), tsgm.get(),
                 pathResolver(state.actionFileSystem));
       }
     }
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/ActionMetadataHandler.java b/src/main/java/com/google/devtools/build/lib/skyframe/ActionMetadataHandler.java
index 8bf3b15..daf2041 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/ActionMetadataHandler.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/ActionMetadataHandler.java
@@ -83,7 +83,6 @@
    * <p>This should never be read directly. Use {@link #getInputFileArtifactValue} instead.
    */
   private final ActionInputMap inputArtifactData;
-  private final boolean missingArtifactsAllowed;
 
   /** FileValues for each output Artifact. */
   private final ConcurrentMap<Artifact, FileValue> outputArtifactData =
@@ -136,12 +135,10 @@
   @VisibleForTesting
   public ActionMetadataHandler(
       ActionInputMap inputArtifactData,
-      boolean missingArtifactsAllowed,
       Iterable<Artifact> outputs,
       TimestampGranularityMonitor tsgm,
       ArtifactPathResolver artifactPathResolver)  {
     this.inputArtifactData = Preconditions.checkNotNull(inputArtifactData);
-    this.missingArtifactsAllowed = missingArtifactsAllowed;
     this.outputs = ImmutableSet.copyOf(outputs);
     this.tsgm = tsgm;
     this.artifactPathResolver = artifactPathResolver;
@@ -183,25 +180,15 @@
   }
 
   @Override
-  public FileArtifactValue getMetadata(ActionInput actionInput) throws IOException {
-    // TODO(shahan): is this bypass needed?
-    if (!(actionInput instanceof Artifact)) {
-      return null;
-    }
-
-    Artifact artifact = (Artifact) actionInput;
+  public FileArtifactValue getMetadata(Artifact artifact) throws IOException {
     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
@@ -223,11 +210,7 @@
       // Calling code depends on this particular exception.
       throw new FileNotFoundException(artifact + " not found");
     }
-    // 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).
+    // It's an ordinary artifact.
     FileValue fileValue = outputArtifactData.get(artifact);
     if (fileValue != null) {
       // Non-middleman artifacts should only have additionalOutputData if they have
@@ -244,28 +227,12 @@
       }
       return FileArtifactValue.createNormalFile(fileValue);
     }
-
-    // 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.
+    // We do not cache exceptions besides nonexistence here, because it is unlikely that the file
+    // will be requested from this cache too many times.
     fileValue = constructFileValue(artifact, /*statNoFollow=*/ null);
     return maybeStoreAdditionalData(artifact, fileValue, null);
   }
 
-  @Override
-  public ActionInput getInput(String execPath) {
-    return inputArtifactData.getInput(execPath);
-  }
-
   /**
    * Check that the new {@code data} we just calculated for an {@link Artifact} agrees with the
    * {@code oldData} (presumably calculated concurrently), if it was present.
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/PerActionFileCache.java b/src/main/java/com/google/devtools/build/lib/skyframe/PerActionFileCache.java
new file mode 100644
index 0000000..f27bd36
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/PerActionFileCache.java
@@ -0,0 +1,61 @@
+// Copyright 2014 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.base.Preconditions;
+import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.ActionInputMap;
+import com.google.devtools.build.lib.actions.Artifact;
+import com.google.devtools.build.lib.actions.FileArtifactValue;
+import com.google.devtools.build.lib.actions.MetadataProvider;
+import javax.annotation.Nullable;
+
+/**
+ * Cache provided by an {@link ActionExecutionFunction}, allowing Blaze to obtain artifact metadata
+ * from the graph.
+ *
+ * <p>Data for the action's inputs is injected into this cache on construction, using the graph as
+ * the source of truth.
+ */
+class PerActionFileCache implements MetadataProvider {
+  private final ActionInputMap inputArtifactData;
+  private final boolean missingArtifactsAllowed;
+
+  /**
+   * @param inputArtifactData Map from artifact to metadata, used to return metadata upon request.
+   * @param missingArtifactsAllowed whether to tolerate missing artifacts: can happen during input
+   *     discovery.
+   */
+  PerActionFileCache(ActionInputMap inputArtifactData, boolean missingArtifactsAllowed) {
+    this.inputArtifactData = Preconditions.checkNotNull(inputArtifactData);
+    this.missingArtifactsAllowed = missingArtifactsAllowed;
+  }
+
+  @Nullable
+  @Override
+  public FileArtifactValue getMetadata(ActionInput input) {
+    // TODO(shahan): is this bypass needed?
+    if (!(input instanceof Artifact)) {
+      return null;
+    }
+    FileArtifactValue result = inputArtifactData.getMetadata(input);
+    Preconditions.checkState(missingArtifactsAllowed || result != null, "null for %s", input);
+    return result;
+  }
+
+  @Override
+  public ActionInput getInput(String execPath) {
+    return inputArtifactData.getInput(execPath);
+  }
+}
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeActionExecutor.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeActionExecutor.java
index 215e7f0..ee7f348 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeActionExecutor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeActionExecutor.java
@@ -527,6 +527,7 @@
    * tasks related to that action.
    */
   public ActionExecutionContext getContext(
+      MetadataProvider graphFileCache,
       MetadataHandler metadataHandler,
       Map<Artifact, Collection<Artifact>> expandedInputs,
       ImmutableMap<PathFragment, ImmutableList<FilesetOutputSymlink>> inputFilesetMappings,
@@ -535,7 +536,7 @@
         ArtifactPathResolver.createPathResolver(actionFileSystem, executorEngine.getExecRoot()));
     return new ActionExecutionContext(
         executorEngine,
-        createFileCache(metadataHandler, actionFileSystem),
+        createFileCache(graphFileCache, actionFileSystem),
         actionInputPrefetcher,
         actionKeyContext,
         metadataHandler,
@@ -655,6 +656,7 @@
    */
   Iterable<Artifact> discoverInputs(
       Action action,
+      PerActionFileCache graphFileCache,
       MetadataHandler metadataHandler,
       Environment env,
       @Nullable FileSystem actionFileSystem)
@@ -662,7 +664,7 @@
     ActionExecutionContext actionExecutionContext =
         ActionExecutionContext.forInputDiscovery(
             executorEngine,
-            createFileCache(metadataHandler, actionFileSystem),
+            createFileCache(graphFileCache, actionFileSystem),
             actionInputPrefetcher,
             actionKeyContext,
             metadataHandler,
diff --git a/src/test/java/com/google/devtools/build/lib/actions/ActionCacheCheckerTest.java b/src/test/java/com/google/devtools/build/lib/actions/ActionCacheCheckerTest.java
index 817cf38..a84efb3 100644
--- a/src/test/java/com/google/devtools/build/lib/actions/ActionCacheCheckerTest.java
+++ b/src/test/java/com/google/devtools/build/lib/actions/ActionCacheCheckerTest.java
@@ -333,12 +333,8 @@
   /** A fake metadata handler that is able to obtain metadata from the file system. */
   private static class FakeMetadataHandler extends FakeMetadataHandlerBase {
     @Override
-    public FileArtifactValue getMetadata(ActionInput input) throws IOException {
-      if (input instanceof Artifact) {
-        return FileArtifactValue.create((Artifact) input);
-      } else {
-        return null;
-      }
+    public FileArtifactValue getMetadata(Artifact artifact) throws IOException {
+      return FileArtifactValue.create(artifact);
     }
 
     @Override
diff --git a/src/test/java/com/google/devtools/build/lib/actions/util/ActionsTestUtil.java b/src/test/java/com/google/devtools/build/lib/actions/util/ActionsTestUtil.java
index cd3489d..013f57e 100644
--- a/src/test/java/com/google/devtools/build/lib/actions/util/ActionsTestUtil.java
+++ b/src/test/java/com/google/devtools/build/lib/actions/util/ActionsTestUtil.java
@@ -78,7 +78,6 @@
 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;
@@ -710,12 +709,7 @@
    */
   public static class FakeMetadataHandlerBase implements MetadataHandler {
     @Override
-    public FileArtifactValue getMetadata(ActionInput input) throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ActionInput getInput(String execPath) {
+    public FileArtifactValue getMetadata(Artifact artifact) throws IOException {
       throw new UnsupportedOperationException();
     }