Introduce the apply-objects REST-API for the whole '/meta' chain

When the parent commit of a '/meta' chain of NoteDb changes is missing
falling back to the Git fetch command would not help, because of the
inability of the Git fetch negotiation to find a common base.
Replication is not transactional and not guaranteed to succeed which
may lead some refs being left behind with missing parent commits.

The Git fetch negotiation is the process where the two peers are
trying to agree on a common base to use for producing a packfile with
the delta from there. However, the common base may not exist for
the '/meta' chain of commits because every NoteDb ref starts from
a brand new base and the parent commits are missing.
JGit client isn't able to communicate this hint to the server-side
and would instead start a long negotiation phase that is eventually
destinated to fail.

Introduce a new apply-objects REST-API that is capable of applying
an entire chain of commits on the linear '/meta' ref chain so that the
Git fetch can be avoided altogether.

Bug: Issue 16052
Change-Id: I7cfbeeaba2fe6190b99fe78c7631bb884abe321f
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueue.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueue.java
index ef1edcb..f3e38fd 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueue.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueue.java
@@ -15,9 +15,11 @@
 package com.googlesource.gerrit.plugins.replication.pull;
 
 import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Queues;
 import com.google.gerrit.entities.Project;
 import com.google.gerrit.entities.Project.NameKey;
+import com.google.gerrit.entities.RefNames;
 import com.google.gerrit.extensions.events.GitReferenceUpdatedListener;
 import com.google.gerrit.extensions.events.HeadUpdatedListener;
 import com.google.gerrit.extensions.events.LifecycleListener;
@@ -37,7 +39,10 @@
 import com.googlesource.gerrit.plugins.replication.pull.filter.ExcludedRefsFilter;
 import java.io.IOException;
 import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Optional;
 import java.util.Queue;
 import java.util.Set;
@@ -47,7 +52,11 @@
 import java.util.concurrent.TimeoutException;
 import java.util.function.Consumer;
 import org.apache.http.client.ClientProtocolException;
+import org.eclipse.jgit.errors.CorruptObjectException;
+import org.eclipse.jgit.errors.IncorrectObjectTypeException;
 import org.eclipse.jgit.errors.InvalidObjectIdException;
+import org.eclipse.jgit.errors.MissingObjectException;
+import org.eclipse.jgit.errors.RepositoryNotFoundException;
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.transport.URIish;
 import org.slf4j.Logger;
@@ -75,7 +84,7 @@
   private FetchApiClient.Factory fetchClientFactory;
   private Integer fetchCallsTimeout;
   private ExcludedRefsFilter refsFilter;
-  private RevisionReader revisionReader;
+  private Provider<RevisionReader> revReaderProvider;
   private final ApplyObjectMetrics applyObjectMetrics;
   private final FetchReplicationMetrics fetchMetrics;
 
@@ -87,7 +96,7 @@
       ReplicationStateListeners sl,
       FetchApiClient.Factory fetchClientFactory,
       ExcludedRefsFilter refsFilter,
-      RevisionReader revReader,
+      Provider<RevisionReader> revReaderProvider,
       ApplyObjectMetrics applyObjectMetrics,
       FetchReplicationMetrics fetchMetrics) {
     workQueue = wq;
@@ -97,7 +106,7 @@
     beforeStartupEventsQueue = Queues.newConcurrentLinkedQueue();
     this.fetchClientFactory = fetchClientFactory;
     this.refsFilter = refsFilter;
-    this.revisionReader = revReader;
+    this.revReaderProvider = revReaderProvider;
     this.applyObjectMetrics = applyObjectMetrics;
     this.fetchMetrics = fetchMetrics;
   }
