Merge branch 'stable-2.16' into stable-3.0

* stable-2.16:
  Revert "Revert "Do not reload config when queue is not ready""
  Add replication refs-filtering before push

Change-Id: I99b0902b8f909745c3eb00490d615935cec62db1
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/AutoReloadConfigDecorator.java b/src/main/java/com/googlesource/gerrit/plugins/replication/AutoReloadConfigDecorator.java
index bec2019..171069d 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/AutoReloadConfigDecorator.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/AutoReloadConfigDecorator.java
@@ -97,9 +97,13 @@
   private synchronized void reloadIfNeeded() {
     if (isAutoReload()) {
       ReplicationQueue queue = replicationQueue.get();
+
       long lastModified = getLastModified(currentConfig);
       try {
-        if (lastModified > currentConfigTs && lastModified > lastFailedConfigTs) {
+        if (lastModified > currentConfigTs
+            && lastModified > lastFailedConfigTs
+            && queue.isRunning()
+            && !queue.isReplaying()) {
           queue.stop();
           currentConfig = loadConfig();
           currentConfigTs = lastModified;
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/PushOne.java b/src/main/java/com/googlesource/gerrit/plugins/replication/PushOne.java
index 747b6ba..56cecfe 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/PushOne.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/PushOne.java
@@ -23,6 +23,7 @@
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Sets;
 import com.google.gerrit.extensions.events.GitReferenceUpdatedListener;
+import com.google.gerrit.extensions.registration.DynamicItem;
 import com.google.gerrit.extensions.restapi.AuthException;
 import com.google.gerrit.extensions.restapi.ResourceConflictException;
 import com.google.gerrit.metrics.Timer1;
@@ -114,6 +115,8 @@
   private final ProjectCache projectCache;
   private final CreateProjectTask.Factory createProjectFactory;
   private final AtomicBoolean canceledWhileRunning;
+  private final TransportFactory transportFactory;
+  private DynamicItem<ReplicationPushFilter> replicationPushFilter;
 
   @Inject
   PushOne(
@@ -129,6 +132,7 @@
       ReplicationMetrics m,
       ProjectCache pc,
       CreateProjectTask.Factory cpf,
+      TransportFactory tf,
       @Assisted Project.NameKey d,
       @Assisted URIish u) {
     gitManager = grm;
@@ -150,6 +154,12 @@
     createProjectFactory = cpf;
     canceledWhileRunning = new AtomicBoolean(false);
     maxRetries = p.getMaxRetries();
+    transportFactory = tf;
+  }
+
+  @Inject(optional = true)
+  public void setReplicationPushFilter(DynamicItem<ReplicationPushFilter> replicationPushFilter) {
+    this.replicationPushFilter = replicationPushFilter;
   }
 
   @Override
@@ -440,7 +450,7 @@
 
   private void runImpl() throws IOException, PermissionBackendException {
     PushResult res;
-    try (Transport tn = Transport.open(git, uri)) {
+    try (Transport tn = transportFactory.open(git, uri)) {
       res = pushVia(tn);
     }
     updateStates(res.getRemoteUpdates());
@@ -509,7 +519,12 @@
       local = forProject.filter(local, git, RefFilterOptions.builder().setFilterMeta(true).build());
     }
 
-    return pushAllRefs ? doPushAll(tn, local) : doPushDelta(local);
+    List<RemoteRefUpdate> remoteUpdatesList =
+        pushAllRefs ? doPushAll(tn, local) : doPushDelta(local);
+
+    return replicationPushFilter == null || replicationPushFilter.get() == null
+        ? remoteUpdatesList
+        : replicationPushFilter.get().filter(projectName.get(), remoteUpdatesList);
   }
 
   private List<RemoteRefUpdate> doPushAll(Transport tn, Map<String, Ref> local)
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationExtensionPointModule.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationExtensionPointModule.java
new file mode 100644
index 0000000..b92a54a
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationExtensionPointModule.java
@@ -0,0 +1,32 @@
+// Copyright (C) 2019 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;
+
+import com.google.gerrit.extensions.registration.DynamicItem;
+import com.google.inject.AbstractModule;
+
+/**
+ * Gerrit libModule for applying a ref-filter for outgoing replications.
+ *
+ * <p>It should be used only when an actual filter is defined, otherwise the default replication
+ * plugin behaviour will be pushing all refs without any filtering.
+ */
+public class ReplicationExtensionPointModule extends AbstractModule {
+
+  @Override
+  protected void configure() {
+    DynamicItem.itemOf(binder(), ReplicationPushFilter.class);
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationModule.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationModule.java
index 9b7e8e6..f9a2b2c 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationModule.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationModule.java
@@ -66,5 +66,7 @@
     EventTypes.register(RefReplicationDoneEvent.TYPE, RefReplicationDoneEvent.class);
     EventTypes.register(ReplicationScheduledEvent.TYPE, ReplicationScheduledEvent.class);
     bind(SshSessionFactory.class).toProvider(ReplicationSshSessionFactoryProvider.class);
+
+    bind(TransportFactory.class).to(TransportFactoryImpl.class).in(Scopes.SINGLETON);
   }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationPushFilter.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationPushFilter.java
new file mode 100644
index 0000000..eb6ba90
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationPushFilter.java
@@ -0,0 +1,30 @@
+// Copyright (C) 2019 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;
+
+import com.google.gerrit.extensions.annotations.ExtensionPoint;
+import java.util.List;
+import org.eclipse.jgit.transport.RemoteRefUpdate;
+
+/**
+ * Filter that is invoked before list of remote ref updates is pushed to remote instance.
+ *
+ * <p>It can be used to filter out unwanted updates.
+ */
+@ExtensionPoint
+public interface ReplicationPushFilter {
+
+  public List<RemoteRefUpdate> filter(String projectName, List<RemoteRefUpdate> remoteUpdatesList);
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationQueue.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationQueue.java
index d2a34be..6229686 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationQueue.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationQueue.java
@@ -47,6 +47,7 @@
   private final ReplicationState.Factory replicationStateFactory;
   private final EventsStorage eventsStorage;
   private volatile boolean running;
+  private volatile boolean replaying;
 
   @Inject
   ReplicationQueue(
@@ -82,6 +83,14 @@
     }
   }
 
+  public boolean isRunning() {
+    return running;
+  }
+
+  public boolean isReplaying() {
+    return replaying;
+  }
+
   void scheduleFullSync(Project.NameKey project, String urlMatch, ReplicationState state) {
     scheduleFullSync(project, urlMatch, state, false);
   }
@@ -129,9 +138,14 @@
   }
 
   private void firePendingEvents() {
-    for (EventsStorage.ReplicateRefUpdate e : eventsStorage.list()) {
-      repLog.info("Firing pending event {}", e);
-      onGitReferenceUpdated(e.project, e.ref);
+    replaying = true;
+    try {
+      for (EventsStorage.ReplicateRefUpdate e : eventsStorage.list()) {
+        repLog.info("Firing pending event {}", e);
+        onGitReferenceUpdated(e.project, e.ref);
+      }
+    } finally {
+      replaying = false;
     }
   }
 
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/TransportFactory.java b/src/main/java/com/googlesource/gerrit/plugins/replication/TransportFactory.java
new file mode 100644
index 0000000..ba14299
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/TransportFactory.java
@@ -0,0 +1,26 @@
+// Copyright (C) 2019 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;
+
+import org.eclipse.jgit.errors.NotSupportedException;
+import org.eclipse.jgit.errors.TransportException;
+import org.eclipse.jgit.lib.Repository;
+import org.eclipse.jgit.transport.Transport;
+import org.eclipse.jgit.transport.URIish;
+
+public interface TransportFactory {
+
+  Transport open(Repository local, URIish uri) throws NotSupportedException, TransportException;
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/TransportFactoryImpl.java b/src/main/java/com/googlesource/gerrit/plugins/replication/TransportFactoryImpl.java
new file mode 100644
index 0000000..58c1214
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/TransportFactoryImpl.java
@@ -0,0 +1,30 @@
+// Copyright (C) 2019 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;
+
+import org.eclipse.jgit.errors.NotSupportedException;
+import org.eclipse.jgit.errors.TransportException;
+import org.eclipse.jgit.lib.Repository;
+import org.eclipse.jgit.transport.Transport;
+import org.eclipse.jgit.transport.URIish;
+
+public class TransportFactoryImpl implements TransportFactory {
+
+  @Override
+  public Transport open(Repository git, URIish uri)
+      throws NotSupportedException, TransportException {
+    return Transport.open(git, uri);
+  }
+}
diff --git a/src/main/resources/Documentation/extension-point.md b/src/main/resources/Documentation/extension-point.md
new file mode 100644
index 0000000..345fd8f
--- /dev/null
+++ b/src/main/resources/Documentation/extension-point.md
@@ -0,0 +1,39 @@
+@PLUGIN@ extension points
+==============
+
+The replication plugin exposes an extension point to allow influencing the behaviour of the replication events from another plugin or a script.
+Extension points can be defined from the replication plugin only when it is loaded as [libModule](/config-gerrit.html#gerrit.installModule) and
+implemented by another plugin by declaring a `provided` dependency from the replication plugin.
+
+### Install extension libModule
+
+The replication plugin's extension points are defined in the `c.g.g.p.r.ReplicationExtensionPointModule`
+that needs to be configured as libModule.
+
+Create a symbolic link from `$GERRIT_SITE/plugins/replication.jar` into `$GERRIT_SITE/lib`
+and then add the replication extension module to the `gerrit.config`.
+
+Example:
+
+```
+[gerrit]
+  installModule = com.googlesource.gerrit.plugins.replication.ReplicationExtensionPointModule
+```
+
+> **NOTE**: Use and configuration of the replication plugin as library module requires a Gerrit server restart and does not support hot plugin install or upgrade.
+
+
+### Extension points
+
+* `com.googlesource.gerrit.plugins.replication.ReplicationPushFilter`
+
+  Filter out the ref updates pushed to a remote instance.
+  Only one filter at a time is supported. Filter implementation needs to bind a `DynamicItem`.
+
+  Default: no filtering
+
+  Example:
+
+  ```
+  DynamicItem.bind(binder(),ReplicationPushFilter.class).to(ReplicationPushFilterImpl.class);
+  ```
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/PushOneTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/PushOneTest.java
new file mode 100644
index 0000000..836da2f
--- /dev/null
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/PushOneTest.java
@@ -0,0 +1,384 @@
+// Copyright (C) 2019 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;
+
+import static com.googlesource.gerrit.plugins.replication.RemoteRefUpdateCollectionMatcher.eqRemoteRef;
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.createNiceMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.getCurrentArguments;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.eclipse.jgit.lib.Ref.Storage.NEW;
+
+import com.google.common.collect.ImmutableList;
+import com.google.gerrit.extensions.registration.DynamicItem;
+import com.google.gerrit.metrics.Timer1;
+import com.google.gerrit.reviewdb.client.Project;
+import com.google.gerrit.server.git.GitRepositoryManager;
+import com.google.gerrit.server.git.PerThreadRequestScope;
+import com.google.gerrit.server.permissions.PermissionBackend;
+import com.google.gerrit.server.permissions.PermissionBackend.ForProject;
+import com.google.gerrit.server.permissions.PermissionBackend.WithUser;
+import com.google.gerrit.server.project.ProjectCache;
+import com.google.gerrit.server.project.ProjectState;
+import com.google.gerrit.server.util.IdGenerator;
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import junit.framework.AssertionFailedError;
+import org.easymock.IAnswer;
+import org.eclipse.jgit.errors.NotSupportedException;
+import org.eclipse.jgit.errors.TransportException;
+import org.eclipse.jgit.lib.Config;
+import org.eclipse.jgit.lib.ObjectId;
+import org.eclipse.jgit.lib.ObjectIdRef;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.lib.RefDatabase;
+import org.eclipse.jgit.lib.RefUpdate;
+import org.eclipse.jgit.lib.Repository;
+import org.eclipse.jgit.storage.file.FileBasedConfig;
+import org.eclipse.jgit.transport.FetchConnection;
+import org.eclipse.jgit.transport.PushConnection;
+import org.eclipse.jgit.transport.PushResult;
+import org.eclipse.jgit.transport.RefSpec;
+import org.eclipse.jgit.transport.RemoteConfig;
+import org.eclipse.jgit.transport.RemoteRefUpdate;
+import org.eclipse.jgit.transport.Transport;
+import org.eclipse.jgit.transport.URIish;
+import org.eclipse.jgit.util.FS;
+import org.junit.Before;
+import org.junit.Test;
+
+public class PushOneTest {
+  private static final int TEST_PUSH_TIMEOUT_SECS = 10;
+
+  private GitRepositoryManager gitRepositoryManagerMock;
+  private Repository repositoryMock;
+  private PermissionBackend permissionBackendMock;
+  private PermissionBackend.WithUser withUserMock;
+  private PermissionBackend.ForProject forProjectMock;
+
+  private Destination destinationMock;
+  private RemoteConfig remoteConfigMock;
+  private RefSpec refSpecMock;
+  private CredentialsFactory credentialsFactory;
+  private PerThreadRequestScope.Scoper threadRequestScoperMock;
+  private ReplicationQueue replicationQueueMock;
+  private IdGenerator idGeneratorMock;
+  private ReplicationStateListeners replicationStateListenersMock;
+  private ReplicationMetrics replicationMetricsMock;
+  private Timer1.Context timerContextMock;
+  private ProjectCache projectCacheMock;
+  private TransportFactory transportFactoryMock;
+  private Transport transportMock;
+  private FetchConnection fetchConnection;
+  private PushConnection pushConnection;
+  private ProjectState projectStateMock;
+  private RefUpdate refUpdateMock;
+  private CreateProjectTask.Factory createProjectTaskFactoryMock;
+  private ReplicationConfig replicationConfigMock;
+  private RefDatabase refDatabaseMock;
+
+  private Project.NameKey projectNameKey;
+  private URIish urish;
+  private List<Ref> localRefs;
+
+  private Map<String, Ref> remoteRefs;
+  private CountDownLatch isCallFinished;
+  private Ref newLocalRef;
+
+  @Before
+  public void setup() throws Exception {
+    projectNameKey = new Project.NameKey("fooProject");
+    urish = new URIish("http://foo.com/fooProject.git");
+
+    newLocalRef =
+        new ObjectIdRef.Unpeeled(
+            NEW, "foo", ObjectId.fromString("0000000000000000000000000000000000000001"));
+
+    localRefs = Arrays.asList(newLocalRef);
+
+    Ref remoteRef = new ObjectIdRef.Unpeeled(NEW, "foo", ObjectId.zeroId());
+    remoteRefs = new HashMap<>();
+    remoteRefs.put("fooProject", remoteRef);
+
+    isCallFinished = new CountDownLatch(1);
+
+    setupMocks();
+  }
+
+  private void setupMocks() throws Exception {
+    FileBasedConfig config = new FileBasedConfig(new Config(), new File("/foo"), FS.DETECTED);
+    config.setString("remote", "Replication", "push", "foo");
+
+    setupRefUpdateMock();
+    setupRepositoryMock(config);
+    setupGitRepoManagerMock();
+
+    projectStateMock = createNiceMock(ProjectState.class);
+    forProjectMock = createNiceMock(ForProject.class);
+    setupWithUserMock();
+    setupPermissionBackedMock();
+
+    setupDestinationMock();
+
+    setupRefSpecMock();
+    setupRemoteConfigMock();
+
+    credentialsFactory = createNiceMock(CredentialsFactory.class);
+
+    setupFetchConnectionMock();
+    setupPushConnectionMock();
+    setupRequestScopeMock();
+    replicationQueueMock = createNiceMock(ReplicationQueue.class);
+    idGeneratorMock = createNiceMock(IdGenerator.class);
+    replicationStateListenersMock = createNiceMock(ReplicationStateListeners.class);
+
+    timerContextMock = createNiceMock(Timer1.Context.class);
+    setupReplicationMetricsMock();
+
+    setupTransportMock();
+
+    setupProjectCacheMock();
+
+    replicationConfigMock = createNiceMock(ReplicationConfig.class);
+
+    replay(
+        gitRepositoryManagerMock,
+        refUpdateMock,
+        repositoryMock,
+        permissionBackendMock,
+        destinationMock,
+        remoteConfigMock,
+        credentialsFactory,
+        threadRequestScoperMock,
+        replicationQueueMock,
+        idGeneratorMock,
+        replicationStateListenersMock,
+        replicationMetricsMock,
+        projectCacheMock,
+        timerContextMock,
+        transportFactoryMock,
+        projectStateMock,
+        withUserMock,
+        forProjectMock,
+        fetchConnection,
+        pushConnection,
+        refSpecMock,
+        refDatabaseMock,
+        replicationConfigMock);
+  }
+
+  @Test
+  public void shouldPushAllRefsWhenNoFilters() throws InterruptedException, IOException {
+    shouldPushAllRefsWithDynamicItemFilter(DynamicItem.itemOf(ReplicationPushFilter.class, null));
+  }
+
+  @Test
+  public void shouldPushAllRefsWhenNoFiltersSetup() throws InterruptedException, IOException {
+    shouldPushAllRefsWithDynamicItemFilter(null);
+  }
+
+  private void shouldPushAllRefsWithDynamicItemFilter(
+      DynamicItem<ReplicationPushFilter> replicationPushFilter)
+      throws IOException, NotSupportedException, TransportException, InterruptedException {
+    List<RemoteRefUpdate> expectedUpdates =
+        Arrays.asList(
+            new RemoteRefUpdate(
+                repositoryMock,
+                newLocalRef.getName(),
+                newLocalRef.getObjectId(),
+                "fooProject",
+                false,
+                "fooProject",
+                null));
+
+    PushResult pushResult = new PushResult();
+
+    expect(transportMock.push(anyObject(), eqRemoteRef(expectedUpdates)))
+        .andReturn(pushResult)
+        .once();
+    replay(transportMock);
+
+    PushOne pushOne = createPushOne(replicationPushFilter);
+
+    pushOne.addRef(PushOne.ALL_REFS);
+    pushOne.run();
+
+    isCallFinished.await(TEST_PUSH_TIMEOUT_SECS, TimeUnit.SECONDS);
+
+    verify(transportMock);
+  }
+
+  @Test
+  public void shouldBlockReplicationUsingPushFilter() throws InterruptedException, IOException {
+    DynamicItem<ReplicationPushFilter> replicationPushFilter =
+        DynamicItem.itemOf(
+            ReplicationPushFilter.class,
+            new ReplicationPushFilter() {
+
+              @Override
+              public List<RemoteRefUpdate> filter(
+                  String projectName, List<RemoteRefUpdate> remoteUpdatesList) {
+                return Collections.emptyList();
+              }
+            });
+
+    // easymock way to check if method was never called
+    expect(transportMock.push(anyObject(), anyObject()))
+        .andThrow(new AssertionFailedError())
+        .anyTimes();
+    replay(transportMock);
+
+    PushOne pushOne = createPushOne(replicationPushFilter);
+
+    pushOne.addRef(PushOne.ALL_REFS);
+    pushOne.run();
+
+    isCallFinished.await(10, TimeUnit.SECONDS);
+
+    verify(transportMock);
+  }
+
+  private PushOne createPushOne(DynamicItem<ReplicationPushFilter> replicationPushFilter) {
+    PushOne push =
+        new PushOne(
+            gitRepositoryManagerMock,
+            permissionBackendMock,
+            destinationMock,
+            remoteConfigMock,
+            replicationConfigMock,
+            credentialsFactory,
+            threadRequestScoperMock,
+            idGeneratorMock,
+            replicationStateListenersMock,
+            replicationMetricsMock,
+            projectCacheMock,
+            createProjectTaskFactoryMock,
+            transportFactoryMock,
+            projectNameKey,
+            urish);
+
+    push.setReplicationPushFilter(replicationPushFilter);
+    return push;
+  }
+
+  private void setupProjectCacheMock() throws IOException {
+    projectCacheMock = createNiceMock(ProjectCache.class);
+    expect(projectCacheMock.checkedGet(projectNameKey)).andReturn(projectStateMock);
+  }
+
+  private void setupTransportMock() throws NotSupportedException, TransportException {
+    transportMock = createNiceMock(Transport.class);
+    expect(transportMock.openFetch()).andReturn(fetchConnection);
+    transportFactoryMock = createNiceMock(TransportFactory.class);
+    expect(transportFactoryMock.open(repositoryMock, urish)).andReturn(transportMock).anyTimes();
+  }
+
+  private void setupReplicationMetricsMock() {
+    replicationMetricsMock = createNiceMock(ReplicationMetrics.class);
+    expect(replicationMetricsMock.start(anyObject())).andReturn(timerContextMock);
+  }
+
+  private void setupRequestScopeMock() {
+    threadRequestScoperMock = createNiceMock(PerThreadRequestScope.Scoper.class);
+    expect(threadRequestScoperMock.scope(anyObject()))
+        .andAnswer(
+            new IAnswer<Callable<Object>>() {
+              @SuppressWarnings("unchecked")
+              @Override
+              public Callable<Object> answer() throws Throwable {
+                Callable<Object> originalCall = (Callable<Object>) getCurrentArguments()[0];
+                return new Callable<Object>() {
+
+                  @Override
+                  public Object call() throws Exception {
+                    Object result = originalCall.call();
+                    isCallFinished.countDown();
+                    return result;
+                  }
+                };
+              }
+            })
+        .anyTimes();
+  }
+
+  private void setupPushConnectionMock() {
+    pushConnection = createNiceMock(PushConnection.class);
+    expect(pushConnection.getRefsMap()).andReturn(remoteRefs);
+  }
+
+  private void setupFetchConnectionMock() {
+    fetchConnection = createNiceMock(FetchConnection.class);
+    expect(fetchConnection.getRefsMap()).andReturn(remoteRefs);
+  }
+
+  private void setupRemoteConfigMock() {
+    remoteConfigMock = createNiceMock(RemoteConfig.class);
+    expect(remoteConfigMock.getPushRefSpecs()).andReturn(ImmutableList.of(refSpecMock));
+  }
+
+  private void setupRefSpecMock() {
+    refSpecMock = createNiceMock(RefSpec.class);
+    expect(refSpecMock.matchSource(anyObject(String.class))).andReturn(true);
+    expect(refSpecMock.expandFromSource(anyObject(String.class))).andReturn(refSpecMock);
+    expect(refSpecMock.getDestination()).andReturn("fooProject").anyTimes();
+    expect(refSpecMock.isForceUpdate()).andReturn(false).anyTimes();
+  }
+
+  private void setupDestinationMock() {
+    destinationMock = createNiceMock(Destination.class);
+    expect(destinationMock.requestRunway(anyObject())).andReturn(RunwayStatus.allowed());
+  }
+
+  private void setupPermissionBackedMock() {
+    permissionBackendMock = createNiceMock(PermissionBackend.class);
+    expect(permissionBackendMock.currentUser()).andReturn(withUserMock);
+  }
+
+  private void setupWithUserMock() {
+    withUserMock = createNiceMock(WithUser.class);
+    expect(withUserMock.project(projectNameKey)).andReturn(forProjectMock);
+  }
+
+  private void setupGitRepoManagerMock() throws IOException {
+    gitRepositoryManagerMock = createNiceMock(GitRepositoryManager.class);
+    expect(gitRepositoryManagerMock.openRepository(projectNameKey)).andReturn(repositoryMock);
+  }
+
+  private void setupRepositoryMock(FileBasedConfig config) throws IOException {
+    repositoryMock = createNiceMock(Repository.class);
+    refDatabaseMock = createNiceMock(RefDatabase.class);
+    expect(repositoryMock.getConfig()).andReturn(config).anyTimes();
+    expect(repositoryMock.getRefDatabase()).andReturn(refDatabaseMock);
+    expect(refDatabaseMock.getRefs()).andReturn(localRefs);
+    expect(repositoryMock.updateRef("fooProject")).andReturn(refUpdateMock);
+  }
+
+  private void setupRefUpdateMock() {
+    refUpdateMock = createNiceMock(RefUpdate.class);
+    expect(refUpdateMock.getOldObjectId())
+        .andReturn(ObjectId.fromString("0000000000000000000000000000000000000001"))
+        .anyTimes();
+  }
+}
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/RemoteRefUpdateCollectionMatcher.java b/src/test/java/com/googlesource/gerrit/plugins/replication/RemoteRefUpdateCollectionMatcher.java
new file mode 100644
index 0000000..111a792
--- /dev/null
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/RemoteRefUpdateCollectionMatcher.java
@@ -0,0 +1,64 @@
+// Copyright (C) 2019 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;
+
+import java.util.Collection;
+import java.util.Objects;
+import org.easymock.EasyMock;
+import org.easymock.IArgumentMatcher;
+import org.eclipse.jgit.transport.RemoteRefUpdate;
+
+public class RemoteRefUpdateCollectionMatcher implements IArgumentMatcher {
+  Collection<RemoteRefUpdate> expectedRemoteRefs;
+
+  public static Collection<RemoteRefUpdate> eqRemoteRef(
+      Collection<RemoteRefUpdate> expectedRemoteRefs) {
+    EasyMock.reportMatcher(new RemoteRefUpdateCollectionMatcher(expectedRemoteRefs));
+    return null;
+  }
+
+  public RemoteRefUpdateCollectionMatcher(Collection<RemoteRefUpdate> expectedRemoteRefs) {
+    this.expectedRemoteRefs = expectedRemoteRefs;
+  }
+
+  @Override
+  public boolean matches(Object argument) {
+    if (!(argument instanceof Collection)) return false;
+
+    @SuppressWarnings("unchecked")
+    Collection<RemoteRefUpdate> refs = (Collection<RemoteRefUpdate>) argument;
+
+    if (expectedRemoteRefs.size() != refs.size()) return false;
+    return refs.stream()
+        .allMatch(
+            ref -> expectedRemoteRefs.stream().anyMatch(expectedRef -> compare(ref, expectedRef)));
+  }
+
+  @Override
+  public void appendTo(StringBuffer buffer) {
+    buffer.append("expected:" + expectedRemoteRefs.toString());
+  }
+
+  private boolean compare(RemoteRefUpdate ref, RemoteRefUpdate expectedRef) {
+    return Objects.equals(ref.getRemoteName(), expectedRef.getRemoteName())
+        && Objects.equals(ref.getStatus(), expectedRef.getStatus())
+        && Objects.equals(ref.getExpectedOldObjectId(), expectedRef.getExpectedOldObjectId())
+        && Objects.equals(ref.getNewObjectId(), expectedRef.getNewObjectId())
+        && Objects.equals(ref.isFastForward(), expectedRef.isFastForward())
+        && Objects.equals(ref.getSrcRef(), expectedRef.getSrcRef())
+        && Objects.equals(ref.isForceUpdate(), expectedRef.isForceUpdate())
+        && Objects.equals(ref.getMessage(), expectedRef.getMessage());
+  }
+}