Merge branch 'stable-3.6' into stable-3.7

* stable-3.6:
  Remove unused Java class InitProjectException
  Invoke the service method when using ProjectInitializationAction
  Add missing local JGit fetch test assertion
  Fix typo in test names
  Do not rely on async/wait for synchronous fetch replication
  Make sure that the EventListener receives replication events
  Add mirror replication option for JGit client
  Remove unnecessary checked exception
  Cover the replication failure scenario and fix the metrics
  Remove white-box unit tests on synchronous FetchCommand
  Throw Exception from tests

Change-Id: I91fd3967d0c17eb5645b0531bb6986478ad3b8e2
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/DeleteProjectTask.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/DeleteProjectTask.java
index 4818ec7..eb890bd 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/DeleteProjectTask.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/DeleteProjectTask.java
@@ -18,7 +18,11 @@
 
 import com.google.common.flogger.FluentLogger;
 import com.google.gerrit.entities.Project;
+import com.google.gerrit.entities.RefNames;
+import com.google.gerrit.extensions.registration.DynamicItem;
+import com.google.gerrit.server.events.EventDispatcher;
 import com.google.gerrit.server.ioutil.HexFormat;
+import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.google.gerrit.server.util.IdGenerator;
 import com.google.inject.Inject;
 import com.google.inject.assistedinject.Assisted;
@@ -26,6 +30,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.client.HttpResult;
 import java.io.IOException;
 import java.net.URISyntaxException;
+import org.eclipse.jgit.lib.RefUpdate;
 import org.eclipse.jgit.transport.URIish;
 
 public class DeleteProjectTask implements Runnable, Completable {
@@ -40,17 +45,20 @@
   private final String uri;
   private final Project.NameKey project;
   private final FetchApiClient.Factory fetchClientFactory;
+  private final DynamicItem<EventDispatcher> eventDispatcher;
   private boolean succeeded;
 
   @Inject
   DeleteProjectTask(
       FetchApiClient.Factory fetchClientFactory,
       IdGenerator ig,
+      DynamicItem<EventDispatcher> eventDispatcher,
       @Assisted Source source,
       @Assisted String uri,
       @Assisted Project.NameKey project) {
     this.fetchClientFactory = fetchClientFactory;
     this.id = ig.next();
+    this.eventDispatcher = eventDispatcher;
     this.uri = uri;
     this.source = source;
     this.project = project;
@@ -71,6 +79,30 @@
           String.format("Cannot delete project %s on remote site %s.", project, uri);
       logger.atWarning().withCause(e).log("%s", errorMessage);
       repLog.warn(errorMessage);
+    } finally {
+      fireEvent();
+    }
+  }
+
+  private void fireEvent() {
+    try {
+      Context.setLocalEvent(true);
+      eventDispatcher
+          .get()
+          .postEvent(
+              new FetchRefReplicatedEvent(
+                  project.get(),
+                  RefNames.REFS_CONFIG,
+                  source.getURI(project),
+                  succeeded
+                      ? ReplicationState.RefFetchResult.SUCCEEDED
+                      : ReplicationState.RefFetchResult.FAILED,
+                  RefUpdate.Result.FORCED));
+    } catch (PermissionBackendException e) {
+      logger.atSevere().withCause(e).log(
+          "Cannot post event for refs/meta/config on project %s", project.get());
+    } finally {
+      Context.unsetLocalEvent();
     }
   }
 
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchAll.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchAll.java
index 42310ff..5a7362a 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchAll.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchAll.java
@@ -30,11 +30,7 @@
   private final ReplicationStateListener stateLog;
 
   public interface Factory {
-    FetchAll create(
-        String urlMatch,
-        ReplicationFilter filter,
-        ReplicationState state,
-        ReplicationType replicationType);
+    FetchAll create(String urlMatch, ReplicationFilter filter, ReplicationState state, boolean now);
   }
 
   private final WorkQueue workQueue;
@@ -42,8 +38,8 @@
   private final String urlMatch;
   private final ReplicationFilter filter;
   private final ReplicationState state;
-  private final ReplicationType replicationType;
   private final SourcesCollection sources;