@@ -261,7 +270,8 @@
     }
 
     try {
-      Optional<RevisionData> revisionData = revisionReader.read(project, objectId, refName);
+      Optional<RevisionData> revisionData =
+          revReaderProvider.get().read(project, objectId, refName, 0);
       repLog.info(
           "RevisionData is {} for {}:{}",
           revisionData.map(RevisionData::toString).orElse("ABSENT"),
@@ -270,7 +280,8 @@
 
       if (revisionData.isPresent()) {
         return ((source) ->
-            callSendObject(source, project, refName, isDelete, revisionData.get(), state));
+            callSendObject(
+                source, project, refName, isDelete, Arrays.asList(revisionData.get()), state));
       }
     } catch (InvalidObjectIdException | IOException e) {
       stateLog.error(
@@ -289,7 +300,7 @@
       Project.NameKey project,
       String refName,
       boolean isDelete,
-      RevisionData revision,
+      List<RevisionData> revision,
       ReplicationState state)
       throws MissingParentObjectException {
     boolean resultIsSuccessful = true;
@@ -305,7 +316,11 @@
               refName,
               revision);
           Context<String> apiTimer = applyObjectMetrics.startEnd2End(source.getRemoteConfigName());
-          HttpResult result = fetchClient.callSendObject(project, refName, isDelete, revision, uri);
+          HttpResult result =
+              isDelete
+                  ? fetchClient.callSendObject(project, refName, isDelete, null, uri)
+                  : fetchClient.callSendObjects(project, refName, revision, uri);
+          boolean resultSuccessful = result.isSuccessful();
           repLog.info(
               "Pull replication REST API apply object to {} COMPLETED for {}:{} - {}, HTTP Result:"
                   + " {} - time:{} ms",
@@ -316,19 +331,34 @@
               result,
               apiTimer.stop() / 1000000.0);
 
-          if (isProjectMissing(result, project) && source.isCreateMissingRepositories()) {
+          if (!resultSuccessful
+              && result.isProjectMissing(project)
+              && source.isCreateMissingRepositories()) {
             result = initProject(project, uri, fetchClient, result);
             repLog.info("Missing project {} created, HTTP Result:{}", project, result);
           }
 
-          if (!result.isSuccessful()) {
+          if (!resultSuccessful) {
             if (result.isParentObjectMissing()) {
+
+              if (RefNames.isNoteDbMetaRef(refName) && revision.size() == 1) {
+                List<RevisionData> allRevisions =
+                    fetchWholeMetaHistory(project, refName, revision.get(0));
+                repLog.info(
+                    "Pull replication REST API apply object to {} for {}:{} - {}",
+                    apiUrl,
+                    project,
+                    refName,
+                    allRevisions);
+                return callSendObject(source, project, refName, isDelete, allRevisions, state);
+              }
+
               throw new MissingParentObjectException(
                   project, refName, source.getRemoteConfigName());
             }
           }
 
-          resultIsSuccessful &= result.isSuccessful();
+          resultIsSuccessful &= resultSuccessful;
         } catch (URISyntaxException e) {
           repLog.warn(
               "Pull replication REST API apply object to {} *FAILED* for {}:{} - {}",
@@ -362,6 +392,28 @@
     return resultIsSuccessful;
   }
 
+  private List<RevisionData> fetchWholeMetaHistory(
+      NameKey project, String refName, RevisionData revision)
+      throws RepositoryNotFoundException, MissingObjectException, IncorrectObjectTypeException,
+          CorruptObjectException, IOException {
+    RevisionReader revisionReader = revReaderProvider.get();
+    Optional<RevisionData> revisionDataWithParents =
+        revisionReader.read(project, refName, Integer.MAX_VALUE);
+
+    ImmutableList.Builder<RevisionData> revisionDataBuilder = ImmutableList.builder();
+    List<ObjectId> parentObjectIds =
+        revisionDataWithParents
+            .map(RevisionData::getParentObjetIds)
+            .orElse(Collections.emptyList());
+    for (ObjectId parentObjectId : parentObjectIds) {
+      revisionReader.read(project, parentObjectId, refName, 0).ifPresent(revisionDataBuilder::add);
+    }
+
+    revisionDataBuilder.add(revision);
+
+    return revisionDataBuilder.build();
+  }
+
   private boolean callFetch(
       Source source, Project.NameKey project, String refName, ReplicationState state) {
     boolean resultIsSuccessful = true;
@@ -374,6 +426,7 @@
           Context<String> timer = fetchMetrics.startEnd2End(source.getRemoteConfigName());
           HttpResult result = fetchClient.callFetch(project, refName, uri, timer.getStartTime());
           long elapsedMs = TimeUnit.NANOSECONDS.toMillis(timer.stop());
+          boolean resultSuccessful = result.isSuccessful();
           repLog.info(
               "Pull replication REST API fetch to {} COMPLETED for {}:{}, HTTP Result:"
                   + " {} - time:{} ms",
@@ -382,10 +435,12 @@
               refName,
               result,
               elapsedMs);
-          if (isProjectMissing(result, project) && source.isCreateMissingRepositories()) {
+          if (!resultSuccessful
+              && result.isProjectMissing(project)
+              && source.isCreateMissingRepositories()) {
             result = initProject(project, uri, fetchClient, result);
           }
-          if (!result.isSuccessful()) {
+          if (!resultSuccessful) {
             stateLog.warn(
                 String.format(
                     "Pull replication rest api fetch call failed. Endpoint url: %s, reason:%s",
@@ -417,10 +472,6 @@
     return maxRetries == 0 || attempt < maxRetries;
   }
 
-  private Boolean isProjectMissing(HttpResult result, Project.NameKey project) {
-    return !result.isSuccessful() && result.isProjectMissing(project);
-  }
-
   private HttpResult initProject(
       Project.NameKey project, URIish uri, FetchApiClient fetchClient, HttpResult result)
       throws IOException, ClientProtocolException {
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/RevisionReader.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/RevisionReader.java
index eedd9eb..db46b23 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/RevisionReader.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/RevisionReader.java
@@ -25,7 +25,9 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionData;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionObjectData;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
 import org.eclipse.jgit.diff.DiffEntry;
@@ -48,8 +50,11 @@
 public class RevisionReader {
   private static final String CONFIG_MAX_API_PAYLOAD_SIZE = "maxApiPayloadSize";
   private static final Long DEFAULT_MAX_PAYLOAD_SIZE_IN_BYTES = 10000L;
+  static final String CONFIG_MAX_API_HISTORY_DEPTH = "maxApiHistoryDepth";
+  private static final int DEFAULT_MAX_API_HISTORY_DEPTH = 128;
   private GitRepositoryManager gitRepositoryManager;
   private Long maxRefSize;
+  private final int maxDepth;
 
   @Inject
   public RevisionReader(GitRepositoryManager gitRepositoryManager, ReplicationConfig cfg) {
@@ -57,16 +62,23 @@
     this.maxRefSize =
         cfg.getConfig()
             .getLong("replication", CONFIG_MAX_API_PAYLOAD_SIZE, DEFAULT_MAX_PAYLOAD_SIZE_IN_BYTES);
-  }
-
-  public Optional<RevisionData> read(Project.NameKey project, String refName)
-      throws RepositoryNotFoundException, MissingObjectException, IncorrectObjectTypeException,
-          CorruptObjectException, IOException {
-    return read(project, null, refName);
+    this.maxDepth =
+        cfg.getConfig()
+            .getInt("replication", CONFIG_MAX_API_HISTORY_DEPTH, DEFAULT_MAX_API_HISTORY_DEPTH);
   }
 
   public Optional<RevisionData> read(
-      Project.NameKey project, @Nullable ObjectId refObjectId, String refName)
+      Project.NameKey project, String refName, int maxParentObjectIds)
+      throws RepositoryNotFoundException, MissingObjectException, IncorrectObjectTypeException,
+          CorruptObjectException, IOException {
+    return read(project, null, refName, maxParentObjectIds);
+  }
+
+  public Optional<RevisionData> read(
+      Project.NameKey project,
+      @Nullable ObjectId refObjectId,
+      String refName,
+      int maxParentObjectIds)
       throws MissingObjectException, IncorrectObjectTypeException, CorruptObjectException,
           RepositoryNotFoundException, IOException {
     try (Repository git = gitRepositoryManager.openRepository(project)) {
@@ -86,10 +98,12 @@
       if (commitLoader.getType() == Constants.OBJ_BLOB) {
         return Optional.of(
             new RevisionData(
+                Collections.emptyList(),
                 null,
                 null,
                 Arrays.asList(
-                    new RevisionObjectData(Constants.OBJ_BLOB, commitLoader.getCachedBytes()))));
+                    new RevisionObjectData(
+                        objectId.name(), Constants.OBJ_BLOB, commitLoader.getCachedBytes()))));
       }
 
       if (commitLoader.getType() != Constants.OBJ_COMMIT) {
@@ -103,15 +117,16 @@
 
       RevCommit commit = RevCommit.parse(commitLoader.getCachedBytes());
       RevisionObjectData commitRev =
-          new RevisionObjectData(commit.getType(), commitLoader.getCachedBytes());
+          new RevisionObjectData(objectId.name(), commit.getType(), commitLoader.getCachedBytes());
 
       RevTree tree = commit.getTree();
-      ObjectLoader treeLoader = git.open(commit.getTree().toObjectId());
+      ObjectId treeObjectId = commit.getTree().toObjectId();
+      ObjectLoader treeLoader = git.open(treeObjectId);
       totalRefSize += treeLoader.getSize();
-      verifySize(project, refName, commit.getTree().toObjectId(), totalRefSize, treeLoader);
+      verifySize(project, refName, treeObjectId, totalRefSize, treeLoader);
 
       RevisionObjectData treeRev =
-          new RevisionObjectData(tree.getType(), treeLoader.getCachedBytes());
+          new RevisionObjectData(treeObjectId.name(), tree.getType(), treeLoader.getCachedBytes());
 
       List<RevisionObjectData> blobs = Lists.newLinkedList();
       try (TreeWalk walk = new TreeWalk(git)) {
@@ -124,7 +139,12 @@
           blobs = readBlobs(project, refName, git, totalRefSize, walk);
         }
       }
-      return Optional.of(new RevisionData(commitRev, treeRev, blobs));
+
+      List<ObjectId> parentObjectIds =
+          getParentObjectIds(git, commit.getParents(), 0, Math.min(maxDepth, maxParentObjectIds));
+      Collections.reverse(parentObjectIds);
+
+      return Optional.of(new RevisionData(parentObjectIds, commitRev, treeRev, blobs));
     } catch (LargeObjectException e) {
       repLog.trace(
           "Ref {} size for project {} is greater than configured '{}'",
@@ -135,6 +155,32 @@
     }
   }
 
+  private List<ObjectId> getParentObjectIds(
+      Repository git, RevCommit[] commit, int parentsDepth, int maxParentObjectIds)
+      throws MissingObjectException, IncorrectObjectTypeException, IOException {
+    if (commit == null || commit.length == 0) {
+      return Collections.emptyList();
+    }
+
+    ArrayList<ObjectId> parentObjectIds = new ArrayList<>();
+    for (RevCommit revCommit : commit) {
+      if (parentsDepth < maxParentObjectIds) {
+        parentObjectIds.add(revCommit.getId());
+        parentsDepth++;
+
+        ObjectLoader ol = git.open(revCommit.getId(), Constants.OBJ_COMMIT);
+        RevCommit[] commitParents = RevCommit.parse(ol.getCachedBytes()).getParents();
+
+        List<ObjectId> nestedParentObjectIds =
+            getParentObjectIds(git, commitParents, parentsDepth, maxParentObjectIds);
+        parentObjectIds.addAll(nestedParentObjectIds);
+        parentsDepth += nestedParentObjectIds.size();
+      }
+    }
+
+    return parentObjectIds;
+  }
+
   private List<DiffEntry> readDiffs(Repository git, RevCommit commit, RevTree tree, TreeWalk walk)
       throws MissingObjectException, IncorrectObjectTypeException, CorruptObjectException,
           IOException {
@@ -155,7 +201,8 @@
       verifySize(projectName, refName, objectId, totalRefSize, objectLoader);
 
       RevisionObjectData rev =
-          new RevisionObjectData(objectLoader.getType(), objectLoader.getCachedBytes());
+          new RevisionObjectData(
+              objectId.name(), objectLoader.getType(), objectLoader.getCachedBytes());
       blobs.add(rev);
     }
     return blobs;
@@ -171,12 +218,13 @@
     List<RevisionObjectData> blobs = Lists.newLinkedList();
     for (DiffEntry diffEntry : diffEntries) {
       if (!ChangeType.DELETE.equals(diffEntry.getChangeType())) {
-        ObjectLoader objectLoader = git.open(diffEntry.getNewId().toObjectId());
+        ObjectId diffObjectId = diffEntry.getNewId().toObjectId();
+        ObjectLoader objectLoader = git.open(diffObjectId);
         totalRefSize += objectLoader.getSize();
-        verifySize(
-            projectName, refName, diffEntry.getNewId().toObjectId(), totalRefSize, objectLoader);
+        verifySize(projectName, refName, diffObjectId, totalRefSize, objectLoader);
         RevisionObjectData rev =
-            new RevisionObjectData(objectLoader.getType(), objectLoader.getCachedBytes());
+            new RevisionObjectData(
+                diffObjectId.name(), objectLoader.getType(), objectLoader.getCachedBytes());
         blobs.add(rev);
       }
     }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectAction.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectAction.java
index 872e17a..04cc9e8 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectAction.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectAction.java
@@ -35,16 +35,16 @@
 
 public class ApplyObjectAction implements RestModifyView<ProjectResource, RevisionInput> {
 
-  private final ApplyObjectCommand command;
+  private final ApplyObjectCommand applyObjectCommand;
   private final DeleteRefCommand deleteRefCommand;
   private final FetchPreconditions preConditions;
 
   @Inject
   public ApplyObjectAction(
-      ApplyObjectCommand command,
+      ApplyObjectCommand applyObjectCommand,
       DeleteRefCommand deleteRefCommand,
       FetchPreconditions preConditions) {
-    this.command = command;
+    this.applyObjectCommand = applyObjectCommand;
     this.deleteRefCommand = deleteRefCommand;
     this.preConditions = preConditions;
   }
@@ -53,16 +53,16 @@
   public Response<?> apply(ProjectResource resource, RevisionInput input) throws RestApiException {
 
     if (!preConditions.canCallFetchApi()) {
-      throw new AuthException("not allowed to call fetch command");
+      throw new AuthException("Not allowed to call fetch command");
     }
-    try {
-      if (Strings.isNullOrEmpty(input.getLabel())) {
-        throw new BadRequestException("Source label cannot be null or empty");
-      }
-      if (Strings.isNullOrEmpty(input.getRefName())) {
-        throw new BadRequestException("Ref-update refname cannot be null or empty");
-      }
+    if (Strings.isNullOrEmpty(input.getLabel())) {
+      throw new BadRequestException("Source label cannot be null or empty");
+    }
+    if (Strings.isNullOrEmpty(input.getRefName())) {
+      throw new BadRequestException("Ref-update refname cannot be null or empty");
+    }
 
+    try {
       repLog.info(
           "Apply object API from {} for {}:{} - {}",
           resource.getNameKey(),
@@ -96,7 +96,7 @@
         throw bre;
       }
 
-      command.applyObject(
+      applyObjectCommand.applyObject(
           resource.getNameKey(), input.getRefName(), input.getRevisionData(), input.getLabel());
       return Response.created(input);
     } catch (MissingParentObjectException e) {
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectCommand.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectCommand.java
index ff231d8..b27d8b7 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectCommand.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectCommand.java
@@ -37,6 +37,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.fetch.ApplyObject;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.RefUpdateState;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Set;
 import org.eclipse.jgit.lib.RefUpdate;
 import org.eclipse.jgit.transport.RefSpec;
@@ -70,12 +71,25 @@
   }
 
   public void applyObject(
-      Project.NameKey name, String refName, RevisionData revisionData, String sourceLabel)
+      Project.NameKey name, String refName, RevisionData revisionsData, String sourceLabel)
+      throws IOException, RefUpdateException, MissingParentObjectException {
+    applyObjects(name, refName, new RevisionData[] {revisionsData}, sourceLabel);
+  }
+
+  public void applyObjects(
+      Project.NameKey name, String refName, RevisionData[] revisionsData, String sourceLabel)
       throws IOException, RefUpdateException, MissingParentObjectException {
 
-    repLog.info("Apply object from {} for {}:{} - {}", sourceLabel, name, refName, revisionData);
+    repLog.info(
+        "Apply object from {} for {}:{} - {}",
+        sourceLabel,
+        name,
+        refName,
+        Arrays.toString(revisionsData));
     Timer1.Context<String> context = metrics.start(sourceLabel);
-    RefUpdateState refUpdateState = applyObject.apply(name, new RefSpec(refName), revisionData);
+
+    RefUpdateState refUpdateState = applyObject.apply(name, new RefSpec(refName), revisionsData);
+
     long elapsed = NANOSECONDS.toMillis(context.stop());
 
     try {
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectsAction.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectsAction.java
new file mode 100644
index 0000000..a1e1f5b
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectsAction.java
@@ -0,0 +1,131 @@
+// Copyright (C) 2022 The Android Open Source Project
+//
+// 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.googlesource.gerrit.plugins.replication.pull.api;
+
+import static com.googlesource.gerrit.plugins.replication.pull.PullReplicationLogger.repLog;
+
+import com.google.common.base.Strings;
+import com.google.gerrit.extensions.restapi.AuthException;
+import com.google.gerrit.extensions.restapi.BadRequestException;
+import com.google.gerrit.extensions.restapi.ResourceConflictException;
+import com.google.gerrit.extensions.restapi.Response;
+import com.google.gerrit.extensions.restapi.RestApiException;
+import com.google.gerrit.extensions.restapi.RestModifyView;
+import com.google.gerrit.extensions.restapi.UnprocessableEntityException;
+import com.google.gerrit.server.project.ProjectResource;
+import com.google.inject.Inject;
+import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionsInput;
+import com.googlesource.gerrit.plugins.replication.pull.api.exception.MissingParentObjectException;
+import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Objects;
+import javax.servlet.http.HttpServletResponse;
+
+public class ApplyObjectsAction implements RestModifyView<ProjectResource, RevisionsInput> {
+
+  private final ApplyObjectCommand command;
+  private final DeleteRefCommand deleteRefCommand;
+  private final FetchPreconditions preConditions;
+
+  @Inject
+  public ApplyObjectsAction(
+      ApplyObjectCommand command,
+      DeleteRefCommand deleteRefCommand,
+      FetchPreconditions preConditions) {
+    this.command = command;
+    this.deleteRefCommand = deleteRefCommand;
+    this.preConditions = preConditions;
+  }
+
+  @Override
+  public Response<?> apply(ProjectResource resource, RevisionsInput input) throws RestApiException {
+    if (!preConditions.canCallFetchApi()) {
+      throw new AuthException("not allowed to call fetch command");
+    }
+
+    try {
+      if (Strings.isNullOrEmpty(input.getLabel())) {
+        throw new BadRequestException("Source label cannot be null or empty");
+      }
+      if (Strings.isNullOrEmpty(input.getRefName())) {
+        throw new BadRequestException("Ref-update refname cannot be null or empty");
+      }
+
+      repLog.info(
+          "Apply object API from {} for {}:{} - {}",
+          resource.getNameKey(),
+          input.getLabel(),
+          input.getRefName(),
+          Arrays.toString(input.getRevisionsData()));
+
+      if (Objects.isNull(input.getRevisionsData())) {
+        deleteRefCommand.deleteRef(resource.getNameKey(), input.getRefName(), input.getLabel());
+        repLog.info(
+            "Apply object API - REF DELETED - from {} for {}:{}",
+            resource.getNameKey(),
+            input.getLabel(),
+            input.getRefName());
+        return Response.withStatusCode(HttpServletResponse.SC_NO_CONTENT, "");
+      }
+
+      try {
+        input.validate();
+      } catch (IllegalArgumentException e) {
+        BadRequestException bre =
+            new BadRequestException("Ref-update with invalid input: " + e.getMessage(), e);
+        repLog.error(
+            "Apply object API *FAILED* from {} for {}:{} - {}",
+            input.getLabel(),
+            resource.getNameKey(),
+            input.getRefName(),
+            Arrays.toString(input.getRevisionsData()),
+            bre);
+        throw bre;
+      }
+
+      command.applyObjects(
+          resource.getNameKey(), input.getRefName(), input.getRevisionsData(), input.getLabel());
+      return Response.created(input);
+    } catch (MissingParentObjectException e) {
+      repLog.error(
+          "Apply object API *FAILED* from {} for {}:{} - {}",
+          input.getLabel(),
+          resource.getNameKey(),
+          input.getRefName(),
+          Arrays.toString(input.getRevisionsData()),
+          e);
+      throw new ResourceConflictException(e.getMessage(), e);
+    } catch (NumberFormatException | IOException e) {
+      repLog.error(
+          "Apply object API *FAILED* from {} for {}:{} - {}",
+          input.getLabel(),
+          resource.getNameKey(),
+          input.getRefName(),
+          Arrays.toString(input.getRevisionsData()),
+          e);
+      throw RestApiException.wrap(e.getMessage(), e);
+    } catch (RefUpdateException e) {
+      repLog.error(
+          "Apply object API *FAILED* from {} for {}:{} - {}",
+          input.getLabel(),
+          resource.getNameKey(),
+          input.getRefName(),
+          Arrays.toString(input.getRevisionsData()),
+          e);
+      throw new UnprocessableEntityException(e.getMessage());
+    }
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationApiModule.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationApiModule.java
index e7b3a7f..d1d28a6 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationApiModule.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationApiModule.java
@@ -31,6 +31,7 @@
     bind(UpdateHeadAction.class).in(Scopes.SINGLETON);
     post(PROJECT_KIND, "fetch").to(FetchAction.class);
     post(PROJECT_KIND, "apply-object").to(ApplyObjectAction.class);
+    post(PROJECT_KIND, "apply-objects").to(ApplyObjectsAction.class);
     delete(PROJECT_KIND, "delete-project").to(ProjectDeletionAction.class);
     put(PROJECT_KIND, "HEAD").to(UpdateHeadAction.class);
 
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationFilter.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationFilter.java
index 4af2bf7..0a9b266 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationFilter.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationFilter.java
@@ -55,6 +55,7 @@
 import com.google.inject.TypeLiteral;
 import com.googlesource.gerrit.plugins.replication.pull.api.FetchAction.Input;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionInput;
+import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionsInput;
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.InitProjectException;
 import java.io.BufferedReader;
 import java.io.EOFException;
@@ -75,6 +76,7 @@
 
   private FetchAction fetchAction;
   private ApplyObjectAction applyObjectAction;
+  private ApplyObjectsAction applyObjectsAction;
   private ProjectInitializationAction projectInitializationAction;
   private UpdateHeadAction updateHEADAction;
   private ProjectDeletionAction projectDeletionAction;
@@ -87,6 +89,7 @@
   public PullReplicationFilter(
       FetchAction fetchAction,
       ApplyObjectAction applyObjectAction,
+      ApplyObjectsAction applyObjectsAction,
       ProjectInitializationAction projectInitializationAction,
       UpdateHeadAction updateHEADAction,
       ProjectDeletionAction projectDeletionAction,
@@ -95,6 +98,7 @@
       @PluginName String pluginName) {
     this.fetchAction = fetchAction;
     this.applyObjectAction = applyObjectAction;
+    this.applyObjectsAction = applyObjectsAction;
     this.projectInitializationAction = projectInitializationAction;
     this.updateHEADAction = updateHEADAction;
     this.projectDeletionAction = projectDeletionAction;
@@ -127,6 +131,12 @@
         } else {
           httpResponse.sendError(SC_UNAUTHORIZED);
         }
+      } else if (isApplyObjectsAction(httpRequest)) {
+        if (userProvider.get().isIdentifiedUser()) {
+          writeResponse(httpResponse, doApplyObjects(httpRequest));
+        } else {
+          httpResponse.sendError(SC_UNAUTHORIZED);
+        }
       } else if (isInitProjectAction(httpRequest)) {
         if (userProvider.get().isIdentifiedUser()) {
           if (!checkAcceptHeader(httpRequest, httpResponse)) {
@@ -199,6 +209,16 @@
   }
 
   @SuppressWarnings("unchecked")
+  private Response<Map<String, Object>> doApplyObjects(HttpServletRequest httpRequest)
+      throws RestApiException, IOException, PermissionBackendException {
+    RevisionsInput input = readJson(httpRequest, TypeLiteral.get(RevisionsInput.class));
+    IdString id = getProjectName(httpRequest);
+    ProjectResource projectResource = projectsCollection.parse(TopLevelResource.INSTANCE, id);
+
+    return (Response<Map<String, Object>>) applyObjectsAction.apply(projectResource, input);
+  }
+
+  @SuppressWarnings("unchecked")
   private Response<String> doUpdateHEAD(HttpServletRequest httpRequest) throws Exception {
     HeadInput input = readJson(httpRequest, TypeLiteral.get(HeadInput.class));
     ProjectResource projectResource =
@@ -293,6 +313,10 @@
     return httpRequest.getRequestURI().endsWith("pull-replication~apply-object");
   }
 
+  private boolean isApplyObjectsAction(HttpServletRequest httpRequest) {
+    return httpRequest.getRequestURI().endsWith("pull-replication~apply-objects");
+  }
+
   private boolean isFetchAction(HttpServletRequest httpRequest) {
     return httpRequest.getRequestURI().endsWith("pull-replication~fetch");
   }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionData.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionData.java
index 9c33d07..ffe98da 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionData.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionData.java
@@ -15,8 +15,11 @@
 package com.googlesource.gerrit.plugins.replication.pull.api.data;
 
 import java.util.List;
+import org.eclipse.jgit.lib.ObjectId;
 
 public class RevisionData {
+  private transient List<ObjectId> parentObjectIds;
+
   private RevisionObjectData commitObject;
 
   private RevisionObjectData treeObject;
@@ -24,14 +27,20 @@
   private List<RevisionObjectData> blobs;
 
   public RevisionData(
+      List<ObjectId> parentObjectIds,
       RevisionObjectData commitObject,
       RevisionObjectData treeObject,
       List<RevisionObjectData> blobs) {
+    this.parentObjectIds = parentObjectIds;
     this.commitObject = commitObject;
     this.treeObject = treeObject;
     this.blobs = blobs;
   }
 
+  public List<ObjectId> getParentObjetIds() {
+    return parentObjectIds;
+  }
+
   public RevisionObjectData getCommitObject() {
     return commitObject;
   }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionInput.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionInput.java
index 18fc27e..c18e11d 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionInput.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionInput.java
@@ -44,6 +44,10 @@
   }
 
   public void validate() {
+    validate(refName, revisionData);
+  }
+
+  static void validate(String refName, RevisionData revisionData) {
     // Non-heads refs can point to non-commit objects
     if (!refName.startsWith(Constants.R_HEADS)
         && Objects.isNull(revisionData.getCommitObject())
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionObjectData.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionObjectData.java
index 0962717..b21f495 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionObjectData.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionObjectData.java
@@ -18,10 +18,12 @@
 import org.eclipse.jgit.lib.Constants;
 
 public class RevisionObjectData {
+  private final String sha1;
   private final Integer type;
   private final String content;
 
-  public RevisionObjectData(int type, byte[] content) {
+  public RevisionObjectData(String sha1, int type, byte[] content) {
+    this.sha1 = sha1;
     this.type = type;
     this.content = content == null ? "" : Base64.getEncoder().encodeToString(content);
   }
@@ -34,17 +36,28 @@
     return Base64.getDecoder().decode(content);
   }
 
+  public String getSha1() {
+    return sha1;
+  }
+
   @Override
   public String toString() {
+    String typeStr;
     switch (type) {
       case Constants.OBJ_BLOB:
-        return "BLOB";
+        typeStr = "BLOB";
+        break;
       case Constants.OBJ_COMMIT:
-        return "COMMIT";
+        typeStr = "COMMIT";
+        break;
       case Constants.OBJ_TREE:
-        return "TREE";
+        typeStr = "TREE";
+        break;
       default:
-        return "type:" + type;
+        typeStr = "type:" + type;
+        break;
     }
+
+    return sha1 + " (" + typeStr + ")";
   }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionsInput.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionsInput.java
new file mode 100644
index 0000000..2361f6b
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionsInput.java
@@ -0,0 +1,60 @@
+// Copyright (C) 2022 The Android Open Source Project
+//
+// 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.googlesource.gerrit.plugins.replication.pull.api.data;
+
+import java.util.Arrays;
+
+public class RevisionsInput {
+  private String label;
+
+  private String refName;
+
+  private RevisionData[] revisionsData;
+
+  public RevisionsInput(String label, String refName, RevisionData[] revisionsData) {
+    this.label = label;
+    this.refName = refName;
+    this.revisionsData = revisionsData;
+  }
+
+  public String getLabel() {
+    return label;
+  }
+
+  public String getRefName() {
+    return refName;
+  }
+
+  public RevisionData[] getRevisionsData() {
+    return revisionsData;
+  }
+
+  public void validate() {
+    for (RevisionData revisionData : revisionsData) {
+      RevisionInput.validate(refName, revisionData);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "RevisionsInput { "
+        + label
+        + ":"
+        + refName
+        + " - "
+        + Arrays.toString(revisionsData)
+        + "}";
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchApiClient.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchApiClient.java
index 01b6f1d..6b23a9f 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchApiClient.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchApiClient.java
@@ -18,6 +18,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.Source;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionData;
 import java.io.IOException;
+import java.util.List;
 import org.apache.http.client.ClientProtocolException;
 import org.eclipse.jgit.transport.URIish;
 
@@ -49,4 +50,8 @@
       RevisionData revisionData,
       URIish targetUri)
       throws ClientProtocolException, IOException;
+
+  HttpResult callSendObjects(
+      Project.NameKey project, String refName, List<RevisionData> revisionData, URIish targetUri)
+      throws ClientProtocolException, IOException;
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClient.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClient.java
index 2dd991c..ed919de 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClient.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClient.java
@@ -23,6 +23,7 @@
 import com.google.common.net.MediaType;
 import com.google.gerrit.common.Nullable;
 import com.google.gerrit.entities.Project;
+import com.google.gerrit.entities.Project.NameKey;
 import com.google.gerrit.extensions.annotations.PluginName;
 import com.google.gerrit.extensions.restapi.Url;
 import com.google.gson.Gson;
@@ -35,9 +36,11 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.PullReplicationApiRequestMetrics;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionData;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionInput;
+import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionsInput;
 import com.googlesource.gerrit.plugins.replication.pull.filter.SyncRefsFilter;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.util.List;
 import java.util.Optional;
 import org.apache.http.HttpResponse;
 import org.apache.http.ParseException;
@@ -179,10 +182,7 @@
     }
     RevisionInput input = new RevisionInput(instanceLabel, refName, revisionData);
 
-    String url =
-        String.format(
-            "%s/a/projects/%s/%s~apply-object",
-            targetUri.toString(), Url.encode(project.get()), pluginName);
+    String url = formatUrl(project, targetUri, "apply-object");
 
     HttpPost post = new HttpPost(url);
     post.setEntity(new StringEntity(GSON.toJson(input)));
@@ -190,6 +190,33 @@
     return httpClientFactory.create(source).execute(post, this, getContext(targetUri));
   }
 
+  @Override
+  public HttpResult callSendObjects(
+      NameKey project, String refName, List<RevisionData> revisionData, URIish targetUri)
+      throws ClientProtocolException, IOException {
+    if (revisionData.size() == 1) {
+      return callSendObject(project, refName, false, revisionData.get(0), targetUri);
+    }
+
+    RevisionData[] inputData = new RevisionData[revisionData.size()];
+    RevisionsInput input =
+        new RevisionsInput(instanceLabel, refName, revisionData.toArray(inputData));
+
+    String url = formatUrl(project, targetUri, "apply-objects");
+    HttpPost post = new HttpPost(url);
+    post.setEntity(new StringEntity(GSON.toJson(input)));
+    post.addHeader(new BasicHeader("Content-Type", MediaType.JSON_UTF_8.toString()));
+    return httpClientFactory.create(source).execute(post, this, getContext(targetUri));
+  }
+
+  private String formatUrl(Project.NameKey project, URIish targetUri, String api) {
+    String url =
+        String.format(
+            "%s/a/projects/%s/%s~%s",
+            targetUri.toString(), Url.encode(project.get()), pluginName, api);
+    return url;
+  }
+
   private void requireNull(Object object, String string) {
     if (object != null) {
       throw new IllegalArgumentException(string);
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/fetch/ApplyObject.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/fetch/ApplyObject.java
index fca085b..2bb1caf 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/fetch/ApplyObject.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/fetch/ApplyObject.java
@@ -41,47 +41,51 @@
     this.gitManager = gitManager;
   }
 
-  public RefUpdateState apply(Project.NameKey name, RefSpec refSpec, RevisionData revisionData)
+  public RefUpdateState apply(Project.NameKey name, RefSpec refSpec, RevisionData[] revisionsData)
       throws MissingParentObjectException, IOException {
     try (Repository git = gitManager.openRepository(name)) {
 
-      ObjectId newObjectID = null;
-      RevisionObjectData commitObject = revisionData.getCommitObject();
-
-      try (ObjectInserter oi = git.newObjectInserter()) {
-        if (commitObject != null) {
-          RevCommit commit = RevCommit.parse(commitObject.getContent());
-          for (RevCommit parent : commit.getParents()) {
-            if (!git.getObjectDatabase().has(parent.getId())) {
-              throw new MissingParentObjectException(name, refSpec.getSource(), parent.getId());
-            }
-          }
-          newObjectID = oi.insert(commitObject.getType(), commitObject.getContent());
-
-          RevisionObjectData treeObject = revisionData.getTreeObject();
-          oi.insert(treeObject.getType(), treeObject.getContent());
-        }
-
-        for (RevisionObjectData rev : revisionData.getBlobs()) {
-          ObjectId blobObjectId = oi.insert(rev.getType(), rev.getContent());
-          if (newObjectID == null) {
-            newObjectID = blobObjectId;
-          }
-        }
-
-        oi.flush();
-      }
+      ObjectId refHead = null;
       RefUpdate ru = git.updateRef(refSpec.getSource());
-      ru.setNewObjectId(newObjectID);
+      try (ObjectInserter oi = git.newObjectInserter()) {
+        for (RevisionData revisionData : revisionsData) {
 
-      if (commitObject == null) {
-        // Non-commits must be forced as they do not have a graph associated
-        ru.setForceUpdate(true);
+          ObjectId newObjectID = null;
+          RevisionObjectData commitObject = revisionData.getCommitObject();
+
+          if (commitObject != null) {
+            RevCommit commit = RevCommit.parse(commitObject.getContent());
+            for (RevCommit parent : commit.getParents()) {
+              if (!git.getObjectDatabase().has(parent.getId())) {
+                throw new MissingParentObjectException(name, refSpec.getSource(), parent.getId());
+              }
+            }
+            refHead = newObjectID = oi.insert(commitObject.getType(), commitObject.getContent());
+
+            RevisionObjectData treeObject = revisionData.getTreeObject();
+            oi.insert(treeObject.getType(), treeObject.getContent());
+          }
+
+          for (RevisionObjectData rev : revisionData.getBlobs()) {
+            ObjectId blobObjectId = oi.insert(rev.getType(), rev.getContent());
+            if (newObjectID == null) {
+              newObjectID = blobObjectId;
+            }
+            refHead = newObjectID;
+          }
+
+          oi.flush();
+
+          if (commitObject == null) {
+            // Non-commits must be forced as they do not have a graph associated
+            ru.setForceUpdate(true);
+          }
+        }
+
+        ru.setNewObjectId(refHead);
+        RefUpdate.Result result = ru.update();
+        return new RefUpdateState(refSpec.getSource(), result);
       }
-
-      RefUpdate.Result result = ru.update();
-
-      return new RefUpdateState(refSpec.getSource(), result);
     }
   }
 }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueueTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueueTest.java
index b8ff766..48e0e71 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueueTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueueTest.java
@@ -17,10 +17,12 @@
 import static com.google.common.truth.Truth.assertThat;
 import static java.nio.file.Files.createTempDirectory;
 import static org.mockito.ArgumentMatchers.anyBoolean;
-import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyInt;
 import static org.mockito.Mockito.anyLong;
 import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.lenient;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -51,6 +53,8 @@
 import com.googlesource.gerrit.plugins.replication.pull.filter.ExcludedRefsFilter;
 import java.io.IOException;
 import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Optional;
 import org.apache.http.client.ClientProtocolException;
 import org.eclipse.jgit.errors.LargeObjectException;
@@ -82,12 +86,15 @@
   @Mock RevisionData revisionData;
   @Mock HttpResult successfulHttpResult;
   @Mock HttpResult fetchHttpResult;
+  @Mock RevisionData revisionDataWithParents;
+  List<ObjectId> revisionDataParentObjectIds;
   @Mock HttpResult httpResult;
   ApplyObjectMetrics applyObjectMetrics;
   FetchReplicationMetrics fetchMetrics;
 
   @Captor ArgumentCaptor<String> stringCaptor;
   @Captor ArgumentCaptor<Project.NameKey> projectNameKeyCaptor;
+  @Captor ArgumentCaptor<List<RevisionData>> revisionsDataCaptor;
 
   private ExcludedRefsFilter refsFilter;
   private ReplicationQueue objectUnderTest;
@@ -108,9 +115,28 @@
     when(source.getApis()).thenReturn(apis);
     when(sourceCollection.getAll()).thenReturn(Lists.newArrayList(source));
     when(rd.get()).thenReturn(sourceCollection);
-    when(revReader.read(any(), any(), anyString())).thenReturn(Optional.of(revisionData));
+    lenient()
+        .when(revReader.read(any(), any(), anyString(), eq(0)))
+        .thenReturn(Optional.of(revisionData));
+    lenient().when(revReader.read(any(), anyString(), eq(0))).thenReturn(Optional.of(revisionData));
+    lenient()
+        .when(revReader.read(any(), any(), anyString(), eq(Integer.MAX_VALUE)))
+        .thenReturn(Optional.of(revisionDataWithParents));
+    lenient()
+        .when(revReader.read(any(), anyString(), eq(Integer.MAX_VALUE)))
+        .thenReturn(Optional.of(revisionDataWithParents));
+    revisionDataParentObjectIds =
+        Arrays.asList(
+            ObjectId.fromString("9f8d52853089a3cf00c02ff7bd0817bd4353a95a"),
+            ObjectId.fromString("b5d7bcf1d1c5b0f0726d10a16c8315f06f900bfb"));
+    when(revisionDataWithParents.getParentObjetIds()).thenReturn(revisionDataParentObjectIds);
+
     when(fetchClientFactory.create(any())).thenReturn(fetchRestApiClient);
-    when(fetchRestApiClient.callSendObject(any(), anyString(), anyBoolean(), any(), any()))
+    lenient()
+        .when(fetchRestApiClient.callSendObject(any(), anyString(), anyBoolean(), any(), any()))
+        .thenReturn(httpResult);
+    lenient()
+        .when(fetchRestApiClient.callSendObjects(any(), anyString(), any(), any()))
         .thenReturn(httpResult);
     when(fetchRestApiClient.callFetch(any(), anyString(), any(), anyLong()))
         .thenReturn(fetchHttpResult);
@@ -131,7 +157,7 @@
             sl,
             fetchClientFactory,
             refsFilter,
-            revReader,
+            () -> revReader,
             applyObjectMetrics,
             fetchMetrics);
   }
@@ -142,7 +168,7 @@
     objectUnderTest.start();
     objectUnderTest.onGitReferenceUpdated(event);
 
-    verify(fetchRestApiClient).callSendObject(any(), anyString(), eq(false), any(), any());
+    verify(fetchRestApiClient).callSendObjects(any(), anyString(), any(), any());
   }
 
   @Test
@@ -177,7 +203,7 @@
     objectUnderTest.start();
     objectUnderTest.onGitReferenceUpdated(event);
 
-    verify(fetchRestApiClient).callSendObject(any(), anyString(), eq(false), any(), any());
+    verify(fetchRestApiClient).callSendObjects(any(), anyString(), any(), any());
   }
 
   @Test
@@ -186,7 +212,7 @@
     Event event = new TestEvent("refs/changes/01/1/meta");
     objectUnderTest.start();
 
-    when(revReader.read(any(), any(), anyString())).thenThrow(IOException.class);
+    when(revReader.read(any(), any(), anyString(), anyInt())).thenThrow(IOException.class);
 
     objectUnderTest.onGitReferenceUpdated(event);
 
@@ -199,7 +225,7 @@
     Event event = new TestEvent("refs/changes/01/1/1");
     objectUnderTest.start();
 
-    when(revReader.read(any(), any(), anyString())).thenReturn(Optional.empty());
+    when(revReader.read(any(), any(), anyString(), anyInt())).thenReturn(Optional.empty());
 
     objectUnderTest.onGitReferenceUpdated(event);
 
@@ -209,12 +235,12 @@
   @Test
   public void shouldFallbackToCallFetchWhenParentObjectIsMissing()
       throws ClientProtocolException, IOException {
-    Event event = new TestEvent("refs/changes/01/1/meta");
+    Event event = new TestEvent("refs/changes/01/1/1");
     objectUnderTest.start();
 
     when(httpResult.isSuccessful()).thenReturn(false);
     when(httpResult.isParentObjectMissing()).thenReturn(true);
-    when(fetchRestApiClient.callSendObject(any(), anyString(), eq(false), any(), any()))
+    when(fetchRestApiClient.callSendObjects(any(), anyString(), any(), any()))
         .thenReturn(httpResult);
 
     objectUnderTest.onGitReferenceUpdated(event);
@@ -223,6 +249,32 @@
   }
 
   @Test
+  public void shouldFallbackToApplyAllParentObjectsWhenParentObjectIsMissingOnMetaRef()
+      throws ClientProtocolException, IOException {
+    Event event = new TestEvent("refs/changes/01/1/meta");
+    objectUnderTest.start();
+
+    when(httpResult.isSuccessful()).thenReturn(false, true);
+    when(httpResult.isParentObjectMissing()).thenReturn(true, false);
+    when(fetchRestApiClient.callSendObjects(any(), anyString(), any(), any()))
+        .thenReturn(httpResult);
+
+    objectUnderTest.onGitReferenceUpdated(event);
+
+    verify(fetchRestApiClient, times(2))
+        .callSendObjects(any(), anyString(), revisionsDataCaptor.capture(), any());
+    List<List<RevisionData>> revisionsDataValues = revisionsDataCaptor.getAllValues();
+    assertThat(revisionsDataValues).hasSize(2);
+
+    List<RevisionData> firstRevisionsValues = revisionsDataValues.get(0);
+    assertThat(firstRevisionsValues).hasSize(1);
+    assertThat(firstRevisionsValues).contains(revisionData);
+
+    List<RevisionData> secondRevisionsValues = revisionsDataValues.get(1);
+    assertThat(secondRevisionsValues).hasSize(1 + revisionDataParentObjectIds.size());
+  }
+
+  @Test
   public void shouldSkipEventWhenUsersRef() {
     Event event = new TestEvent("refs/users/00/1000000");
     objectUnderTest.onGitReferenceUpdated(event);
@@ -271,7 +323,7 @@
             sl,
             fetchClientFactory,
             refsFilter,
-            revReader,
+            () -> revReader,
             applyObjectMetrics,
             fetchMetrics);
     Event event = new TestEvent("refs/multi-site/version");
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/RevisionReaderIT.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/RevisionReaderIT.java
index e300613..1d8520f 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/RevisionReaderIT.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/RevisionReaderIT.java
@@ -24,12 +24,14 @@
 import com.google.gerrit.acceptance.TestPlugin;
 import com.google.gerrit.acceptance.UseLocalDisk;
 import com.google.gerrit.entities.Change;
+import com.google.gerrit.entities.Change.Id;
 import com.google.gerrit.entities.Patch;
 import com.google.gerrit.entities.RefNames;
 import com.google.gerrit.extensions.api.changes.ReviewInput;
 import com.google.gerrit.extensions.api.changes.ReviewInput.CommentInput;
 import com.google.gerrit.extensions.client.Comment;
 import com.google.gerrit.extensions.config.FactoryModule;
+import com.google.gerrit.extensions.restapi.RestApiException;
 import com.google.gerrit.server.notedb.Sequences;
 import com.google.inject.Scopes;
 import com.googlesource.gerrit.plugins.replication.ReplicationConfig;
@@ -38,11 +40,13 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionObjectData;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.ApplyObject;
 import java.io.IOException;
+import java.util.List;
 import java.util.Optional;
 import org.eclipse.jgit.lib.Constants;
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.lib.Ref;
 import org.eclipse.jgit.lib.Repository;
+import org.eclipse.jgit.storage.file.FileBasedConfig;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -53,9 +57,12 @@
 public class RevisionReaderIT extends LightweightPluginDaemonTest {
   RevisionReader objectUnderTest;
 
+  ReplicationFileBasedConfig replicationConfig;
+
   @Before
   public void setup() {
     objectUnderTest = plugin.getSysInjector().getInstance(RevisionReader.class);
+    replicationConfig = plugin.getSysInjector().getInstance(ReplicationFileBasedConfig.class);
   }
 
   @Test
@@ -64,7 +71,7 @@
     String refName = RefNames.changeMetaRef(pushResult.getChange().getId());
 
     Optional<RevisionData> revisionDataOption =
-        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId));
+        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId, 0));
 
     assertThat(revisionDataOption.isPresent()).isTrue();
     RevisionData revisionData = revisionDataOption.get();
@@ -80,9 +87,65 @@
     assertThat(revisionData.getBlobs()).isEmpty();
   }
 
-  private Optional<RevisionData> readRevisionFromObjectUnderTest(String refName, ObjectId objId) {
+  @Test
+  public void shouldReadRefMetaObjectWithMaxNumberOfParents() throws Exception {
+    int numberOfParents = 3;
+    setReplicationConfig(numberOfParents);
+    Result pushResult = createChange();
+    Id changeId = pushResult.getChange().getId();
+    String refName = RefNames.changeMetaRef(pushResult.getChange().getId());
+
+    addMultipleComments(numberOfParents, changeId);
+
+    Optional<RevisionData> revisionDataOption =
+        refObjectId(refName)
+            .flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId, numberOfParents));
+
+    assertThat(revisionDataOption.isPresent()).isTrue();
+    List<ObjectId> parentObjectIds = revisionDataOption.get().getParentObjetIds();
+    assertThat(parentObjectIds).hasSize(numberOfParents);
+  }
+
+  @Test
+  public void shouldReadRefMetaObjectLimitedToMaxNumberOfParents() throws Exception {
+    int numberOfParents = 3;
+    setReplicationConfig(numberOfParents);
+    Result pushResult = createChange();
+    Id changeId = pushResult.getChange().getId();
+    String refName = RefNames.changeMetaRef(pushResult.getChange().getId());
+
+    addMultipleComments(numberOfParents + 1, changeId);
+
+    Optional<RevisionData> revisionDataOption =
+        refObjectId(refName)
+            .flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId, numberOfParents));
+
+    assertThat(revisionDataOption.isPresent()).isTrue();
+    List<ObjectId> parentObjectIds = revisionDataOption.get().getParentObjetIds();
+    assertThat(parentObjectIds).hasSize(numberOfParents);
+  }
+
+  private void addMultipleComments(int numberOfParents, Id changeId) throws RestApiException {
+    for (int i = 0; i < numberOfParents; i++) {
+      addComment(changeId);
+    }
+  }
+
+  private void setReplicationConfig(int numberOfParents) throws IOException {
+    FileBasedConfig config = (FileBasedConfig) replicationConfig.getConfig();
+    config.setInt(
+        "replication", null, RevisionReader.CONFIG_MAX_API_HISTORY_DEPTH, numberOfParents);
+    config.save();
+  }
+
+  private void addComment(Id changeId) throws RestApiException {
+    gApi.changes().id(changeId.get()).current().review(new ReviewInput().message("foo"));
+  }
+
+  private Optional<RevisionData> readRevisionFromObjectUnderTest(
+      String refName, ObjectId objId, int maxParentsDepth) {
     try {
-      return objectUnderTest.read(project, objId, refName);
+      return objectUnderTest.read(project, objId, refName, maxParentsDepth);
     } catch (Exception e) {
       throw new IllegalStateException(e);
     }
@@ -107,7 +170,7 @@
     gApi.changes().id(changeId.get()).current().review(reviewInput);
 
     Optional<RevisionData> revisionDataOption =
-        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId));
+        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId, 0));
 
     assertThat(revisionDataOption.isPresent()).isTrue();
     RevisionData revisionData = revisionDataOption.get();
@@ -131,7 +194,7 @@
     createChange().assertOkStatus();
     String refName = RefNames.REFS_SEQUENCES + Sequences.NAME_CHANGES;
     Optional<RevisionData> revisionDataOption =
-        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId));
+        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId, 0));
 
     Truth8.assertThat(revisionDataOption).isEmpty();
   }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ActionITBase.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ActionITBase.java
