Deja-vu: Using an ActionInputFileCache for SHA1 digests used with remote execution.
If you're feeling like you've already seen this, that's correct, these were the exact contents of commit e860316559eac366d47923a8eb4b5489a661aa35... and then, on Nov 15, something unclear happened and the code disappeared! Perhaps it was the result of a faulty sync. In any case, nobody noticed, and the CL went in. It was later rolloed back and resubmitted, but the crucial code changes were gone.

TESTED=local server with profiling for SHA1 specifically
RELNOTES: n/a

--
PiperOrigin-RevId: 151139685
MOS_MIGRATED_REVID=151139685
diff --git a/src/main/java/com/google/devtools/build/lib/remote/ConcurrentMapActionCache.java b/src/main/java/com/google/devtools/build/lib/remote/ConcurrentMapActionCache.java
index 45b4474..8127575 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/ConcurrentMapActionCache.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/ConcurrentMapActionCache.java
@@ -15,6 +15,8 @@
 package com.google.devtools.build.lib.remote;
 
 import com.google.common.collect.ImmutableList;
+import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.ActionInputFileCache;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
 import com.google.devtools.build.lib.remote.ContentDigests.ActionKey;
 import com.google.devtools.build.lib.remote.RemoteProtocol.ActionResult;
@@ -60,7 +62,7 @@
       uploadBlob(fileNode.toByteArray());
     }
     for (TreeNode leaf : repository.leaves(root)) {
-      uploadFileContents(execRoot.getRelative(leaf.getActionInput().getExecPathString()));
+      uploadFileContents(leaf.getActionInput(), execRoot, repository.getInputFileCache());
     }
   }
 
@@ -84,6 +86,17 @@
   }
 
   @Override