+  private final boolean now;
 
   @Inject
   protected FetchAll(
@@ -54,7 +50,7 @@
       @Assisted @Nullable String urlMatch,
       @Assisted ReplicationFilter filter,
       @Assisted ReplicationState state,
-      @Assisted ReplicationType replicationType) {
+      @Assisted boolean now) {
     this.workQueue = wq;
     this.projectCache = projectCache;
     this.stateLog = stateLog;
@@ -62,7 +58,7 @@
     this.urlMatch = urlMatch;
     this.filter = filter;
     this.state = state;
-    this.replicationType = replicationType;
+    this.now = now;
   }
 
   Future<?> schedule(long delay, TimeUnit unit) {
@@ -74,7 +70,7 @@
     try {
       for (Project.NameKey nameKey : projectCache.all()) {
         if (filter.matches(nameKey)) {
-          scheduleFullSync(nameKey, urlMatch, state, replicationType);
+          scheduleFullSync(nameKey, urlMatch, state);
         }
       }
     } catch (Exception e) {
@@ -83,16 +79,16 @@
     state.markAllFetchTasksScheduled();
   }
 
-  private void scheduleFullSync(
-      Project.NameKey project,
-      String urlMatch,
-      ReplicationState state,
-      ReplicationType replicationType) {
+  private void scheduleFullSync(Project.NameKey project, String urlMatch, ReplicationState state) {
 
     for (Source cfg : sources.getAll()) {
       if (cfg.wouldFetchProject(project)) {
         for (URIish uri : cfg.getURIs(project, urlMatch)) {
-          cfg.schedule(project, FetchOne.ALL_REFS, uri, state, replicationType, Optional.empty());
+          if (now) {
+            cfg.scheduleNow(project, FetchOne.ALL_REFS, uri, state, Optional.empty());
+          } else {
+            cfg.schedule(project, FetchOne.ALL_REFS, uri, state, Optional.empty());
+          }
         }
       }
     }
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 a6b5ab7..d02cb02 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
@@ -294,22 +294,28 @@
     }
   }
 
+  public void runSync() {
+    try (TraceContext ctx = TraceContext.open().addTag(ID_KEY, HexFormat.fromInt(id))) {
+      doRunFetchOperation(ReplicationType.SYNC);
+    }
+  }
+
   public Set<TransportException> getFetchFailures() {
     return fetchFailures;
   }
 
   private void runFetchOperation() {
     try (TraceContext ctx = TraceContext.open().addTag(ID_KEY, HexFormat.fromInt(id))) {
-      doRunFetchOperation();
+      doRunFetchOperation(ReplicationType.ASYNC);
     }
   }
 
-  private void doRunFetchOperation() {
+  private void doRunFetchOperation(ReplicationType replicationType) {
     // Lock the queue, and remove ourselves, so we can't be modified once
     // we start replication (instead a new instance, with the same URI, is
     // created and scheduled for a future point in time.)
     //
-    if (!pool.requestRunway(this)) {
+    if (replicationType == ReplicationType.ASYNC && !pool.requestRunway(this)) {
       if (!canceled) {
         repLog.info(
             "[{}] Rescheduling replication from {} to avoid collision with an in-flight fetch task [{}].",
@@ -322,8 +328,9 @@
     }
 
     repLog.info(
-        "[{}] Replication from {} started for refs [{}] ...",
+        "[{}] {} replication from {} started for refs [{}] ...",
         taskIdHex,
+        replicationType,
         uri,
         String.join(",", getRefs()));
     Timer1.Context<String> context = metrics.start(config.getName());
@@ -335,8 +342,9 @@
 
       if (fetchRefSpecs.isEmpty()) {
         repLog.info(
-            "[{}] Replication from {} finished but no refs were replicated, {}ms delay, {} retries",
+            "[{}] {} replication from {} finished but no refs were replicated, {}ms delay, {} retries",
             taskIdHex,
+            replicationType,
             uri,
             delay,
             retryCount);
@@ -368,7 +376,7 @@
 
     } catch (NoRemoteRepositoryException | RemoteRepositoryException e) {
       // Tried to replicate to a remote via anonymous git:// but the repository
-      // does not exist.  In this case NoRemoteRepositoryException is not
+      // does not exist. In this case NoRemoteRepositoryException is not
       // raised.
       String msg = e.getMessage();
       repLog.error(
@@ -379,11 +387,11 @@
       repLog.error(
           String.format("Terminal failure. Cannot replicate [%s] from %s", taskIdHex, uri), e);
     } catch (TransportException e) {
-      if (e instanceof LockFailureException) {
+      repLog.error("[{}] Cannot replicate from {}: {}", taskIdHex, uri, e.getMessage());
+      if (replicationType == ReplicationType.ASYNC && e instanceof LockFailureException) {
         lockRetryCount++;
         // The LockFailureException message contains both URI and reason
         // for this failure.
-        repLog.error("[{}] Cannot replicate from {}: {}", taskIdHex, uri, e.getMessage());
 
         // The remote fetch operation should be retried.
         if (lockRetryCount <= maxLockRetries) {
@@ -401,11 +409,10 @@
               taskIdHex,
               uri);
         }
-      } else {
+      } else if (replicationType == ReplicationType.ASYNC) {
         if (canceledWhileRunning.get()) {
           logCanceledWhileRunningException(e);
         } else {
-          repLog.error("Cannot replicate [{}] from {}", taskIdHex, uri, e);
           // The remote fetch operation should be retried.
           pool.reschedule(this, Source.RetryReason.TRANSPORT_ERROR);
         }
@@ -421,7 +428,10 @@
       if (git != null) {
         git.close();
       }
-      pool.notifyFinished(this);
+
+      if (replicationType == ReplicationType.ASYNC) {
+        pool.notifyFinished(this);
+      }
     }
   }
 
@@ -450,11 +460,14 @@
       }
 
       runImpl();
+    } catch (IOException e) {
+      notifyRefReplicatedIOException();
+      throw e;
     }
     return fetchRefSpecs;
   }
 
-  private List<RefSpec> getFetchRefSpecs() {
+  public List<RefSpec> getFetchRefSpecs() {
     List<RefSpec> configRefSpecs = config.getFetchRefSpecs();
     if (delta.isEmpty()) {
       return configRefSpecs;
@@ -571,6 +584,19 @@
     }
   }
 
+  private void notifyRefReplicatedIOException() {
+    for (Map.Entry<String, ReplicationState> entry : stateMap.entries()) {
+      entry
+          .getValue()
+          .notifyRefReplicated(
+              projectName.get(),
+              entry.getKey(),
+              uri,
+              ReplicationState.RefFetchResult.FAILED,
+              RefUpdate.Result.IO_FAILURE);
+    }
+  }
+
   public static class LockFailureException extends TransportException {
     private static final long serialVersionUID = 1L;
 
@@ -585,6 +611,6 @@
 
   @Override
   public boolean hasSucceeded() {
-    return succeeded;
+    return succeeded || getFetchRefSpecs().isEmpty();
   }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/OnStartStop.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/OnStartStop.java
index 8dacc9a..6457f8a 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/OnStartStop.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/OnStartStop.java
@@ -14,8 +14,6 @@
 
 package com.googlesource.gerrit.plugins.replication.pull;
 
-import static com.googlesource.gerrit.plugins.replication.pull.ReplicationType.ASYNC;
-
 import com.google.common.util.concurrent.Atomics;
 import com.google.gerrit.extensions.events.LifecycleListener;
 import com.google.gerrit.extensions.registration.DynamicItem;
@@ -72,7 +70,7 @@
               new FetchResultProcessing.GitUpdateProcessing(eventDispatcher.get()));
       fetchAllFuture.set(
           fetchAll
-              .create(null, ReplicationFilter.all(), state, ASYNC)
+              .create(null, ReplicationFilter.all(), state, false)
               .schedule(30, TimeUnit.SECONDS));
     }
 
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationStateLogger.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationStateLogger.java
index a62f369..6fa96b4 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationStateLogger.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationStateLogger.java
@@ -47,6 +47,10 @@
   }
 
   private void stateWriteErr(String msg, ReplicationState[] states) {
+    if (states == null) {
+      return;
+    }
+
     for (ReplicationState rs : states) {
       if (rs != null) {
         rs.writeStdErr(msg);
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueueMetrics.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueueMetrics.java
index 741927f..b0fa7e9 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueueMetrics.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueueMetrics.java
@@ -66,8 +66,11 @@
       incrementTaskStarted(source);
       runnable.run();
       if (runnable instanceof Completable) {
-        if (((Completable) runnable).hasSucceeded()) {
+        Completable completedRunnable = (Completable) runnable;
+        if (completedRunnable.hasSucceeded()) {
           incrementTaskCompleted(source);
+        } else {
+          incrementTaskFailed(source);
         }
       }
     }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/Source.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/Source.java
index 5ab3859..012a046 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/Source.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/Source.java
@@ -15,7 +15,6 @@
 package com.googlesource.gerrit.plugins.replication.pull;
 
 import static com.googlesource.gerrit.plugins.replication.ReplicationFileBasedConfig.replaceName;
-import static com.googlesource.gerrit.plugins.replication.pull.ReplicationType.SYNC;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
@@ -444,10 +443,9 @@
       Project.NameKey project,
       String ref,
       ReplicationState state,
-      ReplicationType replicationType,
       Optional<PullReplicationApiRequestMetrics> apiRequestMetrics) {
     URIish uri = getURI(project);
-    return schedule(project, ref, uri, state, replicationType, apiRequestMetrics);
+    return schedule(project, ref, uri, state, apiRequestMetrics, false);
   }
 
   public Future<?> schedule(
@@ -455,8 +453,26 @@
       String ref,
       URIish uri,
       ReplicationState state,
-      ReplicationType replicationType,
       Optional<PullReplicationApiRequestMetrics> apiRequestMetrics) {
+    return schedule(project, ref, uri, state, apiRequestMetrics, false);
+  }
+
+  public Future<?> scheduleNow(
+      Project.NameKey project,
+      String ref,
+      URIish uri,
+      ReplicationState state,
+      Optional<PullReplicationApiRequestMetrics> apiRequestMetrics) {
+    return schedule(project, ref, uri, state, apiRequestMetrics, true);
+  }
+
+  private Future<?> schedule(
+      Project.NameKey project,
+      String ref,
+      URIish uri,
+      ReplicationState state,
+      Optional<PullReplicationApiRequestMetrics> apiRequestMetrics,
+      boolean now) {
 
     repLog.info("scheduling replication {}:{} => {}", uri, ref, project);
     if (!shouldReplicate(project, ref, state)) {
@@ -503,7 +519,7 @@
         f =
             pool.schedule(
                 queueMetrics.runWithMetrics(this, e),
-                isSyncCall(replicationType) ? 0 : config.getDelay(),
+                now ? 0 : config.getDelay(),
                 TimeUnit.SECONDS);
         queueMetrics.incrementTaskScheduled(this);
       } else if (!e.getRefs().contains(ref)) {
@@ -519,6 +535,25 @@
     }
   }
 
+  public Optional<FetchOne> fetchSync(
+      Project.NameKey project,
+      String ref,
+      URIish uri,
+      ReplicationState state,
+      Optional<PullReplicationApiRequestMetrics> apiRequestMetrics) {
+    if (shouldReplicate(project, ref)
+        && (config.replicatePermissions() || !ref.equals(RefNames.REFS_CONFIG))) {
+
+      FetchOne e = opFactory.create(project, uri, apiRequestMetrics);
+      e.addRef(ref);
+      e.addState(ref, state);
+      e.runSync();
+      return Optional.of(e);
+    }
+
+    return Optional.empty();
+  }
+
   void scheduleDeleteProject(String uri, Project.NameKey project) {
     @SuppressWarnings("unused")
     ScheduledFuture<?> ignored =
@@ -542,10 +577,6 @@
     postReplicationScheduledEvent(e, ref);
   }
 
-  private boolean isSyncCall(ReplicationType replicationType) {
-    return SYNC.equals(replicationType);
-  }
-
   /**
    * It schedules again a FetchOp instance.
    *
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/StartFetchCommand.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/StartFetchCommand.java
index 97f8e9e..fed33d7 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/StartFetchCommand.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/StartFetchCommand.java
@@ -14,8 +14,6 @@
 
 package com.googlesource.gerrit.plugins.replication.pull;
 
-import static com.googlesource.gerrit.plugins.replication.pull.ReplicationType.ASYNC;
-import static com.googlesource.gerrit.plugins.replication.pull.ReplicationType.SYNC;
 
 import com.google.gerrit.extensions.annotations.RequiresCapability;
 import com.google.gerrit.extensions.registration.DynamicItem;
@@ -80,10 +78,7 @@
       projectFilter = new ReplicationFilter(projectPatterns);
     }
 
-    future =
-        fetchFactory
-            .create(urlMatch, projectFilter, state, replicationType(now))
-            .schedule(0, TimeUnit.SECONDS);
+    future = fetchFactory.create(urlMatch, projectFilter, state, now).schedule(0, TimeUnit.SECONDS);
 
     if (wait) {
       if (future != null) {
@@ -111,10 +106,6 @@
     }
   }
 
-  private ReplicationType replicationType(Boolean now) {
-    return now ? SYNC : ASYNC;
-  }
-
   @Override
   public void writeStdOutSync(String message) {
     if (wait) {
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchAction.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchAction.java
index 9817f2c..9e69f8d 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchAction.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchAction.java
@@ -37,6 +37,7 @@
 import java.util.Optional;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeoutException;
+import org.eclipse.jgit.errors.TransportException;
 
 @Singleton
 public class FetchAction implements RestModifyView<ProjectResource, Input> {
@@ -88,7 +89,8 @@
     } catch (InterruptedException
         | ExecutionException
         | IllegalStateException
-        | TimeoutException e) {
+        | TimeoutException
+        | TransportException e) {
       throw RestApiException.wrap(e.getMessage(), e);
     } catch (RemoteConfigurationMissingException e) {
       throw new UnprocessableEntityException(e.getMessage());
@@ -97,7 +99,7 @@
 
   private Response<?> applySync(Project.NameKey project, Input input)
       throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
-          TimeoutException {
+          TimeoutException, TransportException {
     command.fetchSync(project, input.label, input.refName);
     return Response.created(input);
   }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchCommand.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchCommand.java
index 991ef07..5323425 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchCommand.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchCommand.java
@@ -22,6 +22,7 @@
 import com.google.gerrit.server.events.EventDispatcher;
 import com.google.inject.Inject;
 import com.googlesource.gerrit.plugins.replication.pull.Command;
+import com.googlesource.gerrit.plugins.replication.pull.FetchOne;
 import com.googlesource.gerrit.plugins.replication.pull.FetchResultProcessing;
 import com.googlesource.gerrit.plugins.replication.pull.PullReplicationStateLogger;
 import com.googlesource.gerrit.plugins.replication.pull.ReplicationState;
@@ -29,11 +30,15 @@
 import com.googlesource.gerrit.plugins.replication.pull.Source;
 import com.googlesource.gerrit.plugins.replication.pull.SourcesCollection;
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RemoteConfigurationMissingException;
+import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import org.eclipse.jgit.errors.TransportException;
+import org.eclipse.jgit.transport.RefSpec;
 
 public class FetchCommand implements Command {
 
@@ -60,13 +65,13 @@
       String refName,
       PullReplicationApiRequestMetrics apiRequestMetrics)
       throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
-          TimeoutException {
+          TimeoutException, TransportException {
     fetch(name, label, refName, ASYNC, Optional.of(apiRequestMetrics));
   }
 
   public void fetchSync(Project.NameKey name, String label, String refName)
       throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
-          TimeoutException {
+          TimeoutException, TransportException {
     fetch(name, label, refName, SYNC, Optional.empty());
   }
 
@@ -77,7 +82,7 @@
       ReplicationType fetchType,
       Optional<PullReplicationApiRequestMetrics> apiRequestMetrics)
       throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
-          TimeoutException {
+          TimeoutException, TransportException {
     ReplicationState state =
         fetchReplicationStateFactory.create(
             new FetchResultProcessing.CommandProcessing(this, eventDispatcher.get()));
@@ -89,13 +94,27 @@
     }
 
     try {
-      state.markAllFetchTasksScheduled();
-      Future<?> future = source.get().schedule(name, refName, state, fetchType, apiRequestMetrics);
-      int timeout = source.get().getTimeout();
-      if (timeout == 0) {
-        future.get();
+      if (fetchType == ReplicationType.ASYNC) {
+        state.markAllFetchTasksScheduled();
+        Future<?> future = source.get().schedule(name, refName, state, apiRequestMetrics);
+        int timeout = source.get().getTimeout();
+        if (timeout == 0) {
+          future.get();
+        } else {
+          future.get(timeout, TimeUnit.SECONDS);
+        }
       } else {
-        future.get(timeout, TimeUnit.SECONDS);
+        Optional<FetchOne> maybeFetch =
+            source
+                .get()
+                .fetchSync(name, refName, source.get().getURI(name), state, apiRequestMetrics);
+        if (maybeFetch.map(FetchOne::getFetchRefSpecs).filter(List::isEmpty).isPresent()) {
+          fetchStateLog.warn(
+              String.format(
+                  "[%s] Nothing to fetch, ref-specs is empty", maybeFetch.get().getTaskIdHex()));
+        } else if (maybeFetch.map(fetch -> !fetch.hasSucceeded()).orElse(false)) {
+          throw newTransportException(maybeFetch.get());
+        }
       }
     } catch (ExecutionException
         | IllegalStateException
@@ -106,13 +125,27 @@
     }
 
     try {
-      state.waitForReplication(source.get().getTimeout());
+      if (fetchType == ReplicationType.ASYNC) {
+        state.waitForReplication(source.get().getTimeout());
+      }
     } catch (InterruptedException e) {
       writeStdErrSync("We are interrupted while waiting replication to complete");
       throw e;
     }
   }
 
+  private TransportException newTransportException(FetchOne fetchOne) {
+    List<RefSpec> fetchRefSpecs = fetchOne.getFetchRefSpecs();
+    String combinedErrorMessage =
+        fetchOne.getFetchFailures().stream()
+            .map(TransportException::getMessage)
+            .collect(Collectors.joining("\n"));
+    return new TransportException(
+        String.format(
+            "[%s] %s trying to fetch %s",
+            fetchOne.getTaskIdHex(), combinedErrorMessage, fetchRefSpecs));
+  }
+
   @Override
   public void writeStdOutSync(String message) {}
 
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchJob.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchJob.java
index e15dd68..a613c0e 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchJob.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchJob.java
@@ -21,6 +21,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RemoteConfigurationMissingException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeoutException;
+import org.eclipse.jgit.errors.TransportException;
 
 public class FetchJob implements Runnable {
   private static final FluentLogger log = FluentLogger.forEnclosingClass();
@@ -54,7 +55,8 @@
     } catch (InterruptedException
         | ExecutionException
         | RemoteConfigurationMissingException
-        | TimeoutException e) {
+        | TimeoutException
+        | TransportException e) {
       log.atSevere().withCause(e).log(
           "Exception during the async fetch call for project %s, label %s and ref name %s",
           project.get(), input.label, input.refName);
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 40e39ad..6b903d8 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
@@ -16,7 +16,6 @@
 
 import static com.google.gerrit.httpd.restapi.RestApiServlet.SC_UNPROCESSABLE_ENTITY;
 import static com.googlesource.gerrit.plugins.replication.pull.api.HttpServletOps.checkAcceptHeader;
-import static com.googlesource.gerrit.plugins.replication.pull.api.HttpServletOps.setResponse;
 import static javax.servlet.http.HttpServletResponse.SC_BAD_REQUEST;
 import static javax.servlet.http.HttpServletResponse.SC_CONFLICT;
 import static javax.servlet.http.HttpServletResponse.SC_CREATED;
@@ -56,7 +55,6 @@
 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 com.googlesource.gerrit.plugins.replication.pull.api.exception.UnauthorizedAuthException;
 import java.io.BufferedReader;
 import java.io.EOFException;
@@ -169,7 +167,7 @@
     } catch (ResourceConflictException e) {
       RestApiServlet.replyError(
           httpRequest, httpResponse, SC_CONFLICT, e.getMessage(), e.caching(), e);
-    } catch (InitProjectException | ResourceNotFoundException e) {
+    } catch (ResourceNotFoundException e) {
       RestApiServlet.replyError(
           httpRequest, httpResponse, SC_INTERNAL_SERVER_ERROR, e.getMessage(), e.caching(), e);
     } catch (NoSuchElementException e) {
@@ -194,16 +192,8 @@
   }
 
   private void doInitProject(HttpServletRequest httpRequest, HttpServletResponse httpResponse)
-      throws RestApiException, IOException, PermissionBackendException {
-
-    IdString id = getInitProjectName(httpRequest).get();
-    String projectName = id.get();
-    if (projectInitializationAction.initProject(projectName)) {
-      setResponse(
-          httpResponse, HttpServletResponse.SC_CREATED, "Project " + projectName + " initialized");
-      return;
-    }
-    throw new InitProjectException(projectName);
+      throws IOException, ServletException {
+    projectInitializationAction.service(httpRequest, httpResponse);
   }
 
   @SuppressWarnings("unchecked")
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/UpdateHeadAction.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/UpdateHeadAction.java
index 3f6673b..800f2b2 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/UpdateHeadAction.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/UpdateHeadAction.java
@@ -15,8 +15,11 @@
 package com.googlesource.gerrit.plugins.replication.pull.api;
 
 import com.google.common.base.Strings;
+import com.google.common.flogger.FluentLogger;
+import com.google.gerrit.entities.Project;
 import com.google.gerrit.entities.RefNames;
 import com.google.gerrit.extensions.api.projects.HeadInput;
+import com.google.gerrit.extensions.registration.DynamicItem;
 import com.google.gerrit.extensions.restapi.AuthException;
 import com.google.gerrit.extensions.restapi.BadRequestException;
 import com.google.gerrit.extensions.restapi.ResourceConflictException;
@@ -24,28 +27,44 @@
 import com.google.gerrit.extensions.restapi.Response;
 import com.google.gerrit.extensions.restapi.RestModifyView;
 import com.google.gerrit.extensions.restapi.UnprocessableEntityException;
+import com.google.gerrit.server.events.EventDispatcher;
+import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.google.gerrit.server.project.ProjectResource;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import com.googlesource.gerrit.plugins.replication.LocalFS;
+import com.googlesource.gerrit.plugins.replication.pull.Context;
+import com.googlesource.gerrit.plugins.replication.pull.FetchRefReplicatedEvent;
 import com.googlesource.gerrit.plugins.replication.pull.GerritConfigOps;
+import com.googlesource.gerrit.plugins.replication.pull.ReplicationState;
+import java.net.HttpURLConnection;
 import java.util.Optional;
+import org.eclipse.jgit.lib.RefUpdate;
 import org.eclipse.jgit.transport.URIish;
 
 @Singleton
 public class UpdateHeadAction implements RestModifyView<ProjectResource, HeadInput> {
+  private static final FluentLogger logger = FluentLogger.forEnclosingClass();
   private final GerritConfigOps gerritConfigOps;
   private final FetchPreconditions preconditions;
+  private final DynamicItem<EventDispatcher> eventDispatcher;
+  private static final URIish EMPTY_URI = new URIish();
 
   @Inject
-  UpdateHeadAction(GerritConfigOps gerritConfigOps, FetchPreconditions preconditions) {
+  UpdateHeadAction(
+      GerritConfigOps gerritConfigOps,
+      FetchPreconditions preconditions,
+      DynamicItem<EventDispatcher> eventDispatcher) {
     this.gerritConfigOps = gerritConfigOps;
     this.preconditions = preconditions;
+    this.eventDispatcher = eventDispatcher;
   }
 
   @Override
   public Response<?> apply(ProjectResource projectResource, HeadInput input)
       throws AuthException, BadRequestException, ResourceConflictException, Exception {
+    Response<String> res = null;
+
     if (input == null || Strings.isNullOrEmpty(input.ref)) {
       throw new BadRequestException("ref required");
     }
@@ -61,15 +80,44 @@
     Optional<URIish> maybeRepo =
         gerritConfigOps.getGitRepositoryURI(String.format("%s.git", projectResource.getName()));
 
-    if (maybeRepo.isPresent()) {
-      if (new LocalFS(maybeRepo.get()).updateHead(projectResource.getNameKey(), ref)) {
-        return Response.ok(ref);
+    try {
+      if (maybeRepo.isPresent()) {
+        if (new LocalFS(maybeRepo.get()).updateHead(projectResource.getNameKey(), ref)) {
+          return res = Response.ok(ref);
+        }
+        throw new UnprocessableEntityException(
+            String.format(
+                "Could not update HEAD of repo %s to ref %s", projectResource.getName(), ref));
       }
-      throw new UnprocessableEntityException(
-          String.format(
-              "Could not update HEAD of repo %s to ref %s", projectResource.getName(), ref));
+    } finally {
+      fireEvent(
+          projectResource.getNameKey(),
+          res != null && res.statusCode() == HttpURLConnection.HTTP_OK);
     }
     throw new ResourceNotFoundException(
         String.format("Could not compute URL for repo: %s", projectResource.getName()));
   }
+
+  private void fireEvent(Project.NameKey projectName, boolean succeeded) {
+    try {
+      Context.setLocalEvent(true);
+      eventDispatcher
+          .get()
+          .postEvent(
+              new FetchRefReplicatedEvent(
+                  projectName.get(),
+                  RefNames.HEAD,
+                  EMPTY_URI, // TODO: the remote label is not passed as parameter, hence cannot be
+                  // propagated to the event
+                  succeeded
+                      ? ReplicationState.RefFetchResult.SUCCEEDED
+                      : ReplicationState.RefFetchResult.FAILED,
+                  RefUpdate.Result.FORCED));
+    } catch (PermissionBackendException e) {
+      logger.atSevere().withCause(e).log(
+          "Cannot post event for refs/meta/config on project %s", projectName);
+    } finally {
+      Context.unsetLocalEvent();
+    }
+  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/exception/InitProjectException.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/exception/InitProjectException.java
deleted file mode 100644
index 85a7729..0000000
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/exception/InitProjectException.java
+++ /dev/null
@@ -1,25 +0,0 @@
-// Copyright (C) 2021 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.exception;
-
-import com.google.gerrit.extensions.restapi.RestApiException;
-
-public class InitProjectException extends RestApiException {
-  private static final long serialVersionUID = 1L;
-
-  public InitProjectException(String projectName) {
-    super("Cannot create project " + projectName);
-  }
-}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/transport/TransportProvider.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/transport/TransportProvider.java
index c52db82..ec7f655 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/transport/TransportProvider.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/transport/TransportProvider.java
@@ -56,6 +56,7 @@
       throws NotSupportedException, TransportException {
     Transport tn = Transport.open(local, uri);
     tn.applyConfig(remoteConfig);
+    tn.setRemoveDeletedRefs(remoteConfig.isMirror());
     if (tn instanceof TransportHttp && bearerToken.isPresent()) {
       ((TransportHttp) tn)
           .setAdditionalHeaders(ImmutableMap.of(HDR_AUTHORIZATION, "Bearer " + bearerToken.get()));
diff --git a/src/main/resources/Documentation/config.md b/src/main/resources/Documentation/config.md
index 0c3e02c..b7db2c0 100644
--- a/src/main/resources/Documentation/config.md
+++ b/src/main/resources/Documentation/config.md
@@ -523,6 +523,14 @@
 
 	By default, false, do *not* replicate project deletions.
 
+remote.NAME.mirror
+:	If true, replication will remove local branches and tags that are
+absent remotely or invisible to the replication (for example read access
+denied via `authGroup` option). Note that this option is currently
+implemented for the JGit client only.
+
+	By default, false, do not remove remote branches or tags.
+
 remote.NAME.authGroup
 :	Specifies the name of a group that the remote should use to
 	access the repositories. Multiple authGroups may be specified
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/CGitFetchIT.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/CGitFetchIT.java
index d81a253..baacf20 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/CGitFetchIT.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/CGitFetchIT.java
@@ -86,7 +86,7 @@
   }
 
   @Test(expected = TransportException.class)
-  public void shouldThrowExecptionWhenRefDoesNotExists() throws Exception {
+  public void shouldThrowExceptionWhenRefDoesNotExists() throws Exception {
     testRepo = cloneProject(createTestProject(project + TEST_REPLICATION_SUFFIX));
     String nonExistingRef = "refs/changes/02/20000/1:refs/changes/02/20000/1";
     try (Repository repo = repoManager.openRepository(project)) {
@@ -101,7 +101,7 @@
   }
 
   @Test(expected = TransportException.class)
-  public void shouldThrowExecptionWhenSourceDoesNotExists() throws Exception {
+  public void shouldThrowExceptionWhenSourceDoesNotExists() throws Exception {
     testRepo = cloneProject(createTestProject(project + TEST_REPLICATION_SUFFIX));
     try (Repository repo = repoManager.openRepository(project)) {
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchGitUpdateProcessingTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchGitUpdateProcessingTest.java
index 5a26054..45d30db 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchGitUpdateProcessingTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchGitUpdateProcessingTest.java
@@ -20,11 +20,9 @@
 import static org.mockito.Mockito.verify;
 
 import com.google.gerrit.server.events.EventDispatcher;
-import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.googlesource.gerrit.plugins.replication.pull.FetchResultProcessing.CommandProcessing;
 import com.googlesource.gerrit.plugins.replication.pull.FetchResultProcessing.GitUpdateProcessing;
 import com.googlesource.gerrit.plugins.replication.pull.ReplicationState.RefFetchResult;
-import java.net.URISyntaxException;
 import org.eclipse.jgit.lib.RefUpdate;
 import org.eclipse.jgit.transport.URIish;
 import org.junit.Before;
@@ -47,7 +45,7 @@
   }
 
   @Test
-  public void headRefReplicatedInGitUpdateProcessing() throws PermissionBackendException {
+  public void headRefReplicatedInGitUpdateProcessing() throws Exception {
     FetchRefReplicatedEvent expectedEvent =
         new FetchRefReplicatedEvent(
             "someProject",
@@ -66,8 +64,7 @@
   }
 
   @Test
-  public void headRefReplicatedInCommandProcessing()
-      throws URISyntaxException, PermissionBackendException {
+  public void headRefReplicatedInCommandProcessing() throws Exception {
     FetchRefReplicatedEvent expectedEvent =
         new FetchRefReplicatedEvent(
             "someProject",
@@ -86,7 +83,7 @@
   }
 
   @Test
-  public void changeRefReplicated() throws URISyntaxException, PermissionBackendException {
+  public void changeRefReplicated() throws Exception {
     FetchRefReplicatedEvent expectedEvent =
         new FetchRefReplicatedEvent(
             "someProject",
@@ -105,7 +102,7 @@
   }
 
   @Test
-  public void onAllNodesReplicated() throws PermissionBackendException {
+  public void onAllNodesReplicated() throws Exception {
     FetchRefReplicationDoneEvent expectedDoneEvent =
         new FetchRefReplicationDoneEvent("someProject", "refs/heads/master", 5);
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchITBase.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchITBase.java
index 2a11717..be9f902 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchITBase.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchITBase.java
@@ -21,7 +21,6 @@
 import com.google.gerrit.server.config.SitePaths;
 import com.google.inject.Inject;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.FetchFactory;
-import java.io.IOException;
 import java.nio.file.Path;
 import java.time.Duration;
 import java.util.function.Supplier;
@@ -50,11 +49,11 @@
     fetchFactory = plugin.getSysInjector().getInstance(FetchFactory.class);
   }
 
-  void waitUntil(Supplier<Boolean> waitCondition) throws InterruptedException {
+  void waitUntil(Supplier<Boolean> waitCondition) throws Exception {
     WaitUtil.waitUntil(waitCondition, TEST_TIMEOUT);
   }
 
-  Ref getRef(Repository repo, String branchName) throws IOException {
+  Ref getRef(Repository repo, String branchName) throws Exception {
     return repo.getRefDatabase().exactRef(branchName);
   }
 
@@ -67,7 +66,7 @@
     }
   }
 
-  Project.NameKey createTestProject(String name) throws Exception {
+  Project.NameKey createTestProject(String name) {
     return projectOperations.newProject().name(name).create();
   }
 }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchOneTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchOneTest.java
index d5d4546..dd9f98e 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchOneTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchOneTest.java
@@ -29,7 +29,6 @@
 import com.googlesource.gerrit.plugins.replication.pull.fetch.FetchFactory;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.InexistentRefTransportException;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.RefUpdateState;
-import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.stream.Collectors;
@@ -776,7 +775,7 @@
     when(source.requestRunway(any())).thenReturn(allowed);
   }
 
-  private void setupGitRepoManagerMock() throws IOException {
+  private void setupGitRepoManagerMock() throws Exception {
     when(grm.openRepository(PROJECT_NAME)).thenReturn(repository);
   }
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchReplicationTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchReplicationTest.java
index 3257fda..dd1aed8 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchReplicationTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchReplicationTest.java
@@ -19,7 +19,6 @@
 import static com.googlesource.gerrit.plugins.replication.pull.Source.encode;
 import static com.googlesource.gerrit.plugins.replication.pull.Source.needsUrlEncoding;
 
-import java.net.URISyntaxException;
 import org.eclipse.jgit.lib.Config;
 import org.eclipse.jgit.transport.RemoteConfig;
 import org.eclipse.jgit.transport.URIish;
@@ -28,7 +27,7 @@
 public class FetchReplicationTest {
 
   @Test
-  public void testNeedsUrlEncoding() throws URISyntaxException {
+  public void testNeedsUrlEncoding() throws Exception {
     assertThat(needsUrlEncoding(new URIish("http://host/path"))).isTrue();
     assertThat(needsUrlEncoding(new URIish("https://host/path"))).isTrue();
     assertThat(needsUrlEncoding(new URIish("amazon-s3://config/bucket/path"))).isTrue();
@@ -48,7 +47,7 @@
   }
 
   @Test
-  public void testRefsBatchSizeMustBeGreaterThanZero() throws URISyntaxException {
+  public void testRefsBatchSizeMustBeGreaterThanZero() throws Exception {
     Config cf = new Config();
     cf.setInt("remote", "test_config", "timeout", 0);
     cf.setInt("replication", null, "refsBatchSize", 0);
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/JGitFetchIT.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/JGitFetchIT.java
index 76ff02b..77cf80d 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/JGitFetchIT.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/JGitFetchIT.java
@@ -14,11 +14,21 @@
 
 package com.googlesource.gerrit.plugins.replication.pull;
 
+import static com.google.common.truth.Truth.assertThat;
+import static com.google.common.truth.Truth.assertWithMessage;
+import static com.google.gerrit.acceptance.GitUtil.deleteRef;
+import static com.google.gerrit.acceptance.GitUtil.pushHead;
+import static com.google.gerrit.acceptance.testsuite.project.TestProjectUpdate.allow;
+
 import com.google.common.collect.Lists;
+import com.google.gerrit.acceptance.PushOneCommit;
 import com.google.gerrit.acceptance.SkipProjectClone;
 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.extensions.config.FactoryModule;
+import com.google.inject.Inject;
 import com.google.inject.Scopes;
 import com.google.inject.assistedinject.FactoryModuleBuilder;
 import com.googlesource.gerrit.plugins.replication.AutoReloadSecureCredentialsFactoryDecorator;
@@ -30,12 +40,20 @@
 import com.googlesource.gerrit.plugins.replication.pull.fetch.FetchFactory;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.JGitFetch;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.PermanentTransportException;
+import com.googlesource.gerrit.plugins.replication.pull.fetch.RefUpdateState;
+import java.io.IOException;
 import java.net.URISyntaxException;
+import java.util.List;
 import org.eclipse.jgit.lib.Config;
+import org.eclipse.jgit.lib.Constants;
+import org.eclipse.jgit.lib.RefUpdate;
 import org.eclipse.jgit.lib.Repository;
+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.URIish;
+import org.junit.Before;
 import org.junit.Test;
 
 @SkipProjectClone
@@ -46,6 +64,17 @@
 public class JGitFetchIT extends FetchITBase {
   private static final String TEST_REPLICATION_SUFFIX = "suffix1";
   private static final String TEST_TASK_ID = "taskid";
+  private static final RefSpec ALL_REFS = new RefSpec("+refs/*:refs/*");
+
+  @Inject private ProjectOperations projectOperations;
+
+  @Before
+  public void allowRefDeletion() {
+    projectOperations
+        .allProjectsForUpdate()
+        .add(allow(Permission.DELETE).ref("refs/*").group(adminGroupUuid()))
+        .update();
+  }
 
   @Test(expected = PermanentTransportException.class)
   public void shouldThrowPermanentTransportExceptionWhenRefDoesNotExists() throws Exception {
@@ -59,12 +88,68 @@
     }
   }
 
+  @Test
+  public void shouldPruneRefsWhenMirrorIsTrue() throws Exception {
+    testRepo = cloneProject(createTestProject(project + TEST_REPLICATION_SUFFIX));
+    String branchName = "anyBranch";
+    String branchRef = Constants.R_HEADS + branchName;
+    String tagName = "anyTag";
+    String tagRef = Constants.R_TAGS + tagName;
+
+    PushOneCommit.Result branchPush = pushFactory.create(user.newIdent(), testRepo).to(branchRef);
+    branchPush.assertOkStatus();
+
+    PushResult tagPush = pushHead(testRepo, tagRef, false, false);
+    assertOkStatus(tagPush, tagRef);
+
+    try (Repository localRepo = repoManager.openRepository(project)) {
+      List<RefUpdateState> fetchCreated = fetchAllRefs(localRepo);
+      assertThat(fetchCreated.toString())
+          .contains(new RefUpdateState(branchRef, RefUpdate.Result.NEW).toString());
+      assertThat(getRef(localRepo, branchRef)).isNotNull();
+
+      assertThat(fetchCreated.toString())
+          .contains(new RefUpdateState(tagRef, RefUpdate.Result.NEW).toString());
+      assertThat(getRef(localRepo, tagRef)).isNotNull();
+
+      PushResult deleteBranchResult = deleteRef(testRepo, branchRef);
+      assertOkStatus(deleteBranchResult, branchRef);
+
+      PushResult deleteTagResult = deleteRef(testRepo, tagRef);
+      assertOkStatus(deleteTagResult, tagRef);
+
+      List<RefUpdateState> fetchDeleted = fetchAllRefs(localRepo);
+      assertThat(fetchDeleted.toString())
+          .contains(new RefUpdateState(branchRef, RefUpdate.Result.FORCED).toString());
+      assertThat(getRef(localRepo, branchRef)).isNull();
+
+      assertThat(fetchDeleted.toString())
+          .contains(new RefUpdateState(tagRef, RefUpdate.Result.FORCED).toString());
+      assertThat(getRef(localRepo, tagRef)).isNull();
+    }
+  }
+
+  private List<RefUpdateState> fetchAllRefs(Repository localRepo)
+      throws URISyntaxException, IOException {
+    Fetch fetch = fetchFactory.create(TEST_TASK_ID, new URIish(testRepoPath.toString()), localRepo);
+    return fetch.fetch(Lists.newArrayList(ALL_REFS));
+  }
+
+  private static void assertOkStatus(PushResult result, String ref) {
+    RemoteRefUpdate refUpdate = result.getRemoteUpdate(ref);
+    assertThat(refUpdate).isNotNull();
+    assertWithMessage(refUpdate.getMessage())
+        .that(refUpdate.getStatus())
+        .isEqualTo(RemoteRefUpdate.Status.OK);
+  }
+
   @SuppressWarnings("unused")
   private static class TestModule extends FactoryModule {
     @Override
     protected void configure() {
       Config cf = new Config();
       cf.setInt("remote", "test_config", "timeout", 0);
+      cf.setBoolean("remote", "test_config", "mirror", true);
       try {
         RemoteConfig remoteConfig = new RemoteConfig(cf, "test_config");
         SourceConfiguration sourceConfig = new SourceConfiguration(remoteConfig, cf);
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationFanoutConfigIT.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationFanoutConfigIT.java
index 4ede1ae..0721dd2 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationFanoutConfigIT.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationFanoutConfigIT.java
@@ -30,7 +30,6 @@
 import com.google.gerrit.server.config.SitePaths;
 import com.google.inject.Inject;
 import com.googlesource.gerrit.plugins.replication.AutoReloadConfigDecorator;
-import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.time.Duration;
@@ -251,7 +250,7 @@
     waitUntil(() -> sources.getAll().size() == 1);
   }
 
-  private Ref getRef(Repository repo, String branchName) throws IOException {
+  private Ref getRef(Repository repo, String branchName) throws Exception {
     return repo.getRefDatabase().exactRef(branchName);
   }
 
@@ -264,24 +263,24 @@
     }
   }
 
-  private void setReplicationSource(String remoteName) throws IOException {
+  private void setReplicationSource(String remoteName) throws Exception {
     config.setBoolean("gerrit", null, "autoReload", true);
     config.save();
   }
 
-  private void setRemoteConfig(String replicaSuffix, Optional<String> project) throws IOException {
+  private void setRemoteConfig(String replicaSuffix, Optional<String> project) throws Exception {
     setRemoteConfig(remoteConfig, replicaSuffix, project);
   }
 
   private void setRemoteConfig(
       FileBasedConfig remoteConfig, String replicaSuffix, Optional<String> project)
-      throws IOException {
+      throws Exception {
     setRemoteConfig(remoteConfig, Arrays.asList(replicaSuffix), project);
   }
 
   private void setRemoteConfig(
       FileBasedConfig remoteConfig, List<String> replicaSuffixes, Optional<String> project)
-      throws IOException {
+      throws Exception {
     List<String> replicaUrls =
         replicaSuffixes.stream()
             .map(suffix -> gitPath.resolve("${name}" + suffix + ".git").toString())
@@ -296,13 +295,13 @@
   }
 
   private void setReplicationCredentials(String remoteName, String username, String password)
-      throws IOException {
+      throws Exception {
     secureConfig.setString("remote", remoteName, "username", username);
     secureConfig.setString("remote", remoteName, "password", password);
     secureConfig.save();
   }
 
-  private void waitUntil(Supplier<Boolean> waitCondition) throws InterruptedException {
+  private void waitUntil(Supplier<Boolean> waitCondition) throws Exception {
     WaitUtil.waitUntil(waitCondition, TEST_TIMEOUT);
   }
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationITAbstract.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationITAbstract.java
index 58693bc..8160304 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationITAbstract.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationITAbstract.java
@@ -21,6 +21,7 @@
 import static com.google.gerrit.acceptance.testsuite.project.TestProjectUpdate.allow;
 import static com.google.gerrit.server.group.SystemGroupBackend.REGISTERED_USERS;
 
+import com.google.common.base.Strings;
 import com.google.gerrit.acceptance.PushOneCommit.Result;
 import com.google.gerrit.acceptance.UseLocalDisk;
 import com.google.gerrit.acceptance.config.GerritConfig;
@@ -31,35 +32,106 @@
 import com.google.gerrit.extensions.api.projects.BranchInput;
 import com.google.gerrit.extensions.events.HeadUpdatedListener;
 import com.google.gerrit.extensions.events.ProjectDeletedListener;
+import com.google.gerrit.extensions.registration.DynamicSet;
 import com.google.gerrit.extensions.restapi.RestApiException;
 import com.google.gerrit.server.config.SitePaths;
+import com.google.gerrit.server.events.Event;
+import com.google.gerrit.server.events.EventListener;
 import com.google.inject.Inject;
+import com.google.inject.Singleton;
 import com.googlesource.gerrit.plugins.replication.AutoReloadConfigDecorator;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+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.RemoteRefUpdate;
 import org.eclipse.jgit.transport.RemoteRefUpdate.Status;
+import org.eclipse.jgit.util.FS;
 import org.junit.Ignore;
 import org.junit.Test;
 
 /** Base class to run regular and async acceptance tests */
 public abstract class PullReplicationITAbstract extends PullReplicationSetupBase {
+  private BufferedEventListener eventListener;
 
   public static class PullReplicationTestModule extends PullReplicationModule {
     @Inject
     public PullReplicationTestModule(SitePaths site, InMemoryMetricMaker memMetric) {
       super(site, memMetric);
     }
+
+    @Override
+    protected void configure() {
+      super.configure();
+
+      DynamicSet.bind(binder(), EventListener.class)
+          .to(BufferedEventListener.class)
+          .asEagerSingleton();
+    }
+  }
+
+  @Singleton
+  public static class BufferedEventListener implements EventListener {
+
+    private final List<Event> eventsReceived;
+    private String eventTypeFilter;
+
+    @Inject
+    public BufferedEventListener() {
+      eventsReceived = new ArrayList<>();
+    }
+
+    @Override
+    public void onEvent(Event event) {
+      if (event.getType().equals(eventTypeFilter)) {
+        eventsReceived.add(event);
+      }
+    }
+
+    public void clearFilter(String expectedEventType) {
+      eventsReceived.clear();
+      eventTypeFilter = expectedEventType;
+    }
+
+    public int numEventsReceived() {
+      return eventsReceived.size();
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T extends Event> Stream<T> eventsStream() {
+      return (Stream<T>) eventsReceived.stream();
+    }
+  }
+
+  @Override
+  protected void setUpTestPlugin(boolean loadExisting) throws Exception {
+    super.setUpTestPlugin(loadExisting);
+
+    eventListener = plugin.getSysInjector().getInstance(BufferedEventListener.class);
+  }
+
+  protected boolean isAsyncReplication() {
+    FileBasedConfig config =
+        new FileBasedConfig(sitePaths.etc_dir.resolve("replication.config").toFile(), FS.DETECTED);
+    try {
+      config.load();
+    } catch (IOException | ConfigInvalidException e) {
+      throw new IllegalStateException(e);
+    }
+    return !Strings.isNullOrEmpty(config.getString("replication", null, "syncRefs"));
   }
 
   @Override
@@ -100,6 +172,8 @@
             ObjectId.zeroId().getName(),
             sourceCommit.getId().getName(),
             TEST_REPLICATION_REMOTE);
+
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     pullReplicationQueue.onEvent(event);
     waitUntilReplicationCompleted(1);
 
@@ -110,13 +184,18 @@
       assertThat(targetBranchRef).isNotNull();
       assertThat(targetBranchRef.getObjectId()).isEqualTo(sourceCommit.getId());
     }
+
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
   }
 
   private void assertTasksMetricScheduledAndCompleted(int numTasks) {
-    assertTasksMetric("scheduled", numTasks);
-    assertTasksMetric("started", numTasks);
-    assertTasksMetric("completed", numTasks);
-    assertEmptyTasksMetric("failed");
+    if (isAsyncReplication()) {
+      assertTasksMetric("scheduled", numTasks);
+      assertTasksMetric("started", numTasks);
+      assertTasksMetric("completed", numTasks);
+      assertEmptyTasksMetric("failed");
+    }
   }
 
   @Test
@@ -141,6 +220,7 @@
             ObjectId.zeroId().getName(),
             branchRevision,
             TEST_REPLICATION_REMOTE);
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     pullReplicationQueue.onEvent(event);
     waitUntilReplicationCompleted(1);
 
@@ -153,7 +233,39 @@
       assertThat(targetBranchRef.getObjectId().getName()).isEqualTo(branchRevision);
     }
 
-    assertTasksMetricScheduledAndCompleted(1);
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
+  }
+
+  @Test
+  @GerritConfig(name = "gerrit.instanceId", value = TEST_REPLICATION_REMOTE)
+  public void shouldFailReplicatingInexistentRepository() throws Exception {
+    String newBranch = "refs/heads/mybranch";
+    String branchRevision = "7bb81c29e14a4169e5ca4f43992094c209aae26c";
+
+    ReplicationQueue pullReplicationQueue =
+        plugin.getSysInjector().getInstance(ReplicationQueue.class);
+    FakeGitReferenceUpdatedEvent event =
+        new FakeGitReferenceUpdatedEvent(
+            project,
+            newBranch,
+            ObjectId.zeroId().getName(),
+            branchRevision,
+            TEST_REPLICATION_REMOTE);
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
+    pullReplicationQueue.onEvent(event);
+    waitUntilReplicationFailed(1);
+
+    try (Repository repo = repoManager.openRepository(project);
+        Repository sourceRepo = repoManager.openRepository(project)) {
+
+      Ref targetBranchRef = getRef(repo, newBranch);
+      assertThat(targetBranchRef).isNull();
+    }
+
+    assertThatEventListenerHasReceivedNumEvents(2);
+    assertThatRefReplicatedEventsContainsExactlyStatuses(
+        ReplicationState.RefFetchResult.FAILED, ReplicationState.RefFetchResult.FAILED);
   }
 
   @Test
@@ -187,6 +299,7 @@
             ObjectId.zeroId().getName(),
             branchRevision,
             TEST_REPLICATION_REMOTE);
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     pullReplicationQueue.onEvent(event);
     waitUntilReplicationCompleted(1);
 
@@ -197,6 +310,8 @@
       assertThat(targetBranchRef).isNotNull();
       assertThat(targetBranchRef.getObjectId().getName()).isEqualTo(branchRevision);
     }
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
 
     TestRepository<InMemoryRepository> testProject = cloneProject(testProjectNameKey);
     fetch(testProject, RefNames.REFS_HEADS + "*:" + RefNames.REFS_HEADS + "*");
@@ -214,6 +329,7 @@
             branchRevision,
             amendedCommit.getId().getName(),
             TEST_REPLICATION_REMOTE);
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     pullReplicationQueue.onEvent(forcedPushEvent);
     waitUntilReplicationCompleted(2);
 
@@ -229,6 +345,8 @@
     }
 
     assertTasksMetricScheduledAndCompleted(2);
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
   }
 
   @Test
@@ -256,6 +374,7 @@
             ObjectId.zeroId().getName(),
             sourceCommit.getId().getName(),
             TEST_REPLICATION_REMOTE);
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     pullReplicationQueue.onEvent(event);
     waitUntilReplicationCompleted(1);
 
@@ -268,6 +387,8 @@
     }
 
     assertTasksMetricScheduledAndCompleted(1);
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
   }
 
   @Test
@@ -300,6 +421,7 @@
             ObjectId.zeroId().getName(),
             branchRevision,
             TEST_REPLICATION_REMOTE);
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     pullReplicationQueue.onEvent(event);
     waitUntilReplicationCompleted(1);
 
@@ -313,6 +435,8 @@
     }
 
     assertTasksMetricScheduledAndCompleted(1);
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
   }
 
   @Test
@@ -337,6 +461,7 @@
             return NotifyHandling.NONE;
           }
         };
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     for (ProjectDeletedListener l : deletedListeners) {
       l.onProjectDeleted(event);
     }
@@ -345,6 +470,8 @@
     waitUntil(() -> !repoManager.list().contains(project));
 
     assertTasksMetricScheduledAndCompleted(1);
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
   }
 
   @Test
@@ -368,6 +495,7 @@
         plugin.getSysInjector().getInstance(ReplicationQueue.class);
 
     HeadUpdatedListener.Event event = new FakeHeadUpdateEvent(master, newBranch, testProjectName);
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     pullReplicationQueue.onHeadUpdated(event);
     waitUntilReplicationCompleted(1);
 
@@ -381,6 +509,8 @@
         });
 
     assertTasksMetricScheduledAndCompleted(1);
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
   }
 
   @Ignore
@@ -401,6 +531,7 @@
             ObjectId.zeroId().getName(),
             sourceCommit.getId().getName(),
             TEST_REPLICATION_REMOTE);
+    eventListener.clearFilter(FetchRefReplicatedEvent.TYPE);
     pullReplicationQueue.onEvent(event);
     waitUntilReplicationCompleted(1);
 
@@ -411,15 +542,48 @@
       assertThat(targetBranchRef).isNotNull();
       assertThat(targetBranchRef.getObjectId()).isEqualTo(sourceCommit.getId());
     }
+
+    assertThatEventListenerHasReceivedNumEvents(1);
+    assertThatRefReplicatedEventsContainsStatus(ReplicationState.RefFetchResult.SUCCEEDED);
   }
 
-  private void waitUntilReplicationCompleted(int expected) throws InterruptedException {
-    waitUntil(
-        () ->
-            inMemoryMetrics()
-                .counterValue("tasks/completed", TEST_REPLICATION_REMOTE)
-                .filter(counter -> counter == expected)
-                .isPresent());
+  private void assertThatEventListenerHasReceivedNumEvents(int numExpectedEvents) {
+    assertThat(eventListener.numEventsReceived()).isEqualTo(numExpectedEvents);
+  }
+
+  private void assertThatRefReplicatedEventsContainsStatus(
+      ReplicationState.RefFetchResult refFetchResult) {
+    Stream<FetchRefReplicatedEvent> replicatedStream = eventListener.eventsStream();
+    assertThat(replicatedStream.map(FetchRefReplicatedEvent::getStatus))
+        .contains(refFetchResult.toString());
+  }
+
+  private void assertThatRefReplicatedEventsContainsExactlyStatuses(
+      ReplicationState.RefFetchResult... refFetchResult) {
+    List<String> expectedStatuses =
+        Stream.of(refFetchResult).map(Object::toString).collect(Collectors.toList());
+    Stream<FetchRefReplicatedEvent> replicatedStream = eventListener.eventsStream();
+    assertThat(replicatedStream.map(FetchRefReplicatedEvent::getStatus))
+        .containsExactlyElementsIn(expectedStatuses);
+  }
+
+  private void waitUntilReplicationCompleted(int expected) throws Exception {
+    waitUntilReplicationTask("completed", expected);
+  }
+
+  private void waitUntilReplicationFailed(int expected) throws Exception {
+    waitUntilReplicationTask("failed", expected);
+  }
+
+  private void waitUntilReplicationTask(String status, int expected) throws Exception {
+    if (isAsyncReplication()) {
+      waitUntil(
+          () ->
+              inMemoryMetrics()
+                  .counterValue("tasks/" + status, TEST_REPLICATION_REMOTE)
+                  .filter(counter -> counter == expected)
+                  .isPresent());
+    }
   }
 
   private InMemoryMetricMaker inMemoryMetrics() {
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationSetupBase.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationSetupBase.java
index e07d481..fea576a 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationSetupBase.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationSetupBase.java
@@ -33,7 +33,6 @@
 import java.util.Optional;
 import java.util.function.Function;
 import java.util.function.Supplier;
-import org.eclipse.jgit.errors.ConfigInvalidException;
 import org.eclipse.jgit.lib.Ref;
 import org.eclipse.jgit.lib.Repository;
 import org.eclipse.jgit.storage.file.FileBasedConfig;
@@ -76,7 +75,7 @@
     super.setUpTestPlugin();
   }
 
-  protected Ref getRef(Repository repo, String branchName) throws IOException {
+  protected Ref getRef(Repository repo, String branchName) throws Exception {
     return repo.getRefDatabase().exactRef(branchName);
   }
 
@@ -90,8 +89,7 @@
   }
 
   protected void setReplicationSource(
-      String remoteName, String replicaSuffix, Optional<String> project)
-      throws IOException, ConfigInvalidException {
+      String remoteName, String replicaSuffix, Optional<String> project) throws Exception {
     setReplicationSource(remoteName, Arrays.asList(replicaSuffix), project);
   }
 
@@ -99,13 +97,13 @@
       String remoteName, List<String> replicaSuffixes, Optional<String> project) throws IOException;
 
   protected void setReplicationCredentials(String remoteName, String username, String password)
-      throws IOException {
+      throws Exception {
     secureConfig.setString("remote", remoteName, "username", username);
     secureConfig.setString("remote", remoteName, "password", password);
     secureConfig.save();
   }
 
-  protected void waitUntil(Supplier<Boolean> waitCondition) throws InterruptedException {
+  protected void waitUntil(Supplier<Boolean> waitCondition) throws Exception {
     WaitUtil.waitUntil(waitCondition, TEST_TIMEOUT);
   }
 
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 e40b90c..603528d 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
@@ -48,7 +48,6 @@
 import com.googlesource.gerrit.plugins.replication.ReplicationConfig;
 import com.googlesource.gerrit.plugins.replication.ReplicationFileBasedConfig;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionData;
-import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
 import com.googlesource.gerrit.plugins.replication.pull.client.FetchApiClient;
 import com.googlesource.gerrit.plugins.replication.pull.client.FetchRestApiClient;
 import com.googlesource.gerrit.plugins.replication.pull.client.HttpResult;
@@ -59,8 +58,6 @@
 import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
-import org.apache.http.client.ClientProtocolException;
-import org.eclipse.jgit.errors.LargeObjectException;
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.storage.file.FileBasedConfig;
 import org.eclipse.jgit.util.FS;
@@ -111,7 +108,7 @@
   private Path pluginDataPath;
 
   @Before
-  public void setup() throws IOException, LargeObjectException {
+  public void setup() throws Exception {
     Path sitePath = createTempPath("site");
     sitePaths = new SitePaths(sitePath);
     Path pluginDataPath = createTempPath("data");
@@ -180,7 +177,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectWhenMetaRef() throws ClientProtocolException, IOException {
+  public void shouldCallSendObjectWhenMetaRef() throws Exception {
     Event event = new TestEvent("refs/changes/01/1/meta");
     objectUnderTest.start();
     objectUnderTest.onEvent(event);
@@ -189,8 +186,7 @@
   }
 
   @Test
-  public void shouldIgnoreEventWhenIsNotLocalInstanceId()
-      throws ClientProtocolException, IOException {
+  public void shouldIgnoreEventWhenIsNotLocalInstanceId() throws Exception {
     Event event = new TestEvent();
     event.instanceId = FOREIGN_INSTANCE_ID;
     objectUnderTest.start();
@@ -201,7 +197,7 @@
   }
 
   @Test
-  public void shouldCallInitProjectWhenProjectIsMissing() throws IOException {
+  public void shouldCallInitProjectWhenProjectIsMissing() throws Exception {
     Event event = new TestEvent("refs/changes/01/1/meta");
     when(httpResult.isSuccessful()).thenReturn(false);
     when(httpResult.isProjectMissing(any())).thenReturn(true);
@@ -214,7 +210,7 @@
   }
 
   @Test
-  public void shouldNotCallInitProjectWhenReplicateNewRepositoriesNotSet() throws IOException {
+  public void shouldNotCallInitProjectWhenReplicateNewRepositoriesNotSet() throws Exception {
     Event event = new TestEvent("refs/changes/01/1/meta");
     when(httpResult.isSuccessful()).thenReturn(false);
     when(httpResult.isProjectMissing(any())).thenReturn(true);
@@ -227,7 +223,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectWhenPatchSetRef() throws ClientProtocolException, IOException {
+  public void shouldCallSendObjectWhenPatchSetRef() throws Exception {
     Event event = new TestEvent("refs/changes/01/1/1");
     objectUnderTest.start();
     objectUnderTest.onEvent(event);
@@ -236,8 +232,7 @@
   }
 
   @Test
-  public void shouldFallbackToCallFetchWhenIOException()
-      throws ClientProtocolException, IOException, LargeObjectException, RefUpdateException {
+  public void shouldFallbackToCallFetchWhenIOException() throws Exception {
     Event event = new TestEvent("refs/changes/01/1/meta");
     objectUnderTest.start();
 
@@ -249,8 +244,7 @@
   }
 
   @Test
-  public void shouldFallbackToCallFetchWhenLargeRef()
-      throws ClientProtocolException, IOException, LargeObjectException, RefUpdateException {
+  public void shouldFallbackToCallFetchWhenLargeRef() throws Exception {
     Event event = new TestEvent("refs/changes/01/1/1");
     objectUnderTest.start();
 
@@ -262,8 +256,7 @@
   }
 
   @Test
-  public void shouldFallbackToCallFetchWhenParentObjectIsMissing()
-      throws ClientProtocolException, IOException {
+  public void shouldFallbackToCallFetchWhenParentObjectIsMissing() throws Exception {
     Event event = new TestEvent("refs/changes/01/1/1");
     objectUnderTest.start();
 
@@ -279,7 +272,7 @@
 
   @Test
   public void shouldFallbackToApplyAllParentObjectsWhenParentObjectIsMissingOnMetaRef()
-      throws ClientProtocolException, IOException {
+      throws Exception {
     Event event = new TestEvent("refs/changes/01/1/meta");
     objectUnderTest.start();
 
@@ -305,7 +298,7 @@
 
   @Test
   public void shouldFallbackToApplyAllParentObjectsWhenParentObjectIsMissingOnAllowedRefs()
-      throws ClientProtocolException, IOException {
+      throws Exception {
     String refName = "refs/tags/test-tag";
     Event event = new TestEvent(refName);
     objectUnderTest.start();
@@ -332,7 +325,7 @@
   }
 
   @Test
-  public void shouldSkipEventWhenMultiSiteVersionRef() throws IOException {
+  public void shouldSkipEventWhenMultiSiteVersionRef() throws Exception {
     FileBasedConfig fileConfig =
         new FileBasedConfig(sitePaths.etc_dir.resolve("replication.config").toFile(), FS.DETECTED);
     fileConfig.setString("replication", null, "excludeRefs", "refs/multi-site/version");
@@ -362,7 +355,7 @@
   }
 
   @Test
-  public void shouldSetShutdownStateWhenStopping() throws IOException {
+  public void shouldSetShutdownStateWhenStopping() throws Exception {
     objectUnderTest.stop();
     assertThat(shutdownState.isShuttingDown()).isTrue();
   }
@@ -431,7 +424,7 @@
     verify(source, never()).scheduleUpdateHead(any(), any(), any());
   }
 
-  protected static Path createTempPath(String prefix) throws IOException {
+  protected static Path createTempPath(String prefix) throws Exception {
     return createTempDirectory(prefix);
   }
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationStateTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationStateTest.java
index eb52493..a1ef985 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationStateTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationStateTest.java
@@ -19,7 +19,6 @@
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 
-import java.net.URISyntaxException;
 import org.eclipse.jgit.lib.RefUpdate;
 import org.eclipse.jgit.transport.URIish;
 import org.junit.Before;
@@ -57,7 +56,7 @@
   }
 
   @Test
-  public void shouldFireEventsForReplicationOfOneRefToOneNode() throws URISyntaxException {
+  public void shouldFireEventsForReplicationOfOneRefToOneNode() throws Exception {
     URIish uri = new URIish("git://someHost/someRepo.git");
 
     // actual test
@@ -83,8 +82,7 @@
   }
 
   @Test
-  public void shouldFireEventsForReplicationOfMultipleRefsToMultipleNodes()
-      throws URISyntaxException {
+  public void shouldFireEventsForReplicationOfMultipleRefsToMultipleNodes() throws Exception {
     URIish uri1 = new URIish("git://host1/someRepo.git");
     URIish uri2 = new URIish("git://host2/someRepo.git");
     URIish uri3 = new URIish("git://host3/someRepo.git");
@@ -173,7 +171,7 @@
 
   @Test
   public void shouldFireEventsWhenSomeReplicationCompleteBeforeAllTasksAreScheduled()
-      throws URISyntaxException {
+      throws Exception {
     URIish uri1 = new URIish("git://host1/someRepo.git");
 
     // actual test
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 a541d7e..96bd68e 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
@@ -33,7 +33,6 @@
 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;
@@ -41,7 +40,6 @@
 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.List;
 import java.util.Optional;
 import java.util.stream.Collectors;
@@ -178,20 +176,20 @@
     assertThat(parentObjectIds).hasSize(numberOfParents);
   }
 
-  private void addMultipleComments(int numberOfParents, Id changeId) throws RestApiException {
+  private void addMultipleComments(int numberOfParents, Id changeId) throws Exception {
     for (int i = 0; i < numberOfParents; i++) {
       addComment(changeId);
     }
   }
 
-  private void setReplicationConfig(int numberOfParents) throws IOException {
+  private void setReplicationConfig(int numberOfParents) throws Exception {
     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 {
+  private void addComment(Id changeId) throws Exception {
     gApi.changes().id(changeId.get()).current().review(new ReviewInput().message("foo"));
   }
 
@@ -209,7 +207,7 @@
     }
   }
 
-  protected Optional<ObjectId> refObjectId(String refName) throws IOException {
+  protected Optional<ObjectId> refObjectId(String refName) throws Exception {
     try (Repository repo = repoManager.openRepository(project)) {
       return Optional.ofNullable(repo.exactRef(refName)).map(Ref::getObjectId);
     }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/WaitUtil.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/WaitUtil.java
index a99ac18..77145ba 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/WaitUtil.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/WaitUtil.java
@@ -21,8 +21,7 @@
 import java.util.function.Supplier;
 
 public class WaitUtil {
-  public static void waitUntil(Supplier<Boolean> waitCondition, Duration timeout)
-      throws InterruptedException {
+  public static void waitUntil(Supplier<Boolean> waitCondition, Duration timeout) throws Exception {
     Stopwatch stopwatch = Stopwatch.createStarted();
     while (!waitCondition.get()) {
       if (stopwatch.elapsed().compareTo(timeout) > 0) {
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 e638653..4a3fa55 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
@@ -44,7 +44,6 @@
 import java.util.Optional;
 import org.apache.http.HttpHeaders;
 import org.apache.http.HttpResponse;
-import org.apache.http.auth.AuthenticationException;
 import org.apache.http.auth.UsernamePasswordCredentials;
 import org.apache.http.client.ClientProtocolException;
 import org.apache.http.client.ResponseHandler;
@@ -181,17 +180,17 @@
   }
 
   protected HttpRequestBase withBasicAuthenticationAsAdmin(HttpRequestBase httpRequest)
-      throws AuthenticationException {
+      throws Exception {
     return withBasicAuthentication(httpRequest, admin);
   }
 
   protected HttpRequestBase withBasicAuthenticationAsUser(HttpRequestBase httpRequest)
-      throws AuthenticationException {
+      throws Exception {
     return withBasicAuthentication(httpRequest, user);
   }
 
   private HttpRequestBase withBasicAuthentication(HttpRequestBase httpRequest, TestAccount account)
-      throws AuthenticationException {
+      throws Exception {
     UsernamePasswordCredentials creds =
         new UsernamePasswordCredentials(account.username(), account.httpPassword());
     httpRequest.addHeader(new BasicScheme().authenticate(creds, httpRequest, null));
@@ -203,13 +202,12 @@
   }
 
   private void setReplicationSource(
-      String remoteName, String replicaSuffix, Optional<String> project) throws IOException {
+      String remoteName, String replicaSuffix, Optional<String> project) throws Exception {
     setReplicationSource(remoteName, Arrays.asList(replicaSuffix), project);
   }
 
   private void setReplicationSource(
-      String remoteName, List<String> replicaSuffixes, Optional<String> project)
-      throws IOException {
+      String remoteName, List<String> replicaSuffixes, Optional<String> project) throws Exception {
 
     List<String> replicaUrls =
         replicaSuffixes.stream()
@@ -226,7 +224,7 @@
   }
 
   private void setReplicationCredentials(String remoteName, String username, String password)
-      throws IOException {
+      throws Exception {
     secureConfig.setString("remote", remoteName, "username", username);
     secureConfig.setString("remote", remoteName, "password", password);
     secureConfig.save();
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 d0cd1b4..245003a 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
@@ -28,15 +28,11 @@
 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.server.project.ProjectResource;
 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.RevisionObjectData;
 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.api.exception.UnauthorizedAuthException;
-import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Collections;
@@ -91,14 +87,14 @@
   @Mock FetchPreconditions preConditions;
 
   @Before
-  public void setup() throws UnauthorizedAuthException {
+  public void setup() throws Exception {
     when(preConditions.canCallFetchApi()).thenReturn(true);
 
     applyObjectAction = new ApplyObjectAction(applyObjectCommand, deleteRefCommand, preConditions);
   }
 
   @Test
-  public void shouldReturnCreatedResponseCode() throws RestApiException {
+  public void shouldReturnCreatedResponseCode() throws Exception {
     RevisionInput inputParams =
         new RevisionInput(label, refName, DUMMY_EVENT_TIMESTAMP, createSampleRevisionData());
 
@@ -108,7 +104,7 @@
   }
 
   @Test
-  public void shouldReturnCreatedResponseCodeForBlob() throws RestApiException {
+  public void shouldReturnCreatedResponseCodeForBlob() throws Exception {
     byte[] blobData = "foo".getBytes(StandardCharsets.UTF_8);
     RevisionInput inputParams =
         new RevisionInput(
@@ -191,8 +187,7 @@
   }
 
   @Test(expected = AuthException.class)
-  public void shouldThrowAuthExceptionWhenCallFetchActionCapabilityNotAssigned()
-      throws RestApiException {
+  public void shouldThrowAuthExceptionWhenCallFetchActionCapabilityNotAssigned() throws Exception {
     RevisionInput inputParams =
         new RevisionInput(label, refName, DUMMY_EVENT_TIMESTAMP, createSampleRevisionData());
 
@@ -202,8 +197,7 @@
   }
 
   @Test(expected = ResourceConflictException.class)
-  public void shouldThrowResourceConflictExceptionWhenMissingParentObject()
-      throws RestApiException, IOException, RefUpdateException, MissingParentObjectException {
+  public void shouldThrowResourceConflictExceptionWhenMissingParentObject() throws Exception {
     RevisionInput inputParams =
         new RevisionInput(label, refName, DUMMY_EVENT_TIMESTAMP, createSampleRevisionData());
 
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 7f5a67c..3375cf6 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
@@ -29,7 +29,6 @@
 import com.google.gerrit.metrics.Timer1;
 import com.google.gerrit.server.events.Event;
 import com.google.gerrit.server.events.EventDispatcher;
-import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.googlesource.gerrit.plugins.replication.pull.ApplyObjectMetrics;
 import com.googlesource.gerrit.plugins.replication.pull.ApplyObjectsCacheKey;
 import com.googlesource.gerrit.plugins.replication.pull.FetchRefReplicatedEvent;
@@ -38,12 +37,9 @@
 import com.googlesource.gerrit.plugins.replication.pull.SourcesCollection;
 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.MissingParentObjectException;
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.ApplyObject;
 import com.googlesource.gerrit.plugins.replication.pull.fetch.RefUpdateState;
-import java.io.IOException;
-import java.net.URISyntaxException;
 import java.util.Collections;
 import java.util.Optional;
 import org.eclipse.jgit.lib.Constants;
@@ -86,7 +82,7 @@
   private ApplyObjectCommand objectUnderTest;
 
   @Before
-  public void setup() throws MissingParentObjectException, IOException, URISyntaxException {
+  public void setup() throws Exception {
     cache = CacheBuilder.newBuilder().build();
     RefUpdateState state = new RefUpdateState(TEST_REMOTE_NAME, RefUpdate.Result.NEW);
     TEST_REMOTE_URI = new URIish("git://some.remote.uri");
@@ -103,9 +99,7 @@
   }
 
   @Test
-  public void shouldSendEventWhenApplyObject()
-      throws PermissionBackendException, IOException, RefUpdateException,
-          MissingParentObjectException {
+  public void shouldSendEventWhenApplyObject() throws Exception {
     RevisionData sampleRevisionData =
         createSampleRevisionData(sampleCommitObjectId, sampleTreeObjectId);
     objectUnderTest.applyObject(
@@ -125,8 +119,7 @@
   }
 
   @Test
-  public void shouldInsertIntoApplyObjectsCacheWhenApplyObjectIsSuccessful()
-      throws IOException, RefUpdateException, MissingParentObjectException {
+  public void shouldInsertIntoApplyObjectsCacheWhenApplyObjectIsSuccessful() throws Exception {
     RevisionData sampleRevisionData =
         createSampleRevisionData(sampleCommitObjectId, sampleTreeObjectId);
     RevisionData sampleRevisionData2 =
@@ -155,8 +148,7 @@
   }
 
   @Test(expected = RefUpdateException.class)
-  public void shouldNotInsertIntoApplyObjectsCacheWhenApplyObjectIsFailure()
-      throws IOException, RefUpdateException, MissingParentObjectException {
+  public void shouldNotInsertIntoApplyObjectsCacheWhenApplyObjectIsFailure() throws Exception {
     RevisionData sampleRevisionData =
         createSampleRevisionData(sampleCommitObjectId, sampleTreeObjectId);
     RefUpdateState failureState = new RefUpdateState(TEST_REMOTE_NAME, RefUpdate.Result.IO_FAILURE);
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/BearerAuthenticationFilterTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/BearerAuthenticationFilterTest.java
index 72aee76..95b86cc 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/BearerAuthenticationFilterTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/BearerAuthenticationFilterTest.java
@@ -25,10 +25,8 @@
 import com.google.gerrit.server.util.ThreadLocalRequestContext;
 import com.google.inject.Provider;
 import com.googlesource.gerrit.plugins.replication.pull.auth.PullReplicationInternalUser;
-import java.io.IOException;
 import java.util.Optional;
 import javax.servlet.FilterChain;
-import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import org.junit.Test;
@@ -53,13 +51,13 @@
   private final String pluginName = "pull-replication";
 
   private void authenticateAndFilter(String method, String uri, Optional<String> queryStringMaybe)
-      throws ServletException, IOException {
+      throws Exception {
     when(httpServletRequest.getMethod()).thenReturn(method);
     authenticateAndFilter(uri, queryStringMaybe);
   }
 
   private void authenticateAndFilter(String uri, Optional<String> queryStringMaybe)
-      throws ServletException, IOException {
+      throws Exception {
     final String bearerToken = "some-bearer-token";
     when(httpServletRequest.getRequestURI()).thenReturn(uri);
     queryStringMaybe.ifPresent(qs -> when(httpServletRequest.getQueryString()).thenReturn(qs));
@@ -82,51 +80,49 @@
   }
 
   @Test
-  public void shouldAuthenticateWhenFetch() throws ServletException, IOException {
+  public void shouldAuthenticateWhenFetch() throws Exception {
     authenticateAndFilter("any-prefix/pull-replication~fetch", NO_QUERY_PARAMETERS);
   }
 
   @Test
-  public void shouldAuthenticateWhenApplyObject() throws ServletException, IOException {
+  public void shouldAuthenticateWhenApplyObject() throws Exception {
     authenticateAndFilter("any-prefix/pull-replication~apply-object", NO_QUERY_PARAMETERS);
   }
 
   @Test
-  public void shouldAuthenticateWhenApplyObjects() throws ServletException, IOException {
+  public void shouldAuthenticateWhenApplyObjects() throws Exception {
     authenticateAndFilter("any-prefix/pull-replication~apply-objects", NO_QUERY_PARAMETERS);
   }
 
   @Test
-  public void shouldAuthenticateWhenDeleteProject() throws ServletException, IOException {
+  public void shouldAuthenticateWhenDeleteProject() throws Exception {
     authenticateAndFilter("any-prefix/pull-replication~delete-project", NO_QUERY_PARAMETERS);
   }
 
   @Test
-  public void shouldAuthenticateWhenUpdateHead() throws ServletException, IOException {
+  public void shouldAuthenticateWhenUpdateHead() throws Exception {
     authenticateAndFilter(
         "PUT", "any-prefix/projects/my-project/pull-replication~HEAD", NO_QUERY_PARAMETERS);
   }
 
   @Test
-  public void shouldAuthenticateWhenInitProject() throws ServletException, IOException {
+  public void shouldAuthenticateWhenInitProject() throws Exception {
     authenticateAndFilter(
         "any-prefix/pull-replication/init-project/my-project.git", NO_QUERY_PARAMETERS);
   }
 
   @Test
-  public void shouldAuthenticateWhenGitUploadPack() throws ServletException, IOException {
+  public void shouldAuthenticateWhenGitUploadPack() throws Exception {
     authenticateAndFilter("any-prefix/git-upload-pack", NO_QUERY_PARAMETERS);
   }
 
   @Test
-  public void shouldAuthenticateWhenGitUploadPackInQueryParameter()
-      throws ServletException, IOException {
+  public void shouldAuthenticateWhenGitUploadPackInQueryParameter() throws Exception {
     authenticateAndFilter("any-prefix", GIT_UPLOAD_PACK_QUERY_PARAMETER);
   }
 
   @Test
-  public void shouldGoNextInChainWhenGitUploadPackWithoutAuthenticationHeader()
-      throws ServletException, IOException {
+  public void shouldGoNextInChainWhenGitUploadPackWithoutAuthenticationHeader() throws Exception {
     when(httpServletRequest.getRequestURI()).thenReturn("any-prefix/git-upload-pack");
 
     final BearerAuthenticationFilter filter =
@@ -144,7 +140,7 @@
 
   @Test
   public void shouldGoNextInChainWhenGitUploadPackWithAuthenticationHeaderDifferentThanBearer()
-      throws ServletException, IOException {
+      throws Exception {
     when(httpServletRequest.getRequestURI()).thenReturn("any-prefix/git-upload-pack");
     when(httpServletRequest.getHeader("Authorization")).thenReturn("some-authorization");
     final BearerAuthenticationFilter filter =
@@ -161,7 +157,7 @@
   }
 
   @Test
-  public void shouldBe401WhenBearerTokenDoesNotMatch() throws ServletException, IOException {
+  public void shouldBe401WhenBearerTokenDoesNotMatch() throws Exception {
     when(httpServletRequest.getRequestURI()).thenReturn("any-prefix/pull-replication~fetch");
     when(httpServletRequest.getHeader("Authorization"))
         .thenReturn(String.format("Bearer %s", "some-different-bearer-token"));
@@ -181,7 +177,7 @@
   }
 
   @Test
-  public void shouldBe401WhenBearerTokenCannotBeExtracted() throws ServletException, IOException {
+  public void shouldBe401WhenBearerTokenCannotBeExtracted() throws Exception {
     when(httpServletRequest.getRequestURI()).thenReturn("any-prefix/pull-replication~fetch");
     when(httpServletRequest.getHeader("Authorization")).thenReturn("bearer token");
 
@@ -200,7 +196,7 @@
   }
 
   @Test
-  public void shouldBe401WhenNoAuthorizationHeaderInRequest() throws ServletException, IOException {
+  public void shouldBe401WhenNoAuthorizationHeaderInRequest() throws Exception {
     when(httpServletRequest.getRequestURI()).thenReturn("any-prefix/pull-replication~fetch");
 
     final BearerAuthenticationFilter filter =
@@ -218,7 +214,7 @@
   }
 
   @Test
-  public void shouldGoNextInChainWhenUriDoesNotMatch() throws ServletException, IOException {
+  public void shouldGoNextInChainWhenUriDoesNotMatch() throws Exception {
     when(httpServletRequest.getRequestURI()).thenReturn("any-url");
 
     final BearerAuthenticationFilter filter =
@@ -235,8 +231,7 @@
   }
 
   @Test
-  public void shouldGoNextInChainWhenBasicAuthorizationIsRequired()
-      throws ServletException, IOException {
+  public void shouldGoNextInChainWhenBasicAuthorizationIsRequired() throws Exception {
     when(httpServletRequest.getRequestURI())
         .thenReturn("/a/projects/my-project/pull-replication~fetch");
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchActionTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchActionTest.java
index e7048f6..024a712 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchActionTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchActionTest.java
@@ -33,7 +33,6 @@
 import com.google.gerrit.server.git.WorkQueue.Task;
 import com.google.gerrit.server.project.ProjectResource;
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RemoteConfigurationMissingException;
-import com.googlesource.gerrit.plugins.replication.pull.api.exception.UnauthorizedAuthException;
 import java.util.Optional;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ScheduledExecutorService;
@@ -67,7 +66,7 @@
   @Mock FetchPreconditions preConditions;
 
   @Before
-  public void setup() throws UnauthorizedAuthException {
+  public void setup() throws Exception {
     when(fetchJobFactory.create(any(), any(), any())).thenReturn(fetchJob);
     when(workQueue.getDefaultQueue()).thenReturn(exceutorService);
     when(urlFormatter.getRestUrl(anyString())).thenReturn(Optional.of(location));
@@ -89,7 +88,7 @@
   }
 
   @Test
-  public void shouldReturnCreatedResponseCode() throws RestApiException {
+  public void shouldReturnCreatedResponseCode() throws Exception {
     FetchAction.Input inputParams = new FetchAction.Input();
     inputParams.label = label;
     inputParams.refName = refName;
@@ -146,9 +145,7 @@
   }
 
   @Test(expected = RestApiException.class)
-  public void shouldThrowRestApiExceptionWhenPocessingInterrupted()
-      throws RestApiException, InterruptedException, ExecutionException,
-          RemoteConfigurationMissingException, TimeoutException {
+  public void shouldThrowRestApiExceptionWhenPocessingInterrupted() throws Exception {
     FetchAction.Input inputParams = new FetchAction.Input();
     inputParams.label = label;
     inputParams.refName = refName;
@@ -159,9 +156,7 @@
   }
 
   @Test(expected = UnprocessableEntityException.class)
-  public void shouldThrowRestApiExceptionWhenNoSurceForGivenLabel()
-      throws RestApiException, InterruptedException, ExecutionException,
-          RemoteConfigurationMissingException, TimeoutException {
+  public void shouldThrowRestApiExceptionWhenNoSurceForGivenLabel() throws Exception {
     FetchAction.Input inputParams = new FetchAction.Input();
     inputParams.label = "non-existing-label";
     inputParams.refName = refName;
@@ -174,9 +169,7 @@
   }
 
   @Test(expected = RestApiException.class)
-  public void shouldThrowRestApiExceptionWhenIssueDuringPocessing()
-      throws RestApiException, InterruptedException, ExecutionException,
-          RemoteConfigurationMissingException, TimeoutException {
+  public void shouldThrowRestApiExceptionWhenIssueDuringPocessing() throws Exception {
     FetchAction.Input inputParams = new FetchAction.Input();
     inputParams.label = label;
     inputParams.refName = refName;
@@ -189,9 +182,7 @@
   }
 
   @Test(expected = RestApiException.class)
-  public void shouldThrowRestApiExceptionWhenIssueWithUrlParam()
-      throws RestApiException, InterruptedException, ExecutionException,
-          RemoteConfigurationMissingException, TimeoutException {
+  public void shouldThrowRestApiExceptionWhenIssueWithUrlParam() throws Exception {
     FetchAction.Input inputParams = new FetchAction.Input();
     inputParams.label = label;
     inputParams.refName = refName;
@@ -202,9 +193,7 @@
   }
 
   @Test(expected = RestApiException.class)
-  public void shouldThrowRestApiExceptionWhenTimeout()
-      throws RestApiException, InterruptedException, ExecutionException,
-          RemoteConfigurationMissingException, TimeoutException {
+  public void shouldThrowRestApiExceptionWhenTimeout() throws Exception {
     FetchAction.Input inputParams = new FetchAction.Input();
     inputParams.label = label;
     inputParams.refName = refName;
@@ -215,8 +204,7 @@
   }
 
   @Test(expected = AuthException.class)
-  public void shouldThrowAuthExceptionWhenCallFetchActionCapabilityNotAssigned()
-      throws RestApiException {
+  public void shouldThrowAuthExceptionWhenCallFetchActionCapabilityNotAssigned() throws Exception {
     FetchAction.Input inputParams = new FetchAction.Input();
     inputParams.label = label;
     inputParams.refName = refName;
@@ -227,7 +215,7 @@
   }
 
   @Test
-  public void shouldReturnScheduledTaskForAsyncCall() throws RestApiException {
+  public void shouldReturnScheduledTaskForAsyncCall() throws Exception {
     FetchAction.Input inputParams = new FetchAction.Input();
     inputParams.label = label;
     inputParams.refName = refName;
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchCommandTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchCommandTest.java
index de8036e..777350b 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchCommandTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchCommandTest.java
@@ -15,12 +15,10 @@
 package com.googlesource.gerrit.plugins.replication.pull.api;
 
 import static com.google.gerrit.testing.GerritJUnit.assertThrows;
-import static com.googlesource.gerrit.plugins.replication.pull.ReplicationType.ASYNC;
-import static com.googlesource.gerrit.plugins.replication.pull.ReplicationType.SYNC;
 import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.anyLong;
 import static org.mockito.Mockito.anyString;
 import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -33,13 +31,9 @@
 import com.googlesource.gerrit.plugins.replication.pull.Source;
 import com.googlesource.gerrit.plugins.replication.pull.SourcesCollection;
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RemoteConfigurationMissingException;
-import java.net.URISyntaxException;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import org.eclipse.jgit.transport.URIish;
 import org.junit.Before;
 import org.junit.Test;
@@ -69,48 +63,35 @@
   FetchCommand objectUnderTest;
 
   @Before
-  public void setup() throws URISyntaxException {
+  public void setup() throws Exception {
     projectName = Project.nameKey("sample_project");
     uri = new URIish("file://sample_host/repository_path/repo.git");
     label = "instance-1-label";
 
     when(fetchReplicationStateFactory.create(any())).thenReturn(state);
-    when(sources.getByRemoteName(label)).thenReturn(Optional.of(source));
-    when(source.schedule(eq(projectName), eq(REF_NAME_TO_FETCH), eq(state), any(), any()))
+    when(sources.getByRemoteName(eq(label))).thenReturn(Optional.of(source));
+    when(source.schedule(eq(projectName), eq(REF_NAME_TO_FETCH), eq(state), any()))
         .thenReturn(CompletableFuture.completedFuture(null));
     objectUnderTest =
         new FetchCommand(fetchReplicationStateFactory, fetchStateLog, sources, eventDispatcher);
   }
 
   @Test
-  public void shouldScheduleRefFetch()
-      throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
-          TimeoutException {
-    objectUnderTest.fetchSync(projectName, label, REF_NAME_TO_FETCH);
-
-    verify(source, times(1))
-        .schedule(projectName, REF_NAME_TO_FETCH, state, SYNC, Optional.empty());
-  }
-
-  @Test
-  public void shouldScheduleRefFetchWithDelay()
-      throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
-          TimeoutException {
+  public void shouldScheduleRefFetchWithDelay() throws Exception {
     objectUnderTest.fetchAsync(projectName, label, REF_NAME_TO_FETCH, apiRequestMetrics);
 
     verify(source, times(1))
-        .schedule(projectName, REF_NAME_TO_FETCH, state, ASYNC, Optional.of(apiRequestMetrics));
+        .schedule(
+            eq(projectName), eq(REF_NAME_TO_FETCH), eq(state), eq(Optional.of(apiRequestMetrics)));
   }
 
   @Test
-  public void shouldMarkAllFetchTasksScheduled()
-      throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
-          TimeoutException {
+  public void shouldNotScheduleAsyncTaskWhenFetchSync() throws Exception {
     objectUnderTest.fetchSync(projectName, label, REF_NAME_TO_FETCH);
 
-    verify(source, times(1))
-        .schedule(projectName, REF_NAME_TO_FETCH, state, SYNC, Optional.empty());
-    verify(state, times(1)).markAllFetchTasksScheduled();
+    verify(source, never())
+        .schedule(
+            eq(projectName), eq(REF_NAME_TO_FETCH), eq(state), eq(Optional.of(apiRequestMetrics)));
   }
 
   @Test
@@ -120,50 +101,4 @@
         () -> objectUnderTest.fetchSync(projectName, "unknownLabel", REF_NAME_TO_FETCH));
     verify(fetchStateLog, times(1)).error(anyString(), eq(state));
   }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void shouldUpdateStateWhenInterruptedException()
-      throws InterruptedException, ExecutionException, TimeoutException {
-    when(future.get()).thenThrow(new InterruptedException());
-    when(source.schedule(projectName, REF_NAME_TO_FETCH, state, SYNC, Optional.empty()))
-        .thenReturn(future);
-
-    InterruptedException e =
-        assertThrows(
-            InterruptedException.class,
-            () -> objectUnderTest.fetchSync(projectName, label, REF_NAME_TO_FETCH));
-    verify(fetchStateLog, times(1)).error(anyString(), eq(e), eq(state));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void shouldUpdateStateWhenExecutionException()
-      throws InterruptedException, ExecutionException, TimeoutException {
-    when(future.get()).thenThrow(new ExecutionException(new Exception()));
-    when(source.schedule(projectName, REF_NAME_TO_FETCH, state, SYNC, Optional.empty()))
-        .thenReturn(future);
-
-    ExecutionException e =
-        assertThrows(
-            ExecutionException.class,
-            () -> objectUnderTest.fetchSync(projectName, label, REF_NAME_TO_FETCH));
-    verify(fetchStateLog, times(1)).error(anyString(), eq(e), eq(state));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void shouldUpdateStateWhenTimeoutException()
-      throws InterruptedException, ExecutionException, TimeoutException {
-    when(future.get(anyLong(), eq(TimeUnit.SECONDS))).thenThrow(new TimeoutException());
-    when(source.schedule(projectName, REF_NAME_TO_FETCH, state, SYNC, Optional.empty()))
-        .thenReturn(future);
-    when(source.getTimeout()).thenReturn(1);
-
-    TimeoutException e =
-        assertThrows(
-            TimeoutException.class,
-            () -> objectUnderTest.fetchSync(projectName, label, REF_NAME_TO_FETCH));
-    verify(fetchStateLog, times(1)).error(anyString(), eq(e), eq(state));
-  }
 }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationFilterTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationFilterTest.java
index 5ede668..9492d49 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationFilterTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/PullReplicationFilterTest.java
@@ -4,7 +4,6 @@
 import static com.google.gerrit.httpd.restapi.RestApiServlet.SC_UNPROCESSABLE_ENTITY;
 import static javax.servlet.http.HttpServletResponse.SC_CONFLICT;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.lenient;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -174,15 +173,11 @@
 
     when(request.getRequestURI()).thenReturn(INIT_PROJECT_URI);
     when(request.getHeader(ACCEPT)).thenReturn(MediaType.PLAIN_TEXT_UTF_8.toString());
-    when(projectInitializationAction.initProject(PROJECT_NAME_GIT)).thenReturn(true);
-    when(response.getWriter()).thenReturn(printWriter);
 
     final PullReplicationFilter pullReplicationFilter = createPullReplicationFilter();
     pullReplicationFilter.doFilter(request, response, filterChain);
 
-    verify(request, times(5)).getRequestURI();
-    verify(projectInitializationAction).initProject(eq(PROJECT_NAME_GIT));
-    verify(response).getWriter();
+    verify(projectInitializationAction).service(request, response);
   }
 
   @Test
@@ -254,19 +249,6 @@
   }
 
   @Test
-  public void shouldBe500WhenProjectCannotBeInitiated() throws Exception {
-    when(request.getRequestURI()).thenReturn(INIT_PROJECT_URI);
-    when(request.getHeader(ACCEPT)).thenReturn(MediaType.PLAIN_TEXT_UTF_8.toString());
-    when(projectInitializationAction.initProject(PROJECT_NAME_GIT)).thenReturn(false);
-    when(response.getOutputStream()).thenReturn(outputStream);
-
-    final PullReplicationFilter pullReplicationFilter = createPullReplicationFilter();
-    pullReplicationFilter.doFilter(request, response, filterChain);
-
-    verify(response).setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
-  }
-
-  @Test
   public void shouldBe500WhenResourceNotFound() throws Exception {
     when(request.getRequestURI()).thenReturn(DELETE_PROJECT_URI);
     when(request.getMethod()).thenReturn("DELETE");
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/UpdateHeadActionIT.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/UpdateHeadActionIT.java
index 9ceae5a..f3802a3 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/UpdateHeadActionIT.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/UpdateHeadActionIT.java
@@ -26,10 +26,7 @@
 import com.google.gerrit.extensions.api.projects.HeadInput;
 import com.google.gson.Gson;
 import com.google.inject.Inject;
-import java.io.IOException;
 import javax.servlet.http.HttpServletResponse;
-import org.apache.http.auth.AuthenticationException;
-import org.apache.http.client.ClientProtocolException;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -173,8 +170,7 @@
     shouldReturnOKForUserWithPullReplicationCapabilityTest();
   }
 
-  private void shouldReturnOKForUserWithPullReplicationCapabilityTest()
-      throws ClientProtocolException, IOException, AuthenticationException {
+  private void shouldReturnOKForUserWithPullReplicationCapabilityTest() throws Exception {
     projectOperations
         .allProjectsForUpdate()
         .add(
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientBase.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientBase.java
index cdb238e..25aa2a7 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientBase.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientBase.java
@@ -33,13 +33,10 @@
 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.filter.SyncRefsFilter;
-import java.io.IOException;
 import java.io.InputStreamReader;
-import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import org.apache.http.Header;
-import org.apache.http.client.ClientProtocolException;
 import org.apache.http.client.methods.HttpDelete;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.client.methods.HttpPut;
@@ -150,8 +147,7 @@
   protected abstract void assertAuthentication(HttpRequestBase httpRequest);
 
   @Test
-  public void shouldCallFetchEndpoint()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldCallFetchEndpoint() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -167,8 +163,7 @@
   }
 
   @Test
-  public void shouldByDefaultCallSyncFetchForAllRefs()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldByDefaultCallSyncFetchForAllRefs() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -179,8 +174,7 @@
   }
 
   @Test
-  public void shouldCallAsyncFetchForAllRefs()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldCallAsyncFetchForAllRefs() throws Exception {
 
     when(config.getStringList("replication", null, "syncRefs"))
         .thenReturn(new String[] {"NO_SYNC_REFS"});
@@ -205,8 +199,7 @@
   }
 
   @Test
-  public void shouldCallSyncFetchOnlyForMetaRef()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldCallSyncFetchOnlyForMetaRef() throws Exception {
     String metaRefName = "refs/changes/01/101/meta";
     String expectedMetaRefPayload =
         "{\"label\":\"Replication\", \"ref_name\": \"" + metaRefName + "\", \"async\":false}";
@@ -237,8 +230,7 @@
   }
 
   @Test
-  public void shouldCallFetchEndpointWithPayload()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldCallFetchEndpointWithPayload() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -249,8 +241,7 @@
   }
 
   @Test
-  public void shouldSetContentTypeHeader()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldSetContentTypeHeader() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -262,8 +253,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectEndpoint()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldCallSendObjectEndpoint() throws Exception {
 
     objectUnderTest.callSendObject(
         Project.nameKey("test_repo"),
@@ -285,8 +275,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectEndpointWithPayload()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldCallSendObjectEndpointWithPayload() throws Exception {
 
     objectUnderTest.callSendObject(
         Project.nameKey("test_repo"),
@@ -303,8 +292,7 @@
   }
 
   @Test
-  public void shouldSetContentTypeHeaderForSendObjectCall()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldSetContentTypeHeaderForSendObjectCall() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -364,8 +352,7 @@
   }
 
   @Test
-  public void shouldUseReplicationLabelWhenProvided()
-      throws ClientProtocolException, IOException, URISyntaxException {
+  public void shouldUseReplicationLabelWhenProvided() throws Exception {
     when(config.getString("replication", null, "instanceLabel")).thenReturn(instanceId);
     FetchRestApiClient objectUnderTest =
         new FetchRestApiClient(
@@ -386,7 +373,7 @@
   }
 
   @Test
-  public void shouldCallInitProjectEndpoint() throws IOException, URISyntaxException {
+  public void shouldCallInitProjectEndpoint() throws Exception {
 
     objectUnderTest.initProject(Project.nameKey("test_repo"), new URIish(api));
 
@@ -403,7 +390,7 @@
   }
 
   @Test
-  public void shouldCallDeleteProjectEndpoint() throws IOException, URISyntaxException {
+  public void shouldCallDeleteProjectEndpoint() throws Exception {
 
     objectUnderTest.deleteProject(Project.nameKey("test_repo"), new URIish(api));
 
@@ -420,7 +407,7 @@
   }
 
   @Test
-  public void shouldCallUpdateHEADEndpoint() throws IOException, URISyntaxException {
+  public void shouldCallUpdateHEADEndpoint() throws Exception {
     String newHead = "newHead";
     String projectName = "aProject";
     objectUnderTest.updateHead(Project.nameKey(projectName), newHead, new URIish(api));
@@ -441,7 +428,7 @@
     assertAuthentication(httpPut);
   }
 
-  public String readPayload(HttpPost entity) throws UnsupportedOperationException, IOException {
+  public String readPayload(HttpPost entity) throws Exception {
     ByteBuffer buf = IO.readWholeStream(entity.getEntity().getContent(), 1024);
     return RawParseUtils.decode(buf.array(), buf.arrayOffset(), buf.limit()).trim();
   }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientWithBasicAuthenticationTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientWithBasicAuthenticationTest.java
index 644afce..b4d4e7a 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientWithBasicAuthenticationTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientWithBasicAuthenticationTest.java
@@ -19,11 +19,9 @@
 import static org.mockito.Mockito.*;
 
 import com.googlesource.gerrit.plugins.replication.pull.filter.SyncRefsFilter;
-import java.io.IOException;
 import java.util.Optional;
 import org.apache.http.Header;
 import org.apache.http.HttpHeaders;
-import org.apache.http.client.ClientProtocolException;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.eclipse.jgit.transport.CredentialItem;
 import org.junit.Before;
@@ -36,7 +34,7 @@
 public class FetchRestApiClientWithBasicAuthenticationTest extends FetchRestApiClientBase {
 
   @Before
-  public void setup() throws ClientProtocolException, IOException {
+  public void setup() throws Exception {
     when(bearerTokenProvider.get()).thenReturn(Optional.empty());
     when(credentialProvider.supports(any()))
         .thenAnswer(
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientWithBearerTokenTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientWithBearerTokenTest.java
index 90d71ad..fe50b5c 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientWithBearerTokenTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientWithBearerTokenTest.java
@@ -19,11 +19,9 @@
 import static org.mockito.Mockito.*;
 
 import com.googlesource.gerrit.plugins.replication.pull.filter.SyncRefsFilter;
-import java.io.IOException;
 import java.util.Optional;
 import org.apache.http.Header;
 import org.apache.http.HttpHeaders;
-import org.apache.http.client.ClientProtocolException;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.junit.Before;
 import org.junit.runner.RunWith;
@@ -33,7 +31,7 @@
 public class FetchRestApiClientWithBearerTokenTest extends FetchRestApiClientBase {
 
   @Before
-  public void setup() throws ClientProtocolException, IOException {
+  public void setup() throws Exception {
     when(bearerTokenProvider.get()).thenReturn(Optional.of("some-bearer-token"));
     when(replicationConfig.getConfig()).thenReturn(config);
     when(config.getStringList("replication", null, "syncRefs")).thenReturn(new String[0]);
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/EventsBrokerMessageConsumerTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/EventsBrokerMessageConsumerTest.java
index 8739d4b..0aba20a 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/EventsBrokerMessageConsumerTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/EventsBrokerMessageConsumerTest.java
@@ -24,7 +24,6 @@
 
 import com.gerritforge.gerrit.eventbroker.BrokerApi;
 import com.google.gerrit.extensions.registration.DynamicItem;
-import com.google.gerrit.extensions.restapi.AuthException;
 import com.google.gerrit.server.events.RefUpdatedEvent;
 import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.googlesource.gerrit.plugins.replication.pull.ShutdownState;
@@ -53,29 +52,25 @@
   }
 
   @Test
-  public void shouldRethrowExceptionWhenFetchThrowsAuthException()
-      throws AuthException, PermissionBackendException {
+  public void shouldRethrowExceptionWhenFetchThrowsAuthException() throws Exception {
     doThrow(PermissionBackendException.class).when(eventListener).fetchRefsForEvent(any());
     assertThrows(EventRejectedException.class, () -> objectUnderTest.accept(new RefUpdatedEvent()));
   }
 
   @Test
-  public void shouldRethrowExceptionWhenFetchThrowsPermissionBackendException()
-      throws AuthException, PermissionBackendException {
+  public void shouldRethrowExceptionWhenFetchThrowsPermissionBackendException() throws Exception {
     doThrow(PermissionBackendException.class).when(eventListener).fetchRefsForEvent(any());
     assertThrows(EventRejectedException.class, () -> objectUnderTest.accept(new RefUpdatedEvent()));
   }
 
   @Test
-  public void shouldNotThrowExceptionWhenFetchSucceed()
-      throws AuthException, PermissionBackendException {
+  public void shouldNotThrowExceptionWhenFetchSucceed() throws Exception {
     doNothing().when(eventListener).fetchRefsForEvent(any());
     objectUnderTest.accept(new RefUpdatedEvent());
   }
 
   @Test
-  public void shouldStillAcceptLastEventDuringShutdownAndThenDisconnect()
-      throws AuthException, PermissionBackendException {
+  public void shouldStillAcceptLastEventDuringShutdownAndThenDisconnect() throws Exception {
     doNothing().when(eventListener).fetchRefsForEvent(any());
     when(eventsBrokerDynamicItem.get()).thenReturn(eventsBroker);
 
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListenerTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListenerTest.java
index 8dc0359..552853e 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListenerTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListenerTest.java
@@ -26,14 +26,11 @@
 import com.google.common.collect.Lists;
 import com.google.gerrit.entities.Project;
 import com.google.gerrit.entities.RefNames;
-import com.google.gerrit.extensions.restapi.AuthException;
-import com.google.gerrit.extensions.restapi.RestApiException;
 import com.google.gerrit.server.data.RefUpdateAttribute;
 import com.google.gerrit.server.events.Event;
 import com.google.gerrit.server.events.ProjectCreatedEvent;
 import com.google.gerrit.server.events.RefUpdatedEvent;
 import com.google.gerrit.server.git.WorkQueue;
-import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.googlesource.gerrit.plugins.replication.pull.ApplyObjectsCacheKey;
 import com.googlesource.gerrit.plugins.replication.pull.FetchOne;
 import com.googlesource.gerrit.plugins.replication.pull.Source;
@@ -44,7 +41,6 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.ProjectInitializationAction;
 import com.googlesource.gerrit.plugins.replication.pull.api.PullReplicationApiRequestMetrics;
 import com.googlesource.gerrit.plugins.replication.pull.filter.ExcludedRefsFilter;
-import java.io.IOException;
 import java.util.concurrent.ScheduledExecutorService;
 import org.eclipse.jgit.lib.ObjectId;
 import org.junit.Before;
@@ -151,7 +147,7 @@
   }
 
   @Test
-  public void shouldDeleteRefForRefDeleteEvent() throws IOException, RestApiException {
+  public void shouldDeleteRefForRefDeleteEvent() throws Exception {
     RefUpdatedEvent event = new RefUpdatedEvent();
     RefUpdateAttribute refUpdate = new RefUpdateAttribute();
     refUpdate.refName = TEST_REF_NAME;
@@ -209,8 +205,7 @@
   }
 
   @Test
-  public void shouldCreateProjectForProjectCreatedEvent()
-      throws AuthException, PermissionBackendException {
+  public void shouldCreateProjectForProjectCreatedEvent() throws Exception {
     ProjectCreatedEvent event = new ProjectCreatedEvent();
     event.instanceId = REMOTE_INSTANCE_ID;
     event.projectName = TEST_PROJECT;
@@ -221,8 +216,7 @@
   }
 
   @Test
-  public void shouldNotCreateProjectWhenCreateMissingRepositoriesNotSet()
-      throws AuthException, PermissionBackendException {
+  public void shouldNotCreateProjectWhenCreateMissingRepositoriesNotSet() throws Exception {
     when(source.isCreateMissingRepositories()).thenReturn(false);
 
     ProjectCreatedEvent event = new ProjectCreatedEvent();
@@ -235,8 +229,7 @@
   }
 
   @Test
-  public void shouldNotCreateProjectWhenReplicationNotAllowed()
-      throws AuthException, PermissionBackendException {
+  public void shouldNotCreateProjectWhenReplicationNotAllowed() throws Exception {
     when(source.isCreateMissingRepositories()).thenReturn(false);
 
     ProjectCreatedEvent event = new ProjectCreatedEvent();
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/transport/TransportProviderTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/transport/TransportProviderTest.java
index 0b28c03..9fe4d53 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/transport/TransportProviderTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/transport/TransportProviderTest.java
@@ -23,8 +23,6 @@
 import com.googlesource.gerrit.plugins.replication.CredentialsFactory;
 import com.googlesource.gerrit.plugins.replication.pull.BearerTokenProvider;
 import com.googlesource.gerrit.plugins.replication.pull.SourceConfiguration;
-import java.io.IOException;
-import java.net.URISyntaxException;
 import java.util.Optional;
 import org.eclipse.jgit.lib.Repository;
 import org.eclipse.jgit.lib.StoredConfig;
@@ -65,7 +63,7 @@
   }
 
   @Test
-  public void shouldProvideTransportHttpWithBearerToken() throws URISyntaxException, IOException {
+  public void shouldProvideTransportHttpWithBearerToken() throws Exception {
     when(bearerTokenProvider.get()).thenReturn(Optional.of("some-bearer-token"));
 
     TransportProvider transportProvider =
@@ -80,8 +78,7 @@
   }
 
   @Test
-  public void shouldProvideNativeTransportWhenNoBearerTokenProvided()
-      throws URISyntaxException, IOException {
+  public void shouldProvideNativeTransportWhenNoBearerTokenProvided() throws Exception {
 
     when(bearerTokenProvider.get()).thenReturn(Optional.empty());
 
@@ -96,8 +93,7 @@
   }
 
   @Test
-  public void shouldProvideNativeTransportWhenNoHttpSchemeProvided()
-      throws URISyntaxException, IOException {
+  public void shouldProvideNativeTransportWhenNoHttpSchemeProvided() throws Exception {
     when(bearerTokenProvider.get()).thenReturn(Optional.of("some-bearer-token"));
 
     TransportProvider transportProvider =