index cef1051..1019b86 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ActionITBase.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ActionITBase.java
@@ -149,7 +149,8 @@
   protected Optional<RevisionData> createRevisionData(NameKey projectName, String refName)
       throws Exception {
     try (Repository repository = repoManager.openRepository(projectName)) {
-      return revisionReader.read(projectName, repository.exactRef(refName).getObjectId(), refName);
+      return revisionReader.read(
+          projectName, repository.exactRef(refName).getObjectId(), refName, 0);
     }
   }
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectActionTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectActionTest.java
index 3678e82..814ba76 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectActionTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectActionTest.java
@@ -37,6 +37,7 @@
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
+import java.util.Collections;
 import org.eclipse.jgit.lib.Constants;
 import org.eclipse.jgit.lib.ObjectId;
 import org.junit.Before;
@@ -55,8 +56,14 @@
   String location = "http://gerrit-host/a/config/server/tasks/08d173e9";
   int taskId = 1234;
 
+  private String sampleCommitObjectId = "9f8d52853089a3cf00c02ff7bd0817bd4353a95a";
+  private String sampleTreeObjectId = "4b825dc642cb6eb9a060e54bf8d69288fbee4904";
+  private String sampleBlobObjectId = "b5d7bcf1d1c5b0f0726d10a16c8315f06f900bfb";
+
   private String sampleCommitContent =