+  public ContentDigest uploadFileContents(
+      ActionInput input, Path execRoot, ActionInputFileCache inputCache)
+      throws IOException, InterruptedException {
+    // This unconditionally reads the whole file into memory first!
+    return uploadBlob(
+        ByteString.readFrom(execRoot.getRelative(input.getExecPathString()).getInputStream())
+            .toByteArray(),
+        ContentDigests.getDigestFromInputCache(input, inputCache));
+  }
+
+  @Override
   public void downloadAllResults(ActionResult result, Path execRoot)
       throws IOException, CacheNotFoundException {
     for (Output output : result.getOutputList()) {
@@ -149,9 +162,12 @@
 
   @Override
   public ContentDigest uploadBlob(byte[] blob) throws InterruptedException {
+    return uploadBlob(blob, ContentDigests.computeDigest(blob));
+  }
+
+  private ContentDigest uploadBlob(byte[] blob, ContentDigest digest) throws InterruptedException {
     int blobSizeKBytes = blob.length / 1024;
     checkBlobSize(blobSizeKBytes, "Upload");
-    ContentDigest digest = ContentDigests.computeDigest(blob);
     uploadMemoryAvailable.acquire(blobSizeKBytes);
     try {
       cache.put(ContentDigests.toHexString(digest), blob);
diff --git a/src/main/java/com/google/devtools/build/lib/remote/ContentDigests.java b/src/main/java/com/google/devtools/build/lib/remote/ContentDigests.java
index e7772b2..d192598 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/ContentDigests.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/ContentDigests.java
@@ -16,6 +16,8 @@
 
 import com.google.common.hash.HashCode;
 import com.google.common.hash.Hashing;
+import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.ActionInputFileCache;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
 import com.google.devtools.build.lib.remote.RemoteProtocol.Action;
 import com.google.devtools.build.lib.remote.RemoteProtocol.ContentDigest;
@@ -81,6 +83,11 @@
     return b.build();
   }
 
+  public static ContentDigest getDigestFromInputCache(ActionInput input, ActionInputFileCache cache)
+      throws IOException {
+    return buildDigest(cache.getDigest(input), cache.getSizeInBytes(input));
+  }
+
   public static String toHexString(ContentDigest digest) {
     return HashCode.fromBytes(digest.getDigest().toByteArray()).toString();
   }
diff --git a/src/main/java/com/google/devtools/build/lib/remote/GrpcActionCache.java b/src/main/java/com/google/devtools/build/lib/remote/GrpcActionCache.java
index 4372932..1076933 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/GrpcActionCache.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/GrpcActionCache.java
@@ -19,6 +19,7 @@
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterators;
 import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.ActionInputFileCache;
 import com.google.devtools.build.lib.analysis.config.InvalidConfigurationException;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
 import com.google.devtools.build.lib.remote.CasServiceGrpc.CasServiceBlockingStub;
@@ -216,6 +217,83 @@
     }
   }
 
+  final class BlobChunkActionInputIterator implements BlobChunkIterator {
+    private final Iterator<? extends ActionInput> inputIterator;
+    private final ActionInputFileCache inputCache;
+    private final Path execRoot;
+    private InputStream currentStream;
+    private final Set<ContentDigest> digests;
+    private ContentDigest digest;
+    private long bytesLeft;
+
+    public BlobChunkActionInputIterator(
+        Set<ContentDigest> digests,
+        Path execRoot,
+        Iterator<? extends ActionInput> inputIterator,
+        ActionInputFileCache inputCache)
+        throws IOException {
+      this.digests = digests;
+      this.inputIterator = inputIterator;
+      this.inputCache = inputCache;
+      this.execRoot = execRoot;
+      advanceInput();
+    }
+
+    public BlobChunkActionInputIterator(
+        ActionInput input, Path execRoot, ActionInputFileCache inputCache) throws IOException {
+      inputIterator = Iterators.singletonIterator(input);
+      digests = ImmutableSet.of(ContentDigests.getDigestFromInputCache(input, inputCache));
+      this.inputCache = inputCache;
+      this.execRoot = execRoot;
+      advanceInput();
+    }
+
+    private void advanceInput() throws IOException {
+      do {
+        if (inputIterator != null && inputIterator.hasNext()) {
+          ActionInput input = inputIterator.next();
+          digest = ContentDigests.getDigestFromInputCache(input, inputCache);
+          currentStream = execRoot.getRelative(input.getExecPathString()).getInputStream();
+          bytesLeft = digest.getSizeBytes();
+        } else {
+          digest = null;
+          currentStream = null;
+          bytesLeft = 0;
+        }
+      } while (digest != null && !digests.contains(digest));
+    }
+
+    @Override
+    public boolean hasNext() {
+      return currentStream != null;
+    }
+
+    @Override
+    public BlobChunk next() throws IOException {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      BlobChunk.Builder chunk = BlobChunk.newBuilder();
+      long offset = digest.getSizeBytes() - bytesLeft;
+      if (offset == 0) {
+        chunk.setDigest(digest);
+      } else {
+        chunk.setOffset(offset);
+      }
+      if (bytesLeft > 0) {
+        byte[] blob = new byte[(int) Math.min(bytesLeft, (long) options.grpcMaxChunkSizeBytes)];
+        currentStream.read(blob);
+        chunk.setData(ByteString.copyFrom(blob));
+        bytesLeft -= blob.length;
+      }
+      if (bytesLeft == 0) {
+        currentStream.close();
+        advanceInput();
+      }
+      return chunk.build();
+    }
+  }
+
   @VisibleForTesting
   public GrpcActionCache(ManagedChannel channel, RemoteOptions options) {
     this.options = options;
@@ -278,11 +356,10 @@
       }
     }
     if (!actionInputs.isEmpty()) {
-      ArrayList<Path> paths = new ArrayList<>();
-      for (ActionInput actionInput : actionInputs) {
-        paths.add(execRoot.getRelative(actionInput.getExecPathString()));
-      }
-      uploadChunks(paths.size(), new BlobChunkFileIterator(missingDigests, paths.iterator()));
+      uploadChunks(
+          actionInputs.size(),
+          new BlobChunkActionInputIterator(
+              missingDigests, execRoot, actionInputs.iterator(), repository.getInputFileCache()));
     }
   }
 
@@ -412,8 +489,7 @@
 
   /**
    * Put the file contents cache if it is not already in it. No-op if the file is already stored in
-   * cache. The given path must be a full absolute path. Note: this is horribly inefficient, need to
-   * patch through an overload that uses an ActionInputFile cache to compute the digests!
+   * cache. The given path must be a full absolute path.
    *
    * @return The key for fetching the file contents blob from cache.
    */
