Merge branch 'stable-3.8'

* stable-3.8:
  Do not rely on async/wait for synchronous fetch replication
  Cover the replication failure scenario and fix the metrics
  Remove white-box unit tests on synchronous FetchCommand
  Throw Exception from tests

Change-Id: Ie67000a99bcbfd49eea7e21448920ffb91f6397e
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..5d9af1b 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,7 +328,8 @@
     }
 
     repLog.info(
-        "[{}] Replication from {} started for refs [{}] ...",
+        "[{}] {} replication from {} started for refs [{}] ...",
+        replicationType,
         taskIdHex,
         uri,
         String.join(",", getRefs()));
@@ -335,7 +342,8 @@
 
       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",
+            replicationType,
             taskIdHex,
             uri,
             delay,
@@ -379,7 +387,7 @@
       repLog.error(
           String.format("Terminal failure. Cannot replicate [%s] from %s", taskIdHex, uri), e);
     } catch (TransportException e) {
-      if (e instanceof LockFailureException) {
+      if (replicationType == ReplicationType.ASYNC && e instanceof LockFailureException) {
         lockRetryCount++;
         // The LockFailureException message contains both URI and reason
         // for this failure.
@@ -401,7 +409,7 @@
               taskIdHex,
               uri);
         }
-      } else {
+      } else if (replicationType == ReplicationType.ASYNC) {
         if (canceledWhileRunning.get()) {
           logCanceledWhileRunningException(e);
         } else {
@@ -421,7 +429,10 @@
       if (git != null) {
         git.close();
       }
-      pool.notifyFinished(this);
+
+      if (replicationType == ReplicationType.ASYNC) {
+        pool.notifyFinished(this);
+      }
     }
   }
 
@@ -454,7 +465,7 @@
     return fetchRefSpecs;
   }
 
-  private List<RefSpec> getFetchRefSpecs() {
+  public List<RefSpec> getFetchRefSpecs() {
     List<RefSpec> configRefSpecs = config.getFetchRefSpecs();
     if (delta.isEmpty()) {
       return configRefSpecs;
@@ -585,6 +596,6 @@
 
   @Override
   public boolean hasSucceeded() {
-    return succeeded;
+    return succeeded || getFetchRefSpecs().isEmpty();
   }
 }
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 19fca4d..fb8c239 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
@@ -519,6 +519,22 @@
     }
   }
 
+  public Optional<FetchOne> fetchSync(
+      Project.NameKey project,
+      String ref,
+      Optional<PullReplicationApiRequestMetrics> apiRequestMetrics) {
+    if (shouldReplicate(project, ref)
+        && (config.replicatePermissions() || !ref.equals(RefNames.REFS_CONFIG))) {
+
+      FetchOne e = opFactory.create(project, getURI(project), apiRequestMetrics);
+      e.addRef(ref);
+      e.runSync();
+      return Optional.of(e);
+    }
+
+    return Optional.empty();
+  }
+
   void scheduleDeleteProject(String uri, Project.NameKey project) {
     @SuppressWarnings("unused")
     ScheduledFuture<?> ignored =
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..26e2d11 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,14 @@
 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 org.eclipse.jgit.errors.TransportException;
+import org.eclipse.jgit.transport.RefSpec;
 
 public class FetchCommand implements Command {
 
@@ -60,13 +64,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,10 +81,13 @@
       ReplicationType fetchType,
       Optional<PullReplicationApiRequestMetrics> apiRequestMetrics)
       throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
-          TimeoutException {
+          TimeoutException, TransportException {
     ReplicationState state =
-        fetchReplicationStateFactory.create(
-            new FetchResultProcessing.CommandProcessing(this, eventDispatcher.get()));
+        fetchType == ReplicationType.ASYNC
+            ? fetchReplicationStateFactory.create(
+                new FetchResultProcessing.CommandProcessing(this, eventDispatcher.get()))
+            : null;
+
     Optional<Source> source = sources.getByRemoteName(label);
     if (!source.isPresent()) {
       String msg = String.format("Remote configuration section %s not found", label);
@@ -89,13 +96,25 @@
     }
 
     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, fetchType, 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, apiRequestMetrics);
+        if (maybeFetch.map(FetchOne::getFetchRefSpecs).filter(List::isEmpty).isPresent()) {
+          fetchStateLog.error(
+              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,24 @@
     }
 
     try {
-      state.waitForReplication(source.get().getTimeout());
+      if (state != null) {
+        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)
+            .reduce("", (e1, e2) -> e1 + "\n" + e2);
+    return new TransportException(combinedErrorMessage + " trying to fetch " + 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/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..0c33df4 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);
   }
 
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/PullReplicationFanoutConfigBase.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationFanoutConfigBase.java
index 4349cde..0f6754f 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationFanoutConfigBase.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationFanoutConfigBase.java
@@ -22,7 +22,6 @@
 import com.google.gerrit.extensions.api.projects.BranchInput;
 import com.google.gerrit.server.events.ProjectEvent;
 import com.googlesource.gerrit.plugins.replication.AutoReloadConfigDecorator;
-import java.io.IOException;
 import java.nio.file.Files;
 import java.util.Arrays;
 import java.util.List;
@@ -217,26 +216,27 @@
       String remoteName, List<String> replicaSuffixes, Optional<String> project)
       throws IOException {
     setReplicationSource(remoteName);
+}
   }
 
-  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())
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 c601e52..3afc773 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;
@@ -41,15 +42,18 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
+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;
 