-      "tree 4b825dc642cb6eb9a060e54bf8d69288fbee4904\n"
+      "tree "
+          + sampleTreeObjectId
+          + "\n"
           + "parent 20eb48d28be86dc88fb4bef747f08de0fbefe12d\n"
           + "author Gerrit User 1000000 <1000000@69ec38f0-350e-4d9c-96d4-bc956f2faaac> 1610471611 +0100\n"
           + "committer Gerrit Code Review <root@maczech-XPS-15> 1610471611 +0100\n"
@@ -102,7 +109,8 @@
         new RevisionInput(
             label,
             refMetaName,
-            createSampleRevisionDataBlob(new RevisionObjectData(Constants.OBJ_BLOB, blobData)));
+            createSampleRevisionDataBlob(
+                new RevisionObjectData(sampleBlobObjectId, Constants.OBJ_BLOB, blobData)));
 
     Response<?> response = applyObjectAction.apply(projectResource, inputParams);
 
@@ -148,8 +156,10 @@
 
   @Test(expected = BadRequestException.class)
   public void shouldThrowBadRequestExceptionWhenMissingCommitObjectData() throws Exception {
-    RevisionObjectData commitData = new RevisionObjectData(Constants.OBJ_COMMIT, null);
-    RevisionObjectData treeData = new RevisionObjectData(Constants.OBJ_TREE, new byte[] {});
+    RevisionObjectData commitData =
+        new RevisionObjectData(sampleCommitObjectId, Constants.OBJ_COMMIT, null);
+    RevisionObjectData treeData =
+        new RevisionObjectData(sampleTreeObjectId, Constants.OBJ_TREE, new byte[] {});
     RevisionInput inputParams =
         new RevisionInput(label, refName, createSampleRevisionData(commitData, treeData));
 
@@ -159,7 +169,8 @@
   @Test(expected = BadRequestException.class)
   public void shouldThrowBadRequestExceptionWhenMissingTreeObject() throws Exception {
     RevisionObjectData commitData =
-        new RevisionObjectData(Constants.OBJ_COMMIT, sampleCommitContent.getBytes());
+        new RevisionObjectData(
+            sampleCommitObjectId, Constants.OBJ_COMMIT, sampleCommitContent.getBytes());
     RevisionInput inputParams =
         new RevisionInput(label, refName, createSampleRevisionData(commitData, null));
 
@@ -192,17 +203,19 @@
 
   private RevisionData createSampleRevisionData() {
     RevisionObjectData commitData =
-        new RevisionObjectData(Constants.OBJ_COMMIT, sampleCommitContent.getBytes());
-    RevisionObjectData treeData = new RevisionObjectData(Constants.OBJ_TREE, new byte[] {});
+        new RevisionObjectData(
+            sampleCommitObjectId, Constants.OBJ_COMMIT, sampleCommitContent.getBytes());
+    RevisionObjectData treeData =
+        new RevisionObjectData(sampleTreeObjectId, Constants.OBJ_TREE, new byte[] {});
     return createSampleRevisionData(commitData, treeData);
   }
 
   private RevisionData createSampleRevisionData(
       RevisionObjectData commitData, RevisionObjectData treeData) {
-    return new RevisionData(commitData, treeData, Lists.newArrayList());
+    return new RevisionData(Collections.emptyList(), commitData, treeData, Lists.newArrayList());
   }
 
   private RevisionData createSampleRevisionDataBlob(RevisionObjectData blob) {
-    return new RevisionData(null, null, Arrays.asList(blob));
+    return new RevisionData(Collections.emptyList(), null, null, Arrays.asList(blob));
   }
 }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectCommandTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectCommandTest.java
index 51051c0..d73a6e7 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectCommandTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectCommandTest.java
@@ -38,6 +38,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.fetch.ApplyObject;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.RefUpdateState;
 import java.io.IOException;
+import java.util.Collections;
 import org.eclipse.jgit.lib.Constants;
 import org.eclipse.jgit.lib.RefUpdate;
 import org.junit.Before;
@@ -55,6 +56,10 @@
   private static final NameKey TEST_PROJECT_NAME = Project.nameKey("test-project");
   private static final String TEST_REMOTE_NAME = "test-remote-name";
 
+  private String sampleCommitObjectId = "9f8d52853089a3cf00c02ff7bd0817bd4353a95a";
+  private String sampleTreeObjectId = "4b825dc642cb6eb9a060e54bf8d69288fbee4904";
+  private String sampleBlobObjectId = "b5d7bcf1d1c5b0f0726d10a16c8315f06f900bfb";
+
   @Mock private PullReplicationStateLogger fetchStateLog;
   @Mock private ApplyObject applyObject;
   @Mock private ApplyObjectMetrics metrics;
@@ -93,8 +98,10 @@
   }
 
   private RevisionData createSampleRevisionData() {
-    RevisionObjectData commitData = new RevisionObjectData(Constants.OBJ_COMMIT, new byte[] {});
-    RevisionObjectData treeData = new RevisionObjectData(Constants.OBJ_TREE, new byte[] {});
-    return new RevisionData(commitData, treeData, Lists.newArrayList());
+    RevisionObjectData commitData =
+        new RevisionObjectData(sampleCommitObjectId, Constants.OBJ_COMMIT, new byte[] {});
+    RevisionObjectData treeData =
+        new RevisionObjectData(sampleTreeObjectId, Constants.OBJ_TREE, new byte[] {});
+    return new RevisionData(Collections.emptyList(), commitData, treeData, Lists.newArrayList());
   }
 }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientTest.java