@@ -428,6 +504,24 @@
   }
 
   /**
+   * Put the file contents cache if it is not already in it. No-op if the file is already stored in
+   * cache. The given path must be a full absolute path.
+   *
+   * @return The key for fetching the file contents blob from cache.
+   */
+  @Override
+  public ContentDigest uploadFileContents(
+      ActionInput input, Path execRoot, ActionInputFileCache inputCache)
+      throws IOException, InterruptedException {
+    ContentDigest digest = ContentDigests.getDigestFromInputCache(input, inputCache);
+    ImmutableSet<ContentDigest> missing = getMissingDigests(ImmutableList.of(digest));
+    if (!missing.isEmpty()) {
+      uploadChunks(1, new BlobChunkActionInputIterator(input, execRoot, inputCache));
+    }
+    return digest;
+  }
+
+  /**
    * Download a blob keyed by the given digest and write it to the specified path. Set the
    * executable parameter to the specified value.
    */
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteActionCache.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteActionCache.java
index cc0720b..6dff9b6 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteActionCache.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteActionCache.java
@@ -15,6 +15,8 @@
 package com.google.devtools.build.lib.remote;
 
 import com.google.common.collect.ImmutableList;
+import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.ActionInputFileCache;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadCompatible;
 import com.google.devtools.build.lib.remote.ContentDigests.ActionKey;
 import com.google.devtools.build.lib.remote.RemoteProtocol.ActionResult;
@@ -69,6 +71,16 @@
   ContentDigest uploadFileContents(Path file) throws IOException, InterruptedException;
 
   /**
+   * Put the input file contents in cache if it is not already in it. No-op if the data is already
+   * stored in cache.
+   *
+   * @return The key for fetching the file contents blob from cache.
+   */
+  ContentDigest uploadFileContents(
+      ActionInput input, Path execRoot, ActionInputFileCache inputCache)
+      throws IOException, InterruptedException;
+
+  /**
    * Download a blob keyed by the given digest and write it to the specified path. Set the
    * executable parameter to the specified value.
    */
diff --git a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnStrategy.java b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnStrategy.java
index c8a8643..ed0f0ea 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnStrategy.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/RemoteSpawnStrategy.java
@@ -21,6 +21,7 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.devtools.build.lib.actions.ActionExecutionContext;
 import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.ActionInputFileCache;
 import com.google.devtools.build.lib.actions.ActionStatusMessage;
 import com.google.devtools.build.lib.actions.ExecException;
 import com.google.devtools.build.lib.actions.ExecutionStrategy;