@@ -63,6 +67,17 @@
     }
   }
 
+  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
   protected void setReplicationSource(
       String remoteName, List<String> replicaSuffixes, Optional<String> project)
@@ -114,10 +129,12 @@
   }
 
   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
@@ -158,6 +175,32 @@
   }
 
   @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);
+    pullReplicationQueue.onEvent(event);
+    waitUntilReplicationFailed(1);
+
+    try (Repository repo = repoManager.openRepository(project);
+        Repository sourceRepo = repoManager.openRepository(project)) {
+
+      Ref targetBranchRef = getRef(repo, newBranch);
+      assertThat(targetBranchRef).isNull();
+    }
+  }
+
+  @Test
   @UseLocalDisk
   @GerritConfig(name = "gerrit.instanceId", value = TEST_REPLICATION_REMOTE)
   public void shouldReplicateForceUpdatedBranch() throws Exception {
@@ -414,13 +457,23 @@
     }
   }
 
-  private void waitUntilReplicationCompleted(int expected) throws InterruptedException {
-    waitUntil(
-        () ->
-            inMemoryMetrics()
-                .counterValue("tasks/completed", TEST_REPLICATION_REMOTE)
-                .filter(counter -> counter == expected)
-                .isPresent());
+  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 1ef4d35..9cb2dc9 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
@@ -40,7 +40,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;
@@ -102,7 +101,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);
   }
 
@@ -116,8 +115,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);
   }
 
@@ -125,13 +123,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 ed3ba4a..46e2488 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
@@ -50,14 +50,12 @@
 import com.googlesource.gerrit.plugins.replication.ReplicationFileBasedConfig;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.BatchApplyObjectData;
 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;
 import com.googlesource.gerrit.plugins.replication.pull.filter.ApplyObjectsRefsFilter;
 import com.googlesource.gerrit.plugins.replication.pull.filter.ExcludedRefsFilter;
 import java.io.IOException;
-import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.List;
@@ -124,7 +122,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");
@@ -202,7 +200,7 @@
   }
 
   @Test
-  public void shouldCallBatchSendObjectWhenMetaRef() throws IOException {
+  public void shouldCallBatchSendObjectWhenMetaRef() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/meta");
     objectUnderTest.start();
     objectUnderTest.onEvent(event);
@@ -239,7 +237,7 @@
   }
 
   @Test
-  public void shouldIgnoreEventWhenIsNotLocalInstanceId() throws IOException {
+  public void shouldIgnoreEventWhenIsNotLocalInstanceId() throws Exception {
     Event event = generateBatchRefUpdateEvent(TEST_REF_NAME);
     event.instanceId = FOREIGN_INSTANCE_ID;
     objectUnderTest.start();
@@ -249,7 +247,7 @@
   }
 
   @Test
-  public void shouldCallInitProjectWhenProjectIsMissing() throws IOException {
+  public void shouldCallInitProjectWhenProjectIsMissing() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/meta");
     when(batchHttpResult.isSuccessful()).thenReturn(false);
     when(batchHttpResult.isProjectMissing(any())).thenReturn(true);
@@ -262,7 +260,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectReorderingRefsHavingMetaAtTheEnd() throws IOException {
+  public void shouldCallSendObjectReorderingRefsHavingMetaAtTheEnd() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/meta", "refs/changes/01/1/1");
     objectUnderTest.start();
     objectUnderTest.onEvent(event);
@@ -270,7 +268,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectKeepingMetaAtTheEnd() throws IOException {
+  public void shouldCallSendObjectKeepingMetaAtTheEnd() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1", "refs/changes/01/1/meta");
     objectUnderTest.start();
     objectUnderTest.onEvent(event);
@@ -293,7 +291,7 @@
   }
 
   @Test
-  public void shouldCallBatchSendObjectWhenPatchSetRefAndRefUpdateEvent() throws IOException {
+  public void shouldCallBatchSendObjectWhenPatchSetRefAndRefUpdateEvent() throws Exception {
     when(config.getBoolean("event", "stream-events", "enableBatchRefUpdatedEvents", false))
         .thenReturn(false);
 
@@ -321,7 +319,7 @@
   }
 
   @Test
-  public void shouldCallBatchSendObjectWhenPatchSetRef() throws IOException {
+  public void shouldCallBatchSendObjectWhenPatchSetRef() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1");
     objectUnderTest.start();
     objectUnderTest.onEvent(event);
@@ -343,8 +341,7 @@
   }
 
   @Test
-  public void shouldFallbackToCallBatchFetchWhenLargeRef()
-      throws IOException, LargeObjectException, RefUpdateException {
+  public void shouldFallbackToCallBatchFetchWhenLargeRef() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1");
     objectUnderTest.start();
 
@@ -358,7 +355,7 @@
   @Test
   public void
       shouldFallbackToCallBatchFetchWhenParentObjectIsMissingAndRefDoesntMatchApplyObjectsRefsFilter()
-          throws IOException {
+          throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1");
     objectUnderTest.start();
 
@@ -373,7 +370,7 @@
   @Test
   public void
       shouldFallbackToApplyObjectsForEachRefWhenParentObjectIsMissingAndRefMatchesApplyObjectsRefFilter()
-          throws IOException {
+          throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1", "refs/changes/02/1/1");
     objectUnderTest.start();
 
@@ -390,7 +387,7 @@
 
   @Test
   public void shouldFallbackToCallBatchFetchWhenParentObjectNotMissingButApplyObjectFails()
-      throws IOException {
+      throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1");
     objectUnderTest.start();
 
@@ -406,7 +403,7 @@
 
   @Test
   public void shouldFallbackToApplyAllParentObjectsWhenParentObjectIsMissingOnMetaRef()
-      throws IOException {
+      throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/meta");
     objectUnderTest.start();
 
@@ -427,7 +424,7 @@
 
   @Test
   public void shouldFallbackToApplyAllParentObjectsWhenParentObjectIsMissingOnAllowedRefs()
-      throws IOException {
+      throws Exception {
     String refName = "refs/tags/test-tag";
     Event event = generateBatchRefUpdateEvent(refName);
     objectUnderTest.start();
@@ -449,7 +446,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectsIfBatchedRefsNotEnabledAtSource() throws IOException {
+  public void shouldCallSendObjectsIfBatchedRefsNotEnabledAtSource() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1");
     when(source.enableBatchedRefs()).thenReturn(false);
     objectUnderTest.start();
@@ -460,7 +457,7 @@
   }
 
   @Test
-  public void shouldCallFetchIfBatchedRefsNotEnabledAtSource() throws IOException {
+  public void shouldCallFetchIfBatchedRefsNotEnabledAtSource() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1");
     when(source.enableBatchedRefs()).thenReturn(false);
     when(httpResult.isSuccessful()).thenReturn(false);
@@ -474,8 +471,7 @@
   }
 
   @Test
-  public void shouldCallBatchFetchForAllTheRefsInTheBatchIfApplyObjectFails()
-      throws IOException, URISyntaxException {
+  public void shouldCallBatchFetchForAllTheRefsInTheBatchIfApplyObjectFails() throws Exception {
     Event event = generateBatchRefUpdateEvent("refs/changes/01/1/1", "refs/changes/02/1/1");
     when(batchHttpResult.isSuccessful()).thenReturn(false);
     when(batchHttpResult.isParentObjectMissing()).thenReturn(true);
@@ -495,7 +491,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");
@@ -525,7 +521,7 @@
   }
 
   @Test
-  public void shouldSetShutdownStateWhenStopping() throws IOException {
+  public void shouldSetShutdownStateWhenStopping() throws Exception {
     objectUnderTest.stop();
     assertThat(shutdownState.isShuttingDown()).isTrue();
   }
@@ -594,7 +590,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 029db9e..4eda6b0 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
@@ -29,15 +29,11 @@
 import com.google.gerrit.extensions.restapi.ResourceConflictException;
 import com.google.gerrit.extensions.restapi.ResourceNotFoundException;
 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;
@@ -92,14 +88,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());
 
@@ -109,7 +105,7 @@
   }
 
   @Test
-  public void shouldReturnCreatedResponseCodeForBlob() throws RestApiException {
+  public void shouldReturnCreatedResponseCodeForBlob() throws Exception {
     byte[] blobData = "foo".getBytes(StandardCharsets.UTF_8);
     RevisionInput inputParams =
         new RevisionInput(
@@ -192,8 +188,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());
 
@@ -203,8 +198,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 681b695..446902e 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
@@ -44,7 +44,6 @@
 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;
@@ -87,9 +86,7 @@
   private ApplyObjectCommand objectUnderTest;
 
   @Before
-  public void setup()
-      throws MissingParentObjectException, IOException, URISyntaxException,
-          ResourceNotFoundException {
+  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");
@@ -128,9 +125,7 @@
   }
 
   @Test
-  public void shouldInsertIntoApplyObjectsCacheWhenApplyObjectIsSuccessful()
-      throws IOException, RefUpdateException, MissingParentObjectException,
-          ResourceNotFoundException {
+  public void shouldInsertIntoApplyObjectsCacheWhenApplyObjectIsSuccessful() throws Exception {
     RevisionData sampleRevisionData =
         createSampleRevisionData(sampleCommitObjectId, sampleTreeObjectId);
     RevisionData sampleRevisionData2 =
@@ -159,9 +154,7 @@
   }
 
   @Test(expected = RefUpdateException.class)
-  public void shouldNotInsertIntoApplyObjectsCacheWhenApplyObjectIsFailure()
-      throws IOException, RefUpdateException, MissingParentObjectException,
-          ResourceNotFoundException {
+  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 5babf4d..723ad12 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,96 +80,59 @@
   }
 
   @Test
-  public void shouldAuthenticateWhenFetch() throws ServletException, IOException {
+  public void shouldAuthenticateWhenFetch() throws Exception {
     authenticateAndFilter("any-prefix/pull-replication~fetch", NO_QUERY_PARAMETERS);
   }
 
   @Test
-  public void shouldAuthenticateWhenBatchFetch() throws ServletException, IOException {
+  public void shouldAuthenticateWhenBatchFetch() throws Exception {
     authenticateAndFilter("any-prefix/pull-replication~batch-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 shouldAuthenticateWhenBatchApplyObject() throws ServletException, IOException {
+  public void shouldAuthenticateWhenBatchApplyObject() throws Exception {
     authenticateAndFilter("any-prefix/pull-replication~batch-apply-object", 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 {
-    when(httpServletRequest.getRequestURI()).thenReturn("any-prefix/git-upload-pack");
-
-    final BearerAuthenticationFilter filter =
-        new BearerAuthenticationFilter(
-            session,
-            pluginName,
-            pluginUser,
-            threadLocalRequestContextProvider,
-            "some-bearer-token");
-    filter.doFilter(httpServletRequest, httpServletResponse, filterChain);
-
-    verify(httpServletRequest).getHeader("Authorization");
-    verify(filterChain).doFilter(httpServletRequest, httpServletResponse);
-  }
-
-  @Test
-  public void shouldGoNextInChainWhenGitUploadPackWithAuthenticationHeaderDifferentThanBearer()
-      throws ServletException, IOException {
-    when(httpServletRequest.getRequestURI()).thenReturn("any-prefix/git-upload-pack");
-    when(httpServletRequest.getHeader("Authorization")).thenReturn("some-authorization");
-    final BearerAuthenticationFilter filter =
-        new BearerAuthenticationFilter(
-            session,
-            pluginName,
-            pluginUser,
-            threadLocalRequestContextProvider,
-            "some-bearer-token");
-    filter.doFilter(httpServletRequest, httpServletResponse, filterChain);
-
-    verify(httpServletRequest).getHeader("Authorization");
-    verify(filterChain).doFilter(httpServletRequest, httpServletResponse);
-  }
-
-  @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"));
@@ -191,7 +152,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");
 
@@ -210,7 +171,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 =
@@ -228,7 +189,7 @@
   }
 
   @Test
-  public void shouldGoNextInChainWhenUriDoesNotMatch() throws ServletException, IOException {
+  public void shouldGoNextInChainWhenUriDoesNotMatch() throws Exception {
     when(httpServletRequest.getRequestURI()).thenReturn("any-url");
 
     final BearerAuthenticationFilter filter =
@@ -245,8 +206,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..d420ac7 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
@@ -16,9 +16,7 @@
 
 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.times;
@@ -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,7 +63,7 @@
   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";
@@ -83,19 +77,7 @@
   }
 
   @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))
@@ -103,67 +85,10 @@
   }
 
   @Test
-  public void shouldMarkAllFetchTasksScheduled()
-      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());
-    verify(state, times(1)).markAllFetchTasksScheduled();
-  }
-
-  @Test
   public void shouldUpdateStateWhenRemoteConfigNameIsMissing() {
     assertThrows(
         RemoteConfigurationMissingException.class,
         () -> 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));
+    verify(fetchStateLog, times(1)).error(anyString(), eq(null));
   }
 }
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 82b863e..dd8ec7d 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
@@ -34,9 +34,7 @@
 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.ArrayList;
 import java.util.Collections;
@@ -153,7 +151,7 @@
   protected abstract void assertAuthentication(HttpRequestBase httpRequest);
 
   @Test
-  public void shouldCallFetchEndpoint() throws IOException, URISyntaxException {
+  public void shouldCallFetchEndpoint() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -169,7 +167,7 @@
   }
 
   @Test
-  public void shouldCallBatchFetchEndpoint() throws IOException, URISyntaxException {
+  public void shouldCallBatchFetchEndpoint() throws Exception {
 
     objectUnderTest.callBatchFetch(
         Project.nameKey("test_repo"),
@@ -188,7 +186,7 @@
   }
 
   @Test
-  public void shouldByDefaultCallSyncFetchForAllRefs() throws IOException, URISyntaxException {
+  public void shouldByDefaultCallSyncFetchForAllRefs() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -199,7 +197,7 @@
   }
 
   @Test
-  public void shouldCallAsyncFetchForAllRefs() throws IOException, URISyntaxException {
+  public void shouldCallAsyncFetchForAllRefs() throws Exception {
 
     when(config.getStringList("replication", null, "syncRefs"))
         .thenReturn(new String[] {"NO_SYNC_REFS"});
@@ -224,7 +222,7 @@
   }
 
   @Test
-  public void shouldCallAsyncBatchFetchForAllRefs() throws IOException, URISyntaxException {
+  public void shouldCallAsyncBatchFetchForAllRefs() throws Exception {
 
     when(config.getStringList("replication", null, "syncRefs"))
         .thenReturn(new String[] {"NO_SYNC_REFS"});
@@ -259,7 +257,7 @@
   }
 
   @Test
-  public void shouldCallSyncFetchOnlyForMetaRef() throws IOException, URISyntaxException {
+  public void shouldCallSyncFetchOnlyForMetaRef() throws Exception {
     String metaRefName = "refs/changes/01/101/meta";
     String expectedMetaRefPayload =
         "{\"label\":\"Replication\", \"ref_name\": \"" + metaRefName + "\", \"async\":false}";
@@ -290,7 +288,7 @@
   }
 
   @Test
-  public void shouldCallSyncBatchFetchOnlyForMetaRef() throws IOException, URISyntaxException {
+  public void shouldCallSyncBatchFetchOnlyForMetaRef() throws Exception {
     String metaRefName = "refs/changes/01/101/meta";
     String expectedMetaRefPayload =
         "[{\"label\":\"Replication\", \"ref_name\": \"" + metaRefName + "\", \"async\":false}]";
@@ -317,8 +315,7 @@
   }
 
   @Test
-  public void shouldCallFetchEndpointWithPayload() throws IOException, URISyntaxException {
-
+  public void shouldCallFetchEndpointWithPayload() throws Exception {
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
     verify(httpClient, times(1)).execute(httpPostCaptor.capture(), any());
@@ -415,7 +412,7 @@
   }
 
   @Test
-  public void shouldSetContentTypeHeader() throws IOException, URISyntaxException {
+  public void shouldSetContentTypeHeader() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -427,7 +424,7 @@
   }
 
   @Test
-  public void shouldSetContentTypeHeaderInBatchFetch() throws IOException, URISyntaxException {
+  public void shouldSetContentTypeHeaderInBatchFetch() throws Exception {
 
     objectUnderTest.callBatchFetch(Project.nameKey("test_repo"), List.of(refName), new URIish(api));
 
@@ -439,8 +436,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectEndpoint() throws IOException, URISyntaxException {
-
+  public void shouldCallSendObjectEndpoint() throws Exception {
     objectUnderTest.callSendObject(
         Project.nameKey("test_repo"),
         refName,
@@ -461,7 +457,7 @@
   }
 
   @Test
-  public void shouldCallSendObjectEndpointWithPayload() throws IOException, URISyntaxException {
+  public void shouldCallSendObjectEndpointWithPayload() throws Exception {
 
     objectUnderTest.callSendObject(
         Project.nameKey("test_repo"),
@@ -478,7 +474,7 @@
   }
 
   @Test
-  public void shouldSetContentTypeHeaderForSendObjectCall() throws IOException, URISyntaxException {
+  public void shouldSetContentTypeHeaderForSendObjectCall() throws Exception {
 
     objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
 
@@ -538,7 +534,7 @@
   }
 
   @Test
-  public void shouldUseReplicationLabelWhenProvided() throws IOException, URISyntaxException {
+  public void shouldUseReplicationLabelWhenProvided() throws Exception {
     when(config.getString("replication", null, "instanceLabel")).thenReturn(instanceId);
     FetchRestApiClient objectUnderTest =
         new FetchRestApiClient(
@@ -559,7 +555,7 @@
   }
 
   @Test
-  public void shouldCallInitProjectEndpoint() throws IOException, URISyntaxException {
+  public void shouldCallInitProjectEndpoint() throws Exception {
 
     objectUnderTest.initProject(Project.nameKey("test_repo"), new URIish(api));
 
@@ -576,7 +572,7 @@
   }
 
   @Test
-  public void shouldCallDeleteProjectEndpoint() throws IOException, URISyntaxException {
+  public void shouldCallDeleteProjectEndpoint() throws Exception {
 
     objectUnderTest.deleteProject(Project.nameKey("test_repo"), new URIish(api));
 
@@ -593,7 +589,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));
@@ -615,7 +611,7 @@
   }
 
   @Test
-  public void shouldCallBatchSendObjectEndpoint() throws IOException, URISyntaxException {
+  public void shouldCallBatchSendObjectEndpoint() throws Exception {
 
     List<BatchApplyObjectData> batchApplyObjects = new ArrayList<>();
     batchApplyObjects.add(
@@ -637,8 +633,7 @@
   }
 
   @Test
-  public void shouldCallBatchApplyObjectEndpointWithAListOfRefsInPayload()
-      throws IOException, URISyntaxException {
+  public void shouldCallBatchApplyObjectEndpointWithAListOfRefsInPayload() throws Exception {
     List<BatchApplyObjectData> batchApplyObjects = new ArrayList<>();
     RevisionData revisionA = createSampleRevisionData("a");
     RevisionData revisionB = createSampleRevisionData("b");
@@ -680,8 +675,7 @@
   }
 
   @Test
-  public void shouldCallBatchApplyObjectEndpointWithNoRevisionDataForDeletes()
-      throws IOException, URISyntaxException {
+  public void shouldCallBatchApplyObjectEndpointWithNoRevisionDataForDeletes() throws Exception {
     List<BatchApplyObjectData> batchApplyObjects = new ArrayList<>();
     batchApplyObjects.add(BatchApplyObjectData.create(refName, Optional.empty(), true));
 
@@ -703,7 +697,7 @@
 
   @Test(expected = IllegalArgumentException.class)
   public void shouldThrowExceptionIfDeleteFlagIsSetButRevisionDataIsPresentForBatchSendEndpoint()
-      throws IOException, URISyntaxException {
+      throws Exception {
     List<BatchApplyObjectData> batchApplyObjects = new ArrayList<>();
     batchApplyObjects.add(
         BatchApplyObjectData.create(refName, Optional.of(createSampleRevisionData()), true));
@@ -712,7 +706,7 @@
         Project.nameKey("test_repo"), batchApplyObjects, eventCreatedOn, new URIish(api));
   }
 
-  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 2b03d3f..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,7 +19,6 @@
 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;
@@ -35,7 +34,7 @@
 public class FetchRestApiClientWithBasicAuthenticationTest extends FetchRestApiClientBase {
 
   @Before
-  public void setup() throws 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 a79f4b8..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,7 +19,6 @@
 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;
@@ -32,7 +31,7 @@
 public class FetchRestApiClientWithBearerTokenTest extends FetchRestApiClientBase {
 
   @Before
-  public void setup() throws 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 =