index a1c2172..c45e7be 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientTest.java
@@ -38,6 +38,7 @@
 import java.io.InputStreamReader;
 import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.Optional;
 import org.apache.http.Header;
 import org.apache.http.client.ClientProtocolException;
@@ -89,10 +90,22 @@
   Header expectedHeader = new BasicHeader("Content-Type", "application/json");
   SyncRefsFilter syncRefsFilter;
 
+  String commitObjectId = "9f8d52853089a3cf00c02ff7bd0817bd4353a95a";
+  String treeObjectId = "77814d216a6cab2ddb9f2877fbbd0febdc0fa608";
+  String blobObjectId = "bb383f5249c68a4cc8c82bdd1228b4a8883ff6e8";
+
   String expectedSendObjectPayload =
-      "{\"label\":\"Replication\",\"ref_name\":\"refs/heads/master\",\"revision_data\":{\"commit_object\":{\"type\":1,\"content\":\"dHJlZSA3NzgxNGQyMTZhNmNhYjJkZGI5ZjI4NzdmYmJkMGZlYmRjMGZhNjA4CnBhcmVudCA5ODNmZjFhM2NmNzQ3MjVhNTNhNWRlYzhkMGMwNjEyMjEyOGY1YThkCmF1dGhvciBHZXJyaXQgVXNlciAxMDAwMDAwIDwxMDAwMDAwQDY5ZWMzOGYwLTM1MGUtNGQ5Yy05NmQ0LWJjOTU2ZjJmYWFhYz4gMTYxMDU3ODY0OCArMDEwMApjb21taXR0ZXIgR2Vycml0IENvZGUgUmV2aWV3IDxyb290QG1hY3plY2gtWFBTLTE1PiAxNjEwNTc4NjQ4ICswMTAwCgpVcGRhdGUgcGF0Y2ggc2V0IDEKClBhdGNoIFNldCAxOgoKKDEgY29tbWVudCkKClBhdGNoLXNldDogMQo\\u003d\"},\"tree_object\":{\"type\":2,\"content\":\"MTAwNjQ0IGJsb2IgYmIzODNmNTI0OWM2OGE0Y2M4YzgyYmRkMTIyOGI0YTg4ODNmZjZlOCAgICBmNzVhNjkwMDRhOTNiNGNjYzhjZTIxNWMxMjgwODYzNmMyYjc1Njc1\"},\"blobs\":[{\"type\":3,\"content\":\"ewogICJjb21tZW50cyI6IFsKICAgIHsKICAgICAgImtleSI6IHsKICAgICAgICAidXVpZCI6ICI5MGI1YWJmZl80ZjY3NTI2YSIsCiAgICAgICAgImZpbGVuYW1lIjogIi9DT01NSVRfTVNHIiwKICAgICAgICAicGF0Y2hTZXRJZCI6IDEKICAgICAgfSwKICAgICAgImxpbmVOYnIiOiA5LAogICAgICAiYXV0aG9yIjogewogICAgICAgICJpZCI6IDEwMDAwMDAKICAgICAgfSwKICAgICAgIndyaXR0ZW5PbiI6ICIyMDIxLTAxLTEzVDIyOjU3OjI4WiIsCiAgICAgICJzaWRlIjogMSwKICAgICAgIm1lc3NhZ2UiOiAidGVzdCBjb21tZW50IiwKICAgICAgInJhbmdlIjogewogICAgICAgICJzdGFydExpbmUiOiA5LAogICAgICAgICJzdGFydENoYXIiOiAyMSwKICAgICAgICAiZW5kTGluZSI6IDksCiAgICAgICAgImVuZENoYXIiOiAzNAogICAgICB9LAogICAgICAicmV2SWQiOiAiZjc1YTY5MDA0YTkzYjRjY2M4Y2UyMTVjMTI4MDg2MzZjMmI3NTY3NSIsCiAgICAgICJzZXJ2ZXJJZCI6ICI2OWVjMzhmMC0zNTBlLTRkOWMtOTZkNC1iYzk1NmYyZmFhYWMiLAogICAgICAidW5yZXNvbHZlZCI6IHRydWUKICAgIH0KICBdCn0\\u003d\"}]}}";
+      "{\"label\":\"Replication\",\"ref_name\":\"refs/heads/master\",\"revision_data\":{\"commit_object\":{\"sha1\":\""
+          + commitObjectId
+          + "\",\"type\":1,\"content\":\"dHJlZSA3NzgxNGQyMTZhNmNhYjJkZGI5ZjI4NzdmYmJkMGZlYmRjMGZhNjA4CnBhcmVudCA5ODNmZjFhM2NmNzQ3MjVhNTNhNWRlYzhkMGMwNjEyMjEyOGY1YThkCmF1dGhvciBHZXJyaXQgVXNlciAxMDAwMDAwIDwxMDAwMDAwQDY5ZWMzOGYwLTM1MGUtNGQ5Yy05NmQ0LWJjOTU2ZjJmYWFhYz4gMTYxMDU3ODY0OCArMDEwMApjb21taXR0ZXIgR2Vycml0IENvZGUgUmV2aWV3IDxyb290QG1hY3plY2gtWFBTLTE1PiAxNjEwNTc4NjQ4ICswMTAwCgpVcGRhdGUgcGF0Y2ggc2V0IDEKClBhdGNoIFNldCAxOgoKKDEgY29tbWVudCkKClBhdGNoLXNldDogMQo\\u003d\"},\"tree_object\":{\"sha1\":\""
+          + treeObjectId
+          + "\",\"type\":2,\"content\":\"MTAwNjQ0IGJsb2IgYmIzODNmNTI0OWM2OGE0Y2M4YzgyYmRkMTIyOGI0YTg4ODNmZjZlOCAgICBmNzVhNjkwMDRhOTNiNGNjYzhjZTIxNWMxMjgwODYzNmMyYjc1Njc1\"},\"blobs\":[{\"sha1\":\""
+          + blobObjectId
+          + "\",\"type\":3,\"content\":\"ewogICJjb21tZW50cyI6IFsKICAgIHsKICAgICAgImtleSI6IHsKICAgICAgICAidXVpZCI6ICI5MGI1YWJmZl80ZjY3NTI2YSIsCiAgICAgICAgImZpbGVuYW1lIjogIi9DT01NSVRfTVNHIiwKICAgICAgICAicGF0Y2hTZXRJZCI6IDEKICAgICAgfSwKICAgICAgImxpbmVOYnIiOiA5LAogICAgICAiYXV0aG9yIjogewogICAgICAgICJpZCI6IDEwMDAwMDAKICAgICAgfSwKICAgICAgIndyaXR0ZW5PbiI6ICIyMDIxLTAxLTEzVDIyOjU3OjI4WiIsCiAgICAgICJzaWRlIjogMSwKICAgICAgIm1lc3NhZ2UiOiAidGVzdCBjb21tZW50IiwKICAgICAgInJhbmdlIjogewogICAgICAgICJzdGFydExpbmUiOiA5LAogICAgICAgICJzdGFydENoYXIiOiAyMSwKICAgICAgICAiZW5kTGluZSI6IDksCiAgICAgICAgImVuZENoYXIiOiAzNAogICAgICB9LAogICAgICAicmV2SWQiOiAiZjc1YTY5MDA0YTkzYjRjY2M4Y2UyMTVjMTI4MDg2MzZjMmI3NTY3NSIsCiAgICAgICJzZXJ2ZXJJZCI6ICI2OWVjMzhmMC0zNTBlLTRkOWMtOTZkNC1iYzk1NmYyZmFhYWMiLAogICAgICAidW5yZXNvbHZlZCI6IHRydWUKICAgIH0KICBdCn0\\u003d\"}]}}";
   String commitObject =
-      "tree 77814d216a6cab2ddb9f2877fbbd0febdc0fa608\n"
+      "tree "
+          + treeObjectId
+          + "\n"
           + "parent 983ff1a3cf74725a53a5dec8d0c06122128f5a8d\n"
           + "author Gerrit User 1000000 <1000000@69ec38f0-350e-4d9c-96d4-bc956f2faaac> 1610578648 +0100\n"
           + "committer Gerrit Code Review <root@maczech-XPS-15> 1610578648 +0100\n"
@@ -105,7 +118,7 @@
           + "\n"
           + "Patch-set: 1\n";
   String treeObject =
-      "100644 blob bb383f5249c68a4cc8c82bdd1228b4a8883ff6e8    f75a69004a93b4ccc8ce215c12808636c2b75675";
+      "100644 blob " + blobObjectId + "    f75a69004a93b4ccc8ce215c12808636c2b75675";
   String blobObject =
       "{\n"
           + "  \"comments\": [\n"
@@ -416,9 +429,12 @@
 
   private RevisionData createSampleRevisionData() {
     RevisionObjectData commitData =
-        new RevisionObjectData(Constants.OBJ_COMMIT, commitObject.getBytes());
-    RevisionObjectData treeData = new RevisionObjectData(Constants.OBJ_TREE, treeObject.getBytes());
-    RevisionObjectData blobData = new RevisionObjectData(Constants.OBJ_BLOB, blobObject.getBytes());
-    return new RevisionData(commitData, treeData, Lists.newArrayList(blobData));
+        new RevisionObjectData(commitObjectId, Constants.OBJ_COMMIT, commitObject.getBytes());
+    RevisionObjectData treeData =
+        new RevisionObjectData(treeObjectId, Constants.OBJ_TREE, treeObject.getBytes());
+    RevisionObjectData blobData =
+        new RevisionObjectData(blobObjectId, Constants.OBJ_BLOB, blobObject.getBytes());
+    return new RevisionData(
+        Collections.emptyList(), commitData, treeData, Lists.newArrayList(blobData));
   }
 }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/fetch/ApplyObjectIT.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/fetch/ApplyObjectIT.java
index 0f063a4..161830b 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/fetch/ApplyObjectIT.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/fetch/ApplyObjectIT.java
@@ -78,14 +78,15 @@
     String refName = RefNames.changeMetaRef(pushResult.getChange().getId());
 
     Optional<RevisionData> revisionData =
-        reader.read(Project.nameKey(testRepoProjectName), refName);
+        reader.read(
+            Project.nameKey(testRepoProjectName), pushResult.getCommit().toObjectId(), refName, 0);
 
     RefSpec refSpec = new RefSpec(refName);
-    objectUnderTest.apply(project, refSpec, revisionData.get());
+    objectUnderTest.apply(project, refSpec, toArray(revisionData));
     try (Repository repo = repoManager.openRepository(project);
         TestRepository<Repository> testRepo = new TestRepository<>(repo); ) {
       Optional<RevisionData> newRevisionData =
-          reader.read(project, repo.exactRef(refName).getObjectId(), refName);
+          reader.read(project, repo.exactRef(refName).getObjectId(), refName, 0);
       compareObjects(revisionData.get(), newRevisionData);
       testRepo.fsck();
     }
@@ -99,15 +100,15 @@
     createChange();
     String seqChangesRef = RefNames.REFS_SEQUENCES + "changes";
 
-    Optional<RevisionData> revisionData = reader.read(allProjects, seqChangesRef);
+    Optional<RevisionData> revisionData = reader.read(allProjects, seqChangesRef, 0);
 
     RefSpec refSpec = new RefSpec(seqChangesRef);