@@ -233,7 +234,8 @@
 
     try {
       // Temporary hack: the TreeNodeRepository should be created and maintained upstream!
-      TreeNodeRepository repository = new TreeNodeRepository(execRoot);
+      ActionInputFileCache inputFileCache = actionExecutionContext.getActionInputFileCache();
+      TreeNodeRepository repository = new TreeNodeRepository(execRoot, inputFileCache);
       SortedMap<PathFragment, ActionInput> inputMap =
           spawnInputExpander.getInputMapping(
               spawn,
diff --git a/src/main/java/com/google/devtools/build/lib/remote/TreeNodeRepository.java b/src/main/java/com/google/devtools/build/lib/remote/TreeNodeRepository.java
index 8f66301..89c61c2 100644
--- a/src/main/java/com/google/devtools/build/lib/remote/TreeNodeRepository.java
+++ b/src/main/java/com/google/devtools/build/lib/remote/TreeNodeRepository.java
@@ -23,6 +23,7 @@
 import com.google.common.collect.Iterables;
 import com.google.common.collect.TreeTraverser;
 import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.ActionInputFileCache;
 import com.google.devtools.build.lib.concurrent.BlazeInterners;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.Immutable;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
@@ -31,6 +32,7 @@
 import com.google.devtools.build.lib.util.Preconditions;
 import com.google.devtools.build.lib.vfs.Path;
 import com.google.devtools.build.lib.vfs.PathFragment;
+import com.google.protobuf.ByteString;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -177,14 +179,18 @@
   // Merkle hashes are computed and cached by the repository, therefore execRoot must
   // be part of the state.
   private final Path execRoot;
-  private final Map<ActionInput, ContentDigest> fileContentsDigestCache = new HashMap<>();
-  private final Map<ContentDigest, ActionInput> digestFileContentsCache = new HashMap<>();
+  private final ActionInputFileCache inputFileCache;
   private final Map<TreeNode, ContentDigest> treeNodeDigestCache = new HashMap<>();
   private final Map<ContentDigest, TreeNode> digestTreeNodeCache = new HashMap<>();
   private final Map<TreeNode, FileNode> fileNodeCache = new HashMap<>();
 
-  public TreeNodeRepository(Path execRoot) {
+  public TreeNodeRepository(Path execRoot, ActionInputFileCache inputFileCache) {
     this.execRoot = execRoot;
+    this.inputFileCache = inputFileCache;
+  }
+
+  public ActionInputFileCache getInputFileCache() {
+    return inputFileCache;
   }
 
   @Override
@@ -272,29 +278,16 @@
     return interner.intern(new TreeNode(entries));
   }
 
-  private synchronized ContentDigest getOrComputeActionInputDigest(ActionInput actionInput)
-      throws IOException {
-    ContentDigest digest = fileContentsDigestCache.get(actionInput);
-    if (digest == null) {
-      digest = ContentDigests.computeDigest(execRoot.getRelative(actionInput.getExecPathString()));
-      fileContentsDigestCache.put(actionInput, digest);
-      digestFileContentsCache.put(digest, actionInput);
-    }
-    return digest;
-  }
-
   private synchronized FileNode getOrComputeFileNode(TreeNode node) throws IOException {
     // Assumes all child digests have already been computed!
     FileNode fileNode = fileNodeCache.get(node);
     if (fileNode == null) {
       FileNode.Builder b = FileNode.newBuilder();
       if (node.isLeaf()) {
-        ContentDigest fileDigest = fileContentsDigestCache.get(node.getActionInput());
-        Preconditions.checkState(fileDigest != null);
+        ActionInput input = node.getActionInput();
         b.getFileMetadataBuilder()
-            .setDigest(fileDigest)
-            .setExecutable(
-                execRoot.getRelative(node.getActionInput().getExecPathString()).isExecutable());
+            .setDigest(ContentDigests.getDigestFromInputCache(input, inputFileCache))
+            .setExecutable(execRoot.getRelative(input.getExecPathString()).isExecutable());
       } else {
         for (TreeNode.ChildEntry entry : node.getChildEntries()) {
           ContentDigest childDigest = treeNodeDigestCache.get(entry.getChild());
@@ -321,7 +314,8 @@
       }
     }
     if (root.isLeaf()) {
-      getOrComputeActionInputDigest(root.getActionInput());
+      // Load the digest into the ActionInputFileCache.
+      inputFileCache.getDigest(root.getActionInput());
     } else {
       for (TreeNode child : children(root)) {
         computeMerkleDigests(child);
@@ -342,12 +336,12 @@
    * Returns the precomputed digests for both data and metadata. Should only be used after
    * computeMerkleDigests has been called on one of the node ancestors.
    */
-  public ImmutableCollection<ContentDigest> getAllDigests(TreeNode root) {
+  public ImmutableCollection<ContentDigest> getAllDigests(TreeNode root) throws IOException {
     ImmutableSet.Builder<ContentDigest> digests = ImmutableSet.builder();
     for (TreeNode node : descendants(root)) {
       digests.add(Preconditions.checkNotNull(treeNodeDigestCache.get(node)));
       if (node.isLeaf()) {
-        digests.add(Preconditions.checkNotNull(fileContentsDigestCache.get(node.getActionInput())));
+        digests.add(ContentDigests.getDigestFromInputCache(node.getActionInput(), inputFileCache));
       }
     }
     return digests.build();
@@ -379,7 +373,8 @@
       if (treeNode != null) {
         nodes.add(Preconditions.checkNotNull(fileNodeCache.get(treeNode)));
       } else { // If not there, it must be an ActionInput.
-        actionInputs.add(Preconditions.checkNotNull(digestFileContentsCache.get(digest)));
+        ByteString hexDigest = ByteString.copyFromUtf8(ContentDigests.toHexString(digest));
+        actionInputs.add(Preconditions.checkNotNull(inputFileCache.getInputFromDigest(hexDigest)));
       }
     }
   }
diff --git a/src/test/java/com/google/devtools/build/lib/remote/TreeNodeRepositoryTest.java b/src/test/java/com/google/devtools/build/lib/remote/TreeNodeRepositoryTest.java
index ed3bbfd..db4e55d 100644
--- a/src/test/java/com/google/devtools/build/lib/remote/TreeNodeRepositoryTest.java
+++ b/src/test/java/com/google/devtools/build/lib/remote/TreeNodeRepositoryTest.java
@@ -18,12 +18,17 @@
 import com.google.common.collect.ImmutableCollection;
 import com.google.common.collect.ImmutableList;
 import com.google.devtools.build.lib.actions.ActionInput;
+import com.google.devtools.build.lib.actions.ActionInputFileCache;
 import com.google.devtools.build.lib.actions.Artifact;
 import com.google.devtools.build.lib.actions.Root;
+import com.google.devtools.build.lib.exec.SingleBuildFileCache;
 import com.google.devtools.build.lib.remote.RemoteProtocol.ContentDigest;
 import com.google.devtools.build.lib.remote.RemoteProtocol.FileNode;
 import com.google.devtools.build.lib.remote.TreeNodeRepository.TreeNode;
 import com.google.devtools.build.lib.testutil.Scratch;
+import com.google.devtools.build.lib.vfs.FileSystem;
+import com.google.devtools.build.lib.vfs.FileSystem.HashFunction;
+import com.google.devtools.build.lib.vfs.Path;
 import java.util.ArrayList;
 import org.junit.Before;
 import org.junit.Test;
@@ -35,11 +40,20 @@
 public class TreeNodeRepositoryTest {
   private Scratch scratch;
   private Root rootDir;
+  private Path rootPath;
 
   @Before
   public final void setRootDir() throws Exception {
+    FileSystem.setDigestFunctionForTesting(HashFunction.SHA1);
     scratch = new Scratch();
     rootDir = Root.asDerivedRoot(scratch.dir("/exec/root"));
+    rootPath = rootDir.getPath();
+  }
+
+  private TreeNodeRepository createTestTreeNodeRepository() {
+    ActionInputFileCache inputFileCache = new SingleBuildFileCache(
+        rootPath.getPathString(), scratch.getFileSystem());
+    return new TreeNodeRepository(rootPath, inputFileCache);
   }
 
   @Test
@@ -49,7 +63,7 @@
     Artifact fooH = new Artifact(scratch.file("/exec/root/a/foo.h"), rootDir);
     Artifact bar = new Artifact(scratch.file("/exec/root/b/bar.txt"), rootDir);
     Artifact baz = new Artifact(scratch.file("/exec/root/c/baz.txt"), rootDir);
-    TreeNodeRepository repo = new TreeNodeRepository(rootDir.getPath());
+    TreeNodeRepository repo = createTestTreeNodeRepository();
     TreeNode root1 = repo.buildFromActionInputs(ImmutableList.<ActionInput>of(fooCc, fooH, bar));
     TreeNode root2 = repo.buildFromActionInputs(ImmutableList.<ActionInput>of(fooCc, fooH, baz));
     // Reusing same node for the "a" subtree.
@@ -62,7 +76,7 @@
   public void testMerkleDigests() throws Exception {
     Artifact foo = new Artifact(scratch.file("/exec/root/a/foo", "1"), rootDir);
     Artifact bar = new Artifact(scratch.file("/exec/root/a/bar", "11"), rootDir);
-    TreeNodeRepository repo = new TreeNodeRepository(rootDir.getPath());
+    TreeNodeRepository repo = createTestTreeNodeRepository();
     TreeNode root = repo.buildFromActionInputs(ImmutableList.<ActionInput>of(foo, bar));
     TreeNode aNode = root.getChildEntries().get(0).getChild();
     TreeNode fooNode = aNode.getChildEntries().get(1).getChild(); // foo > bar in sort order!
@@ -104,7 +118,7 @@
     Artifact foo1 = new Artifact(scratch.file("/exec/root/a/foo", "1"), rootDir);
     Artifact foo2 = new Artifact(scratch.file("/exec/root/b/foo", "1"), rootDir);
     Artifact foo3 = new Artifact(scratch.file("/exec/root/c/foo", "1"), rootDir);
-    TreeNodeRepository repo = new TreeNodeRepository(rootDir.getPath());
+    TreeNodeRepository repo = createTestTreeNodeRepository();
     TreeNode root = repo.buildFromActionInputs(ImmutableList.<ActionInput>of(foo1, foo2, foo3));
     repo.computeMerkleDigests(root);
     // Reusing same node for the "foo" subtree: only need the root, root child, foo, and contents: