Honour the fetch ref-spec in replication.config

When an individual ref was fetched as a result of invoking
the fetch REST-API, the ref-spec built on-the-fly did not
respect the ref-specs configured in replication.config.

Even worse, refs that were NOT supposed to be fetched were
not discarded, causing potentially unwanted replications.

Bug: Issue 15426
Change-Id: I7ca9f9de39ec914125b0389dc2ae35054ccd80af
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchOne.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchOne.java
index 32dfe46..cee52b5 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchOne.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchOne.java
@@ -41,6 +41,7 @@
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -373,10 +374,25 @@
   }
 
   private List<RefSpec> getFetchRefSpecs() {
+    List<RefSpec> configRefSpecs = config.getFetchRefSpecs();
     if (delta.isEmpty()) {
-      return config.getFetchRefSpecs();
+      return configRefSpecs;
     }
-    return delta.stream().map(ref -> new RefSpec(ref + ":" + ref)).collect(Collectors.toList());
+
+    return delta.stream()
+        .map(ref -> refToFetchRefSpec(ref, configRefSpecs))
+        .filter(Optional::isPresent)
+        .map(Optional::get)
+        .collect(Collectors.toList());
+  }
+
+  private Optional<RefSpec> refToFetchRefSpec(String ref, List<RefSpec> configRefSpecs) {
+    for (RefSpec refSpec : configRefSpecs) {
+      if (refSpec.matchSource(ref)) {
+        return Optional.of(refSpec.expandFromSource(ref));
+      }
+    }
+    return Optional.empty();
   }
 
   private void updateStates(List<RefUpdateState> refUpdates) throws IOException {
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 fdfe3fd..c4b50b5 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
@@ -31,7 +31,6 @@
 import com.googlesource.gerrit.plugins.replication.pull.FetchResultProcessing.GitUpdateProcessing;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionData;
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.MissingParentObjectException;
-import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
 import com.googlesource.gerrit.plugins.replication.pull.client.FetchRestApiClient;
 import com.googlesource.gerrit.plugins.replication.pull.client.HttpResult;
 import com.googlesource.gerrit.plugins.replication.pull.filter.ExcludedRefsFilter;
@@ -189,7 +188,7 @@
     try {
       fetchCallsPool = new ForkJoinPool(sources.get().getAll().size());
 
-      final Consumer<Source> callFunction = callFunction(project, refName, state);
+      final Consumer<Source> callFunction = callFunction(project, objectId, refName, state);
       fetchCallsPool
           .submit(() -> sources.get().getAll().parallelStream().forEach(callFunction))
           .get(fetchCallsTimeout, TimeUnit.MILLISECONDS);
@@ -207,8 +206,9 @@
     }
   }
 
-  private Consumer<Source> callFunction(NameKey project, String refName, ReplicationState state) {
-    CallFunction call = getCallFunction(project, refName, state);
+  private Consumer<Source> callFunction(
+      NameKey project, ObjectId objectId, String refName, ReplicationState state) {
+    CallFunction call = getCallFunction(project, objectId, refName, state);
 
     return (source) -> {
       try {
@@ -219,13 +219,14 @@
     };
   }
 
-  private CallFunction getCallFunction(NameKey project, String refName, ReplicationState state) {
+  private CallFunction getCallFunction(
+      NameKey project, ObjectId objectId, String refName, ReplicationState state) {
     try {
-      Optional<RevisionData> revisionData = revisionReader.read(project, refName);
+      Optional<RevisionData> revisionData = revisionReader.read(project, objectId, refName);
       if (revisionData.isPresent()) {
         return ((source) -> callSendObject(source, project, refName, revisionData.get(), state));
       }
-    } catch (InvalidObjectIdException | IOException | RefUpdateException e) {
+    } catch (InvalidObjectIdException | IOException e) {
       stateLog.error(
           String.format(
               "Exception during reading ref: %s, project:%s, message: %s",
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 a99256f..f916e59 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
@@ -23,7 +23,6 @@
 import com.googlesource.gerrit.plugins.replication.ReplicationConfig;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionData;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionObjectData;
-import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
 import java.io.IOException;
 import java.util.List;
 import java.util.Optional;
@@ -37,7 +36,6 @@
 import org.eclipse.jgit.lib.Constants;
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.lib.ObjectLoader;
-import org.eclipse.jgit.lib.Ref;
 import org.eclipse.jgit.lib.Repository;
 import org.eclipse.jgit.revwalk.RevCommit;
 import org.eclipse.jgit.revwalk.RevTree;
@@ -58,19 +56,13 @@
             .getLong("replication", CONFIG_MAX_API_PAYLOAD_SIZE, DEFAULT_MAX_PAYLOAD_SIZE_IN_BYTES);
   }
 
-  public Optional<RevisionData> read(Project.NameKey project, String refName)
+  public Optional<RevisionData> read(Project.NameKey project, ObjectId objectId, String refName)
       throws MissingObjectException, IncorrectObjectTypeException, CorruptObjectException,
-          RepositoryNotFoundException, RefUpdateException, IOException {
+          RepositoryNotFoundException, IOException {
     try (Repository git = gitRepositoryManager.openRepository(project)) {
-      Ref head = git.exactRef(refName);
-      if (head == null) {
-        throw new RefUpdateException(
-            String.format("Cannot find ref %s in project %s", refName, project.get()));
-      }
-
       Long totalRefSize = 0l;
 
-      ObjectLoader commitLoader = git.open(head.getObjectId());
+      ObjectLoader commitLoader = git.open(objectId);
       totalRefSize += commitLoader.getSize();
       verifySize(totalRefSize, commitLoader);
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationIT.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationIT.java
index c5b1790..cf94086 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationIT.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationIT.java
@@ -15,6 +15,10 @@
 package com.googlesource.gerrit.plugins.replication.pull;
 
 import static com.google.common.truth.Truth.assertThat;
+import static com.google.gerrit.acceptance.GitUtil.fetch;
+import static com.google.gerrit.acceptance.GitUtil.pushOne;
+import static com.google.gerrit.acceptance.testsuite.project.TestProjectUpdate.allow;
+import static com.google.gerrit.server.group.SystemGroupBackend.REGISTERED_USERS;
 import static java.util.stream.Collectors.toList;
 
 import com.google.common.flogger.FluentLogger;
@@ -24,7 +28,10 @@
 import com.google.gerrit.acceptance.TestPlugin;
 import com.google.gerrit.acceptance.UseLocalDisk;
 import com.google.gerrit.acceptance.testsuite.project.ProjectOperations;
+import com.google.gerrit.entities.Permission;
 import com.google.gerrit.entities.Project;
+import com.google.gerrit.entities.Project.NameKey;
+import com.google.gerrit.entities.RefNames;
 import com.google.gerrit.extensions.api.changes.NotifyHandling;
 import com.google.gerrit.extensions.api.projects.BranchInput;
 import com.google.gerrit.extensions.common.Input;
@@ -47,16 +54,23 @@
 import java.nio.file.Path;
 import java.time.Duration;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
 import java.util.function.Supplier;
 import org.eclipse.jgit.errors.ConfigInvalidException;
+import org.eclipse.jgit.internal.storage.dfs.InMemoryRepository;
+import org.eclipse.jgit.junit.TestRepository;
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.lib.Ref;
 import org.eclipse.jgit.lib.Repository;
 import org.eclipse.jgit.revwalk.RevCommit;
 import org.eclipse.jgit.storage.file.FileBasedConfig;
+import org.eclipse.jgit.transport.PushResult;
 import org.eclipse.jgit.transport.ReceiveCommand;
+import org.eclipse.jgit.transport.RemoteRefUpdate;
+import org.eclipse.jgit.transport.RemoteRefUpdate.Status;
 import org.eclipse.jgit.util.FS;
 import org.junit.Test;
 
@@ -69,7 +83,7 @@
   private static final Optional<String> ALL_PROJECTS = Optional.empty();
   private static final FluentLogger logger = FluentLogger.forEnclosingClass();
   private static final int TEST_REPLICATION_DELAY = 60;
-  private static final Duration TEST_TIMEOUT = Duration.ofSeconds(TEST_REPLICATION_DELAY * 2);
+  private static final Duration TEST_TIMEOUT = Duration.ofSeconds(TEST_REPLICATION_DELAY * 2000);
   private static final String TEST_REPLICATION_SUFFIX = "suffix1";
   private static final String TEST_REPLICATION_REMOTE = "remote1";
 
@@ -161,6 +175,76 @@
   }
 
   @Test
+  @UseLocalDisk
+  public void shouldReplicateForceUpdatedBranch() throws Exception {
+    boolean forcedPush = true;
+    String testProjectName = project + TEST_REPLICATION_SUFFIX;
+    NameKey testProjectNameKey = createTestProject(testProjectName);
+
+    String newBranch = "refs/heads/mybranch";
+    String master = "refs/heads/master";
+    BranchInput input = new BranchInput();
+    input.revision = master;
+    gApi.projects().name(testProjectName).branch(newBranch).create(input);
+
+    projectOperations
+        .project(testProjectNameKey)
+        .forUpdate()
+        .add(allow(Permission.PUSH).ref(newBranch).group(REGISTERED_USERS).force(true))
+        .update();
+
+    String branchRevision = gApi.projects().name(testProjectName).branch(newBranch).get().revision;
+
+    ReplicationQueue pullReplicationQueue =
+        plugin.getSysInjector().getInstance(ReplicationQueue.class);
+    GitReferenceUpdatedListener.Event event =
+        new FakeGitReferenceUpdatedEvent(
+            project,
+            newBranch,
+            ObjectId.zeroId().getName(),
+            branchRevision,
+            ReceiveCommand.Type.CREATE);
+    pullReplicationQueue.onGitReferenceUpdated(event);
+
+    try (Repository repo = repoManager.openRepository(project)) {
+      waitUntil(() -> checkedGetRef(repo, newBranch) != null);
+
+      Ref targetBranchRef = getRef(repo, newBranch);
+      assertThat(targetBranchRef).isNotNull();
+      assertThat(targetBranchRef.getObjectId().getName()).isEqualTo(branchRevision);
+    }
+
+    TestRepository<InMemoryRepository> testProject = cloneProject(testProjectNameKey);
+    fetch(testProject, RefNames.REFS_HEADS + "*:" + RefNames.REFS_HEADS + "*");
+    RevCommit amendedCommit = testProject.amendRef(newBranch).message("Amended commit").create();
+    PushResult pushResult =
+        pushOne(testProject, newBranch, newBranch, false, forcedPush, Collections.emptyList());
+    Collection<RemoteRefUpdate> pushedRefs = pushResult.getRemoteUpdates();
+    assertThat(pushedRefs).hasSize(1);
+    assertThat(pushedRefs.iterator().next().getStatus()).isEqualTo(Status.OK);
+
+    GitReferenceUpdatedListener.Event forcedPushEvent =
+        new FakeGitReferenceUpdatedEvent(
+            project,
+            newBranch,
+            branchRevision,
+            amendedCommit.getId().getName(),
+            ReceiveCommand.Type.UPDATE_NONFASTFORWARD);
+    pullReplicationQueue.onGitReferenceUpdated(forcedPushEvent);
+
+    try (Repository repo = repoManager.openRepository(project);
+        Repository sourceRepo = repoManager.openRepository(project)) {
+      waitUntil(
+          () ->
+              checkedGetRef(repo, newBranch) != null
+                  && checkedGetRef(repo, newBranch)
+                      .getObjectId()
+                      .getName()
+                      .equals(amendedCommit.getId().getName()));
+    }
+  }
+
+  @Test
   public void shouldReplicateNewChangeRefCGitClient() throws Exception {
     AutoReloadConfigDecorator autoReloadConfigDecorator =
         getInstance(AutoReloadConfigDecorator.class);
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 8402782..d742984 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
@@ -99,7 +99,7 @@
     when(source.getApis()).thenReturn(apis);
     when(sourceCollection.getAll()).thenReturn(Lists.newArrayList(source));
     when(rd.get()).thenReturn(sourceCollection);
-    when(revReader.read(any(), anyString())).thenReturn(Optional.of(revisionData));
+    when(revReader.read(any(), any(), anyString())).thenReturn(Optional.of(revisionData));
     when(fetchClientFactory.create(any())).thenReturn(fetchRestApiClient);
     when(fetchRestApiClient.callSendObject(any(), anyString(), any(), any()))
         .thenReturn(httpResult);
@@ -161,7 +161,7 @@
     Event event = new TestEvent("refs/changes/01/1/meta");
     objectUnderTest.start();
 
-    when(revReader.read(any(), anyString())).thenThrow(IOException.class);
+    when(revReader.read(any(), any(), anyString())).thenThrow(IOException.class);
 
     objectUnderTest.onGitReferenceUpdated(event);
 
@@ -174,7 +174,7 @@
     Event event = new TestEvent("refs/changes/01/1/1");
     objectUnderTest.start();
 
-    when(revReader.read(any(), anyString())).thenReturn(Optional.empty());
+    when(revReader.read(any(), any(), anyString())).thenReturn(Optional.empty());
 
     objectUnderTest.onGitReferenceUpdated(event);
 
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 fdd63d2..e300613 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
@@ -37,8 +37,12 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionData;
 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.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.junit.Before;
 import org.junit.Test;
 
@@ -59,7 +63,8 @@
     Result pushResult = createChange();
     String refName = RefNames.changeMetaRef(pushResult.getChange().getId());
 
-    Optional<RevisionData> revisionDataOption = objectUnderTest.read(project, refName);
+    Optional<RevisionData> revisionDataOption =
+        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId));
 
     assertThat(revisionDataOption.isPresent()).isTrue();
     RevisionData revisionData = revisionDataOption.get();
@@ -75,6 +80,20 @@
     assertThat(revisionData.getBlobs()).isEmpty();
   }
 
+  private Optional<RevisionData> readRevisionFromObjectUnderTest(String refName, ObjectId objId) {
+    try {
+      return objectUnderTest.read(project, objId, refName);
+    } catch (Exception e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  protected Optional<ObjectId> refObjectId(String refName) throws IOException {
+    try (Repository repo = repoManager.openRepository(project)) {
+      return Optional.ofNullable(repo.exactRef(refName)).map(Ref::getObjectId);
+    }
+  }
+
   @Test
   public void shouldReadRefMetaObjectWithComments() throws Exception {
     Result pushResult = createChange();
@@ -87,7 +106,8 @@
     reviewInput.comments = ImmutableMap.of(Patch.COMMIT_MSG, ImmutableList.of(comment));
     gApi.changes().id(changeId.get()).current().review(reviewInput);
 
-    Optional<RevisionData> revisionDataOption = objectUnderTest.read(project, refName);
+    Optional<RevisionData> revisionDataOption =
+        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId));
 
     assertThat(revisionDataOption.isPresent()).isTrue();
     RevisionData revisionData = revisionDataOption.get();
@@ -109,8 +129,9 @@
   @Test
   public void shouldNotReadRefsSequences() throws Exception {
     createChange().assertOkStatus();
+    String refName = RefNames.REFS_SEQUENCES + Sequences.NAME_CHANGES;
     Optional<RevisionData> revisionDataOption =
-        objectUnderTest.read(allProjects, RefNames.REFS_SEQUENCES + Sequences.NAME_CHANGES);
+        refObjectId(refName).flatMap(objId -> readRevisionFromObjectUnderTest(refName, objId));
 
     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 34aaf8b..cef1051 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
@@ -55,6 +55,7 @@
 import org.apache.http.entity.StringEntity;
 import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.message.BasicHeader;
+import org.eclipse.jgit.lib.Repository;
 import org.eclipse.jgit.storage.file.FileBasedConfig;
 import org.eclipse.jgit.util.FS;
 
@@ -147,7 +148,9 @@
 
   protected Optional<RevisionData> createRevisionData(NameKey projectName, String refName)
       throws Exception {
-    return revisionReader.read(projectName, refName);
+    try (Repository repository = repoManager.openRepository(projectName)) {
+      return revisionReader.read(projectName, repository.exactRef(refName).getObjectId(), refName);
+    }
   }
 
   protected Object encode(byte[] content) {
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 f6cd342..e2a162e 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
@@ -29,6 +29,7 @@
 import com.google.gerrit.entities.Change;
 import com.google.gerrit.entities.Patch;
 import com.google.gerrit.entities.Project;
+import com.google.gerrit.entities.Project.NameKey;
 import com.google.gerrit.entities.RefNames;
 import com.google.gerrit.extensions.api.changes.ReviewInput;
 import com.google.gerrit.extensions.api.changes.ReviewInput.CommentInput;
@@ -77,16 +78,16 @@
     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);
 
     RefSpec refSpec = new RefSpec(refName);
     objectUnderTest.apply(project, refSpec, revisionData.get());
-    Optional<RevisionData> newRevisionData = reader.read(project, refName);
-
-    compareObjects(revisionData.get(), newRevisionData);
-
     try (Repository repo = repoManager.openRepository(project);
         TestRepository<Repository> testRepo = new TestRepository<>(repo); ) {
+      Optional<RevisionData> newRevisionData =
+          reader.read(project, repo.exactRef(refName).getObjectId(), refName);
+      compareObjects(revisionData.get(), newRevisionData);
       testRepo.fsck();
     }
   }
@@ -101,26 +102,30 @@
     String refName = RefNames.changeMetaRef(changeId);
     RefSpec refSpec = new RefSpec(refName);
 
-    Optional<RevisionData> revisionData =
-        reader.read(Project.nameKey(testRepoProjectName), refName);
-    objectUnderTest.apply(project, refSpec, revisionData.get());
+    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());
+    }
 
     ReviewInput reviewInput = new ReviewInput();
     CommentInput comment = createCommentInput(1, 0, 1, 1, "Test comment");
     reviewInput.comments = ImmutableMap.of(Patch.COMMIT_MSG, ImmutableList.of(comment));
     gApi.changes().id(changeId.get()).current().review(reviewInput);
 
-    Optional<RevisionData> revisionDataWithComment =
-        reader.read(Project.nameKey(testRepoProjectName), refName);
-
-    objectUnderTest.apply(project, refSpec, revisionDataWithComment.get());
-
-    Optional<RevisionData> newRevisionData = reader.read(project, refName);
-
-    compareObjects(revisionDataWithComment.get(), newRevisionData);
-
     try (Repository repo = repoManager.openRepository(project);
         TestRepository<Repository> testRepo = new TestRepository<>(repo)) {
+      Optional<RevisionData> revisionDataWithComment =
+          reader.read(testRepoKey, repo.exactRef(refName).getObjectId(), refName);
+
+      objectUnderTest.apply(project, refSpec, revisionDataWithComment.get());
+
+      Optional<RevisionData> newRevisionData =
+          reader.read(project, repo.exactRef(refName).getObjectId(), refName);
+
+      compareObjects(revisionDataWithComment.get(), newRevisionData);
+
       testRepo.fsck();
     }
   }
@@ -128,24 +133,27 @@
   @Test
   public void shouldThrowExceptionWhenParentCommitObjectIsMissing() throws Exception {
     String testRepoProjectName = project + TEST_REPLICATION_SUFFIX;
-    testRepo = cloneProject(createTestProject(testRepoProjectName));
+    NameKey createTestProject = createTestProject(testRepoProjectName);
+    try (Repository repo = repoManager.openRepository(createTestProject)) {
+      testRepo = cloneProject(createTestProject);
 
-    Result pushResult = createChange();
-    Change.Id changeId = pushResult.getChange().getId();
-    String refName = RefNames.changeMetaRef(changeId);
+      Result pushResult = createChange();
+      Change.Id changeId = pushResult.getChange().getId();
+      String refName = RefNames.changeMetaRef(changeId);
 
-    CommentInput comment = createCommentInput(1, 0, 1, 1, "Test comment");
-    ReviewInput reviewInput = new ReviewInput();
-    reviewInput.comments = ImmutableMap.of(Patch.COMMIT_MSG, ImmutableList.of(comment));
-    gApi.changes().id(changeId.get()).current().review(reviewInput);
+      CommentInput comment = createCommentInput(1, 0, 1, 1, "Test comment");
+      ReviewInput reviewInput = new ReviewInput();
+      reviewInput.comments = ImmutableMap.of(Patch.COMMIT_MSG, ImmutableList.of(comment));
+      gApi.changes().id(changeId.get()).current().review(reviewInput);
 
-    Optional<RevisionData> revisionData =
-        reader.read(Project.nameKey(testRepoProjectName), refName);
+      Optional<RevisionData> revisionData =
+          reader.read(createTestProject, repo.exactRef(refName).getObjectId(), refName);
 
-    RefSpec refSpec = new RefSpec(refName);
-    assertThrows(
-        MissingParentObjectException.class,
-        () -> objectUnderTest.apply(project, refSpec, revisionData.get()));
+      RefSpec refSpec = new RefSpec(refName);
+      assertThrows(
+          MissingParentObjectException.class,
+          () -> objectUnderTest.apply(project, refSpec, revisionData.get()));
+    }
   }
 
   private void compareObjects(RevisionData expected, Optional<RevisionData> actualOption) {