-    objectUnderTest.apply(project, refSpec, revisionData.get());
+    objectUnderTest.apply(project, refSpec, toArray(revisionData));
     try (Repository repo = repoManager.openRepository(project);
         TestRepository<Repository> testRepo = new TestRepository<>(repo); ) {
 
       Optional<RevisionData> newRevisionData =
-          reader.read(project, repo.exactRef(seqChangesRef).getObjectId(), seqChangesRef);
+          reader.read(project, repo.exactRef(seqChangesRef).getObjectId(), seqChangesRef, 0);
       compareObjects(revisionData.get(), newRevisionData);
       testRepo.fsck();
     }
@@ -126,8 +127,8 @@
     NameKey testRepoKey = Project.nameKey(testRepoProjectName);
     try (Repository repo = repoManager.openRepository(testRepoKey)) {
       Optional<RevisionData> revisionData =
-          reader.read(testRepoKey, repo.exactRef(refName).getObjectId(), refName);
-      objectUnderTest.apply(project, refSpec, revisionData.get());
+          reader.read(testRepoKey, repo.exactRef(refName).getObjectId(), refName, 0);
+      objectUnderTest.apply(project, refSpec, toArray(revisionData));
     }
 
     ReviewInput reviewInput = new ReviewInput();
@@ -138,12 +139,12 @@
     try (Repository repo = repoManager.openRepository(project);
         TestRepository<Repository> testRepo = new TestRepository<>(repo)) {
       Optional<RevisionData> revisionDataWithComment =
-          reader.read(testRepoKey, repo.exactRef(refName).getObjectId(), refName);
+          reader.read(testRepoKey, repo.exactRef(refName).getObjectId(), refName, 0);
 
-      objectUnderTest.apply(project, refSpec, revisionDataWithComment.get());
+      objectUnderTest.apply(project, refSpec, toArray(revisionDataWithComment));
 
       Optional<RevisionData> newRevisionData =
-          reader.read(project, repo.exactRef(refName).getObjectId(), refName);
+          reader.read(project, repo.exactRef(refName).getObjectId(), refName, 0);
 
       compareObjects(revisionDataWithComment.get(), newRevisionData);
 
@@ -168,12 +169,12 @@
       gApi.changes().id(changeId.get()).current().review(reviewInput);
 
       Optional<RevisionData> revisionData =
-          reader.read(createTestProject, repo.exactRef(refName).getObjectId(), refName);
+          reader.read(createTestProject, repo.exactRef(refName).getObjectId(), refName, 0);
 
       RefSpec refSpec = new RefSpec(refName);
       assertThrows(
           MissingParentObjectException.class,
-          () -> objectUnderTest.apply(project, refSpec, revisionData.get()));
+          () -> objectUnderTest.apply(project, refSpec, toArray(revisionData)));
     }
   }
 
@@ -229,4 +230,10 @@
       bind(ApplyObject.class);
     }
   }
+
+  private RevisionData[] toArray(Optional<RevisionData> optional) {
+    ImmutableList.Builder<RevisionData> listBuilder = ImmutableList.builder();
+    optional.ifPresent(listBuilder::add);
+    return listBuilder.build().toArray(new RevisionData[1]);
+  }
 }