Allow asynchronous fetch calls

To improve user experience add functionality to decide for which refs
git fetch operation will be called asynchronously.

Also only git fetch operation should be executed asynchronously, sending
ref content as a payload should be synchronous.

Bug: Issue 13340
Change-Id: Iba08f1cc140f4d88016e6850abc51cd658c432e5
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueue.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueue.java
index 6c8ada0..cdba012 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueue.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/ReplicationQueue.java
@@ -32,6 +32,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
 import com.googlesource.gerrit.plugins.replication.pull.client.FetchRestApiClient;
 import com.googlesource.gerrit.plugins.replication.pull.client.HttpResult;
+import com.googlesource.gerrit.plugins.replication.pull.filter.ExcludedRefsFilter;
 import java.io.IOException;
 import java.net.URISyntaxException;
 import java.util.HashSet;
@@ -65,7 +66,7 @@
   private final Queue<ReferenceUpdatedEvent> beforeStartupEventsQueue;
   private FetchRestApiClient.Factory fetchClientFactory;
   private Integer fetchCallsTimeout;
-  private RefsFilter refsFilter;
+  private ExcludedRefsFilter refsFilter;
   private RevisionReader revisionReader;
 
   @Inject
@@ -75,7 +76,7 @@
       DynamicItem<EventDispatcher> dis,
       ReplicationStateListeners sl,
       FetchRestApiClient.Factory fetchClientFactory,
-      RefsFilter refsFilter,
+      ExcludedRefsFilter refsFilter,
       RevisionReader revReader) {
     workQueue = wq;
     dispatcher = dis;
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectAction.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectAction.java
index 678ff73..77c54fa 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectAction.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectAction.java
@@ -14,12 +14,7 @@
 
 package com.googlesource.gerrit.plugins.replication.pull.api;
 
-import static com.google.common.base.Preconditions.checkState;
-
 import com.google.common.base.Strings;
-import com.google.common.flogger.FluentLogger;
-import com.google.gerrit.entities.Project;
-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;
@@ -27,9 +22,6 @@
 import com.google.gerrit.extensions.restapi.RestApiException;
 import com.google.gerrit.extensions.restapi.RestModifyView;
 import com.google.gerrit.extensions.restapi.UnprocessableEntityException;
-import com.google.gerrit.server.config.UrlFormatter;
-import com.google.gerrit.server.git.WorkQueue;
-import com.google.gerrit.server.ioutil.HexFormat;
 import com.google.gerrit.server.project.ProjectResource;
 import com.google.inject.Inject;
 import com.googlesource.gerrit.plugins.replication.pull.api.data.RevisionInput;
@@ -37,24 +29,15 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
 import java.io.IOException;
 import java.util.Objects;
-import java.util.Optional;
 
 public class ApplyObjectAction implements RestModifyView<ProjectResource, RevisionInput> {
 
   private final ApplyObjectCommand command;
-  private final WorkQueue workQueue;
-  private final DynamicItem<UrlFormatter> urlFormatter;
   private final FetchPreconditions preConditions;
 
   @Inject
-  public ApplyObjectAction(
-      ApplyObjectCommand command,
-      WorkQueue workQueue,
-      DynamicItem<UrlFormatter> urlFormatter,
-      FetchPreconditions preConditions) {
+  public ApplyObjectAction(ApplyObjectCommand command, FetchPreconditions preConditions) {
     this.command = command;
-    this.workQueue = workQueue;
-    this.urlFormatter = urlFormatter;
     this.preConditions = preConditions;
   }
 
@@ -89,10 +72,9 @@
         throw new BadRequestException("Ref-update tree object cannot be null");
       }
 
-      if (input.isAsync()) {
-        return applyAsync(resource.getNameKey(), input);
-      }
-      return applySync(resource.getNameKey(), input);
+      command.applyObject(
+          resource.getNameKey(), input.getRefName(), input.getRevisionData(), input.getLabel());
+      return Response.created(input);
     } catch (MissingParentObjectException e) {
       throw new ResourceConflictException(e.getMessage(), e);
     } catch (NumberFormatException | IOException e) {
@@ -101,51 +83,4 @@
       throw new UnprocessableEntityException(e.getMessage());
     }
   }
-
-  private Response<?> applySync(Project.NameKey project, RevisionInput input)
-      throws NumberFormatException, IOException, RefUpdateException, MissingParentObjectException {
-    command.applyObject(project, input.getRefName(), input.getRevisionData(), input.getLabel());
-    return Response.created(input);
-  }
-
-  private Response.Accepted applyAsync(Project.NameKey project, RevisionInput input) {
-    @SuppressWarnings("unchecked")
-    WorkQueue.Task<Void> task =
-        (WorkQueue.Task<Void>)
-            workQueue.getDefaultQueue().submit(new ApplyObjectJob(command, project, input));
-    Optional<String> url =
-        urlFormatter
-            .get()
-            .getRestUrl("a/config/server/tasks/" + HexFormat.fromInt(task.getTaskId()));
-    // We're in a HTTP handler, so must be present.
-    checkState(url.isPresent());
-    return Response.accepted(url.get());
-  }
-
-  private static class ApplyObjectJob implements Runnable {
-    private static final FluentLogger log = FluentLogger.forEnclosingClass();
-
-    private ApplyObjectCommand command;
-    private Project.NameKey project;
-    private RevisionInput input;
-
-    public ApplyObjectJob(
-        ApplyObjectCommand command, Project.NameKey project, RevisionInput input) {
-      this.command = command;
-      this.project = project;
-      this.input = input;
-    }
-
-    @Override
-    public void run() {
-      try {
-        command.applyObject(project, input.getRefName(), input.getRevisionData(), input.getLabel());
-      } catch (IOException | RefUpdateException | MissingParentObjectException e) {
-        log.atSevere().withCause(e).log(
-            "Exception during the applyObject call for project {} and ref name {}",
-            project.get(),
-            input.getRefName());
-      }
-    }
-  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionInput.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionInput.java
index 160a720..bc3e218 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionInput.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/data/RevisionInput.java
@@ -21,17 +21,10 @@
 
   private RevisionData revisionData;
 
-  private boolean async;
-
   public RevisionInput(String label, String refName, RevisionData revisionData) {
-    this(label, refName, revisionData, false);
-  }
-
-  public RevisionInput(String label, String refName, RevisionData revisionData, boolean async) {
     this.label = label;
     this.refName = refName;
     this.revisionData = revisionData;
-    this.async = async;
   }
 
   public String getLabel() {
@@ -45,8 +38,4 @@
   public RevisionData getRevisionData() {
     return revisionData;
   }
-
-  public boolean isAsync() {
-    return async;
-  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClient.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClient.java
index eed1afb..7b876df 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClient.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClient.java
@@ -32,6 +32,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.Source;
 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.filter.SyncRefsFilter;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Optional;
@@ -67,18 +68,21 @@
   private final Source source;
   private final String instanceLabel;
   private final String pluginName;
+  private final SyncRefsFilter syncRefsFilter;
 
   @Inject
   FetchRestApiClient(
       CredentialsFactory credentials,
       SourceHttpClient.Factory httpClientFactory,
       ReplicationConfig replicationConfig,
+      SyncRefsFilter syncRefsFilter,
       @PluginName String pluginName,
       @Assisted Source source) {
     this.credentials = credentials;
     this.httpClientFactory = httpClientFactory;
     this.source = source;
     this.pluginName = pluginName;
+    this.syncRefsFilter = syncRefsFilter;
     this.instanceLabel =
         Strings.nullToEmpty(
                 replicationConfig.getConfig().getString("replication", null, "instanceLabel"))
@@ -93,11 +97,13 @@
         String.format(
             "%s/a/projects/%s/pull-replication~fetch",
             targetUri.toString(), Url.encode(project.get()));
-
+    Boolean callAsync = !syncRefsFilter.match(refName);
     HttpPost post = new HttpPost(url);
     post.setEntity(
         new StringEntity(
-            String.format("{\"label\":\"%s\", \"ref_name\": \"%s\"}", instanceLabel, refName),
+            String.format(
+                "{\"label\":\"%s\", \"ref_name\": \"%s\", \"async\":%s}",
+                instanceLabel, refName, callAsync),
             StandardCharsets.UTF_8));
     post.addHeader(new BasicHeader("Content-Type", "application/json"));
     return httpClientFactory.create(source).execute(post, this, getContext(targetUri));
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/ExcludedRefsFilter.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/ExcludedRefsFilter.java
new file mode 100644
index 0000000..f2e4ab3
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/ExcludedRefsFilter.java
@@ -0,0 +1,51 @@
+// 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.filter;
+
+import com.google.common.collect.ImmutableList;
+import com.google.gerrit.entities.RefNames;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.googlesource.gerrit.plugins.replication.ReplicationConfig;
+import java.util.List;
+import org.eclipse.jgit.lib.Config;
+
+@Singleton
+public class ExcludedRefsFilter extends RefsFilter {
+  @Inject
+  public ExcludedRefsFilter(ReplicationConfig replicationConfig) {
+    super(replicationConfig);
+  }
+
+  @Override
+  protected List<String> getRefNamePatterns(Config cfg) {
+    return ImmutableList.<String>builder()
+        .addAll(getDefaultExcludeRefPatterns())
+        .addAll(ImmutableList.copyOf(cfg.getStringList("replication", null, "excludeRefs")))
+        .build();
+  }
+
+  private List<String> getDefaultExcludeRefPatterns() {
+    return ImmutableList.of(
+        RefNames.REFS_USERS + "*",
+        RefNames.REFS_CONFIG,
+        RefNames.REFS_SEQUENCES + "*",
+        RefNames.REFS_EXTERNAL_IDS,
+        RefNames.REFS_GROUPS + "*",
+        RefNames.REFS_GROUPNAMES,
+        RefNames.REFS_CACHE_AUTOMERGE + "*",
+        RefNames.REFS_STARRED_CHANGES + "*");
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/RefsFilter.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/RefsFilter.java
similarity index 70%
rename from src/main/java/com/googlesource/gerrit/plugins/replication/pull/RefsFilter.java
rename to src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/RefsFilter.java
index de6c97d..0bd7564 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/RefsFilter.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/RefsFilter.java
@@ -12,20 +12,15 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package com.googlesource.gerrit.plugins.replication.pull;
+package com.googlesource.gerrit.plugins.replication.pull.filter;
 
 import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
 import com.google.gerrit.common.data.AccessSection;
-import com.google.gerrit.entities.RefNames;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
 import com.googlesource.gerrit.plugins.replication.ReplicationConfig;
 import java.util.List;
 import org.eclipse.jgit.lib.Config;
 
-@Singleton
-public class RefsFilter {
+public abstract class RefsFilter {
   public enum PatternType {
     REGEX,
     WILDCARD,
@@ -44,7 +39,6 @@
 
   private final List<String> refsPatterns;
 
-  @Inject
   public RefsFilter(ReplicationConfig replicationConfig) {
     refsPatterns = getRefNamePatterns(replicationConfig.getConfig());
   }
@@ -66,12 +60,7 @@
     return false;
   }
 
-  private List<String> getRefNamePatterns(Config cfg) {
-    return ImmutableList.<String>builder()
-        .addAll(getDefaultExcludeRefPatterns())
-        .addAll(ImmutableList.copyOf(cfg.getStringList("replication", null, "excludeRefs")))
-        .build();
-  }
+  protected abstract List<String> getRefNamePatterns(Config cfg);
 
   private boolean matchesPattern(String refName, String pattern) {
     boolean match = false;
@@ -87,16 +76,4 @@
     }
     return match;
   }
-
-  private List<String> getDefaultExcludeRefPatterns() {
-    return ImmutableList.of(
-        RefNames.REFS_USERS + "*",
-        RefNames.REFS_CONFIG,
-        RefNames.REFS_SEQUENCES + "*",
-        RefNames.REFS_EXTERNAL_IDS,
-        RefNames.REFS_GROUPS + "*",
-        RefNames.REFS_GROUPNAMES,
-        RefNames.REFS_CACHE_AUTOMERGE + "*",
-        RefNames.REFS_STARRED_CHANGES + "*");
-  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/SyncRefsFilter.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/SyncRefsFilter.java
new file mode 100644
index 0000000..a069935
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/filter/SyncRefsFilter.java
@@ -0,0 +1,35 @@
+// 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.filter;
+
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.googlesource.gerrit.plugins.replication.ReplicationConfig;
+import java.util.List;
+import org.eclipse.jgit.lib.Config;
+
+@Singleton
+public class SyncRefsFilter extends RefsFilter {
+  @Inject
+  public SyncRefsFilter(ReplicationConfig replicationConfig) {
+    super(replicationConfig);
+  }
+
+  @Override
+  protected List<String> getRefNamePatterns(Config cfg) {
+    return ImmutableList.copyOf(cfg.getStringList("replication", null, "syncRefs"));
+  }
+}
diff --git a/src/main/resources/Documentation/config.md b/src/main/resources/Documentation/config.md
index c3e5ad4..affd094 100644
--- a/src/main/resources/Documentation/config.md
+++ b/src/main/resources/Documentation/config.md
@@ -181,6 +181,30 @@
 
     By default, all other refs are included.
 
+replication.syncRefs
+:   Specify for which refs git fetch calls should be executed synchronously.
+    It can be provided more than once, and supports three formats: regular expressions,
+    wildcard matching, and single ref matching. All three formats match are case-sensitive.
+
+    Values starting with a caret `^` are treated as regular
+    expressions. For the regular expressions details please follow
+    official [java documentation](https://docs.oracle.com/javase/tutorial/essential/regex/).
+
+    Please note that regular expressions could also be used
+    with inverse match.
+
+    Values that are not regular expressions and end in `*` are
+    treated as wildcard matches. Wildcards match refs whose
+    name agrees from the beginning until the trailing `*`. So
+    `foo/b*` would match the refs `foo/b`, `foo/bar`, and
+    `foo/baz`, but neither `foobar`, nor `bar/foo/baz`.
+
+    Values that are neither regular expressions nor wildcards are
+    treated as single ref matches. So `foo/bar` matches only
+    the ref `foo/bar`, but no other refs.
+
+    By default, set to '*' (all refs are replicated synchronously).
+
 replication.maxApiPayloadSize
 :	Maximum size in bytes of the ref to be sent as a REST Api call
 	payload. For refs larger than threshold git fetch operation
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 44b69db..327c6ba 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
@@ -38,6 +38,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
 import com.googlesource.gerrit.plugins.replication.pull.client.FetchRestApiClient;
 import com.googlesource.gerrit.plugins.replication.pull.client.HttpResult;
+import com.googlesource.gerrit.plugins.replication.pull.filter.ExcludedRefsFilter;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.util.Optional;
@@ -69,7 +70,7 @@
   @Mock RevisionData revisionData;
   @Mock HttpResult httpResult;
 
-  private RefsFilter refsFilter;
+  private ExcludedRefsFilter refsFilter;
   private ReplicationQueue objectUnderTest;
   private SitePaths sitePaths;
   private Path pluginDataPath;
@@ -80,7 +81,7 @@
     sitePaths = new SitePaths(sitePath);
     Path pluginDataPath = createTempPath("data");
     ReplicationConfig replicationConfig = new ReplicationFileBasedConfig(sitePaths, pluginDataPath);
-    refsFilter = new RefsFilter(replicationConfig);
+    refsFilter = new ExcludedRefsFilter(replicationConfig);
     when(source.getConnectionTimeout()).thenReturn(CONNECTION_TIMEOUT);
     when(source.wouldFetchProject(any())).thenReturn(true);
     when(source.wouldFetchRef(anyString())).thenReturn(true);
@@ -198,7 +199,7 @@
     fileConfig.setString("replication", null, "excludeRefs", "refs/multi-site/version");
     fileConfig.save();
     ReplicationConfig replicationConfig = new ReplicationFileBasedConfig(sitePaths, pluginDataPath);
-    refsFilter = new RefsFilter(replicationConfig);
+    refsFilter = new ExcludedRefsFilter(replicationConfig);
 
     objectUnderTest =
         new ReplicationQueue(wq, rd, dis, sl, fetchClientFactory, refsFilter, revReader);
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectActionIT.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectActionIT.java
new file mode 100644
index 0000000..75c31de
--- /dev/null
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/ApplyObjectActionIT.java
@@ -0,0 +1,272 @@
+// 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;
+
+import static com.google.common.truth.Truth.assertThat;
+import static java.util.stream.Collectors.toList;
+
+import com.google.gerrit.acceptance.LightweightPluginDaemonTest;
+import com.google.gerrit.acceptance.PushOneCommit.Result;
+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.Project;
+import com.google.gerrit.entities.RefNames;
+import com.google.gerrit.extensions.restapi.Url;
+import com.google.gerrit.server.config.SitePaths;
+import com.google.inject.Inject;
+import com.googlesource.gerrit.plugins.replication.CredentialsFactory;
+import com.googlesource.gerrit.plugins.replication.pull.RevisionReader;
+import com.googlesource.gerrit.plugins.replication.pull.Source;
+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.client.SourceHttpClient;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.List;
+import java.util.Optional;
+import org.apache.http.HttpResponse;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.ClientProtocolException;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.client.ResponseHandler;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.protocol.HttpClientContext;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.message.BasicHeader;
+import org.eclipse.jgit.storage.file.FileBasedConfig;
+import org.eclipse.jgit.util.FS;
+import org.junit.Test;
+
+@SkipProjectClone
+@UseLocalDisk
+@TestPlugin(
+    name = "pull-replication",
+    sysModule = "com.googlesource.gerrit.plugins.replication.pull.PullReplicationModule")
+public class ApplyObjectActionIT extends LightweightPluginDaemonTest {
+  private static final Optional<String> ALL_PROJECTS = Optional.empty();
+  private static final int TEST_REPLICATION_DELAY = 60;
+  private static final String TEST_REPLICATION_SUFFIX = "suffix1";
+  private static final String TEST_REPLICATION_REMOTE = "remote1";
+
+  private Path gitPath;
+  private FileBasedConfig config;
+  private FileBasedConfig secureConfig;
+  private RevisionReader revisionReader;
+
+  @Inject private SitePaths sitePaths;
+  @Inject private ProjectOperations projectOperations;
+  CredentialsFactory credentials;
+  Source source;
+  SourceHttpClient.Factory httpClientFactory;
+  String url;
+
+  @Override
+  public void setUpTestPlugin() throws Exception {
+    gitPath = sitePaths.site_path.resolve("git");
+
+    config =
+        new FileBasedConfig(sitePaths.etc_dir.resolve("replication.config").toFile(), FS.DETECTED);
+    setReplicationSource(
+        TEST_REPLICATION_REMOTE,
+        TEST_REPLICATION_SUFFIX,
+        ALL_PROJECTS); // Simulates a full replication.config initialization
+    config.save();
+
+    secureConfig =
+        new FileBasedConfig(sitePaths.etc_dir.resolve("secure.config").toFile(), FS.DETECTED);
+    setReplicationCredentials(TEST_REPLICATION_REMOTE, admin.username(), admin.httpPassword());
+    secureConfig.save();
+
+    super.setUpTestPlugin();
+
+    httpClientFactory = plugin.getSysInjector().getInstance(SourceHttpClient.Factory.class);
+    credentials = plugin.getSysInjector().getInstance(CredentialsFactory.class);
+    revisionReader = plugin.getSysInjector().getInstance(RevisionReader.class);
+    source = plugin.getSysInjector().getInstance(SourcesCollection.class).getAll().get(0);
+
+    url =
+        String.format(
+            "%s/a/projects/%s/pull-replication~apply-object",
+            adminRestSession.url(), Url.encode(project.get()));
+  }
+
+  @Test
+  public void shouldAcceptPayloadWithAsyncField() throws Exception {
+    String payloadWithAsyncFieldTemplate =
+        "{\"label\":\""
+            + TEST_REPLICATION_REMOTE
+            + "\",\"ref_name\":\"%s\",\"revision_data\":{\"commit_object\":{\"type\":1,\"content\":\"%s\"},\"tree_object\":{\"type\":2,\"content\":\"%s\"},\"blobs\":[]}, \"async\":true}";
+
+    String refName = createRef();
+    Optional<RevisionData> revisionDataOption = createRevisionData(refName);
+    assertThat(revisionDataOption.isPresent()).isTrue();
+
+    RevisionData revisionData = revisionDataOption.get();
+    String sendObjectPayload = createPayload(payloadWithAsyncFieldTemplate, refName, revisionData);
+
+    HttpPost post = createRequest(sendObjectPayload);
+    httpClientFactory.create(source).execute(post, assertHttpResponseCode(201), getContext());
+  }
+
+  @Test
+  public void shouldAcceptPayloadWithoutAsyncField() throws Exception {
+    String payloadWithoutAsyncFieldTemplate =
+        "{\"label\":\""
+            + TEST_REPLICATION_REMOTE
+            + "\",\"ref_name\":\"%s\",\"revision_data\":{\"commit_object\":{\"type\":1,\"content\":\"%s\"},\"tree_object\":{\"type\":2,\"content\":\"%s\"},\"blobs\":[]}}";
+
+    String refName = createRef();
+    Optional<RevisionData> revisionDataOption = createRevisionData(refName);
+    assertThat(revisionDataOption.isPresent()).isTrue();
+
+    RevisionData revisionData = revisionDataOption.get();
+    String sendObjectPayload =
+        createPayload(payloadWithoutAsyncFieldTemplate, refName, revisionData);
+
+    HttpPost post = createRequest(sendObjectPayload);
+    httpClientFactory.create(source).execute(post, assertHttpResponseCode(201), getContext());
+  }
+
+  @Test
+  public void shouldReturnBadRequestCodeWhenMandatoryFieldLabelIsMissing() throws Exception {
+    String payloadWithoutLabelFieldTemplate =
+        "{\"ref_name\":\"%s\",\"revision_data\":{\"commit_object\":{\"type\":1,\"content\":\"%s\"},\"tree_object\":{\"type\":2,\"content\":\"%s\"},\"blobs\":[]}, \"async\":true}";
+
+    String refName = createRef();
+    Optional<RevisionData> revisionDataOption = createRevisionData(refName);
+    assertThat(revisionDataOption.isPresent()).isTrue();
+
+    RevisionData revisionData = revisionDataOption.get();
+    String sendObjectPayload =
+        createPayload(payloadWithoutLabelFieldTemplate, refName, revisionData);
+
+    HttpPost post = createRequest(sendObjectPayload);
+    httpClientFactory.create(source).execute(post, assertHttpResponseCode(400), getContext());
+  }
+
+  @Test
+  public void shouldReturnBadRequestCodeWhenPayloadIsNotAProperJSON() throws Exception {
+    String wrongPayloadTemplate =
+        "{\"label\":\""
+            + TEST_REPLICATION_REMOTE
+            + "\",\"ref_name\":\"%s\",\"revision_data\":{\"commit_object\":{\"type\":1,\"content\":\"%s\"},\"tree_object\":{\"type\":2,\"content\":\"%s\"},\"blobs\":[]}, \"async\":true,}";
+
+    String refName = createRef();
+    Optional<RevisionData> revisionDataOption = createRevisionData(refName);
+    assertThat(revisionDataOption.isPresent()).isTrue();
+
+    RevisionData revisionData = revisionDataOption.get();
+    String sendObjectPayload = createPayload(wrongPayloadTemplate, refName, revisionData);
+
+    HttpPost post = createRequest(sendObjectPayload);
+    httpClientFactory.create(source).execute(post, assertHttpResponseCode(400), getContext());
+  }
+
+  private String createPayload(
+      String wrongPayloadTemplate, String refName, RevisionData revisionData) {
+    String sendObjectPayload =
+        String.format(
+            wrongPayloadTemplate,
+            refName,
+            encode(revisionData.getCommitObject().getContent()),
+            encode(revisionData.getTreeObject().getContent()));
+    return sendObjectPayload;
+  }
+
+  private HttpPost createRequest(String sendObjectPayload) {
+    HttpPost post = new HttpPost(url);
+    post.setEntity(new StringEntity(sendObjectPayload, StandardCharsets.UTF_8));
+    post.addHeader(new BasicHeader("Content-Type", "application/json"));
+    return post;
+  }
+
+  private String createRef() throws Exception {
+    testRepo = cloneProject(createTestProject(project + TEST_REPLICATION_SUFFIX));
+
+    Result pushResult = createChange("topic", "test.txt", "test_content");
+    return RefNames.changeMetaRef(pushResult.getChange().getId());
+  }
+
+  private Optional<RevisionData> createRevisionData(String refName) throws Exception {
+    return revisionReader.read(Project.nameKey(project + TEST_REPLICATION_SUFFIX), refName);
+  }
+
+  private Object encode(byte[] content) {
+    return Base64.getEncoder().encodeToString(content);
+  }
+
+  public ResponseHandler<Object> assertHttpResponseCode(int responseCode) {
+    return new ResponseHandler<Object>() {
+
+      @Override
+      public Object handleResponse(HttpResponse response)
+          throws ClientProtocolException, IOException {
+        assertThat(response.getStatusLine().getStatusCode()).isEqualTo(responseCode);
+        return null;
+      }
+    };
+  }
+
+  private HttpClientContext getContext() {
+    HttpClientContext ctx = HttpClientContext.create();
+    CredentialsProvider adapted = new BasicCredentialsProvider();
+    adapted.setCredentials(
+        AuthScope.ANY, new UsernamePasswordCredentials(admin.username(), admin.httpPassword()));
+    ctx.setCredentialsProvider(adapted);
+    return ctx;
+  }
+
+  private Project.NameKey createTestProject(String name) throws Exception {
+    return projectOperations.newProject().name(name).parent(project).create();
+  }
+
+  private void setReplicationSource(
+      String remoteName, String replicaSuffix, Optional<String> project) throws IOException {
+    setReplicationSource(remoteName, Arrays.asList(replicaSuffix), project);
+  }
+
+  private void setReplicationSource(
+      String remoteName, List<String> replicaSuffixes, Optional<String> project)
+      throws IOException {
+
+    List<String> replicaUrls =
+        replicaSuffixes.stream()
+            .map(suffix -> gitPath.resolve("${name}" + suffix + ".git").toString())
+            .collect(toList());
+    config.setString("replication", null, "instanceLabel", remoteName);
+    config.setStringList("remote", remoteName, "url", replicaUrls);
+    config.setString("remote", remoteName, "apiUrl", adminRestSession.url());
+    config.setString("remote", remoteName, "fetch", "+refs/tags/*:refs/tags/*");
+    config.setInt("remote", remoteName, "timeout", 600);
+    config.setInt("remote", remoteName, "replicationDelay", TEST_REPLICATION_DELAY);
+    project.ifPresent(prj -> config.setString("remote", remoteName, "projects", prj));
+    config.setBoolean("gerrit", null, "autoReload", true);
+    config.save();
+  }
+
+  private void setReplicationCredentials(String remoteName, String username, String password)
+      throws IOException {
+    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 55fdee8..8738046 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
@@ -15,7 +15,6 @@
 package com.googlesource.gerrit.plugins.replication.pull.api;
 
 import static com.google.common.truth.Truth.assertThat;
-import static org.apache.http.HttpStatus.SC_ACCEPTED;
 import static org.apache.http.HttpStatus.SC_CREATED;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
@@ -24,15 +23,11 @@
 
 import com.google.common.collect.Lists;
 import com.google.gerrit.entities.Project;
-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;
 import com.google.gerrit.extensions.restapi.Response;
 import com.google.gerrit.extensions.restapi.RestApiException;
-import com.google.gerrit.server.config.UrlFormatter;
-import com.google.gerrit.server.git.WorkQueue;
-import com.google.gerrit.server.git.WorkQueue.Task;
 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;
@@ -40,17 +35,13 @@
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.MissingParentObjectException;
 import com.googlesource.gerrit.plugins.replication.pull.api.exception.RefUpdateException;
 import java.io.IOException;
-import java.util.Optional;
-import java.util.concurrent.ScheduledExecutorService;
 import org.eclipse.jgit.lib.Constants;
 import org.eclipse.jgit.lib.ObjectId;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
-import org.mockito.invocation.InvocationOnMock;
 import org.mockito.junit.MockitoJUnitRunner;
-import org.mockito.stubbing.Answer;
 
 @RunWith(MockitoJUnitRunner.class)
 public class ApplyObjectActionTest {
@@ -82,32 +73,13 @@
 
   @Mock ApplyObjectCommand applyObjectCommand;
   @Mock ProjectResource projectResource;
-  @Mock WorkQueue workQueue;
-  @Mock ScheduledExecutorService exceutorService;
-  @Mock DynamicItem<UrlFormatter> urlFormatterDynamicItem;
-  @Mock UrlFormatter urlFormatter;
-  @Mock WorkQueue.Task<Void> task;
   @Mock FetchPreconditions preConditions;
 
   @Before
   public void setup() {
-    when(workQueue.getDefaultQueue()).thenReturn(exceutorService);
-    when(urlFormatter.getRestUrl(anyString())).thenReturn(Optional.of(location));
-    when(exceutorService.submit(any(Runnable.class)))
-        .thenAnswer(
-            new Answer<WorkQueue.Task<Void>>() {
-              @Override
-              public Task<Void> answer(InvocationOnMock invocation) throws Throwable {
-                return task;
-              }
-            });
-    when(urlFormatterDynamicItem.get()).thenReturn(urlFormatter);
-    when(task.getTaskId()).thenReturn(taskId);
     when(preConditions.canCallFetchApi()).thenReturn(true);
 
-    applyObjectAction =
-        new ApplyObjectAction(
-            applyObjectCommand, workQueue, urlFormatterDynamicItem, preConditions);
+    applyObjectAction = new ApplyObjectAction(applyObjectCommand, preConditions);
   }
 
   @Test
@@ -207,24 +179,6 @@
     applyObjectAction.apply(projectResource, inputParams);
   }
 
-  @Test
-  public void shouldReturnScheduledTaskForAsyncCall() throws RestApiException {
-    RevisionInput inputParams = new RevisionInput(label, refName, createSampleRevisionData(), true);
-
-    Response<?> response = applyObjectAction.apply(projectResource, inputParams);
-    assertThat(response.statusCode()).isEqualTo(SC_ACCEPTED);
-  }
-
-  @Test
-  public void shouldLocationHeaderForAsyncCall() throws Exception {
-    RevisionInput inputParams = new RevisionInput(label, refName, createSampleRevisionData(), true);
-
-    Response<?> response = applyObjectAction.apply(projectResource, inputParams);
-    assertThat(response).isInstanceOf(Response.Accepted.class);
-    Response.Accepted acceptResponse = (Response.Accepted) response;
-    assertThat(acceptResponse.location()).isEqualTo(location);
-  }
-
   private RevisionData createSampleRevisionData() {
     RevisionObjectData commitData =
         new RevisionObjectData(Constants.OBJ_COMMIT, sampleCommitContent.getBytes());
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientTest.java
index 771acff..a3b0b02 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/client/FetchRestApiClientTest.java
@@ -31,6 +31,7 @@
 import com.googlesource.gerrit.plugins.replication.pull.Source;
 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.net.URISyntaxException;
 import java.nio.ByteBuffer;
@@ -72,11 +73,15 @@
   String label = "Replication";
   String refName = RefNames.REFS_HEADS + "master";
 
-  String expectedPayload = "{\"label\":\"Replication\", \"ref_name\": \"" + refName + "\"}";
+  String expectedPayload =
+      "{\"label\":\"Replication\", \"ref_name\": \"" + refName + "\", \"async\":false}";
+  String expectedAsyncPayload =
+      "{\"label\":\"Replication\", \"ref_name\": \"" + refName + "\", \"async\":true}";
   Header expectedHeader = new BasicHeader("Content-Type", "application/json");
+  SyncRefsFilter syncRefsFilter;
 
   String expectedSendObjectPayload =
-      "{\"label\":\"Replication\",\"ref_name\":\"refs/heads/master\",\"revision_data\":{\"commit_object\":{\"type\":1,\"content\":\"dHJlZSA3NzgxNGQyMTZhNmNhYjJkZGI5ZjI4NzdmYmJkMGZlYmRjMGZhNjA4CnBhcmVudCA5ODNmZjFhM2NmNzQ3MjVhNTNhNWRlYzhkMGMwNjEyMjEyOGY1YThkCmF1dGhvciBHZXJyaXQgVXNlciAxMDAwMDAwIDwxMDAwMDAwQDY5ZWMzOGYwLTM1MGUtNGQ5Yy05NmQ0LWJjOTU2ZjJmYWFhYz4gMTYxMDU3ODY0OCArMDEwMApjb21taXR0ZXIgR2Vycml0IENvZGUgUmV2aWV3IDxyb290QG1hY3plY2gtWFBTLTE1PiAxNjEwNTc4NjQ4ICswMTAwCgpVcGRhdGUgcGF0Y2ggc2V0IDEKClBhdGNoIFNldCAxOgoKKDEgY29tbWVudCkKClBhdGNoLXNldDogMQo\\u003d\"},\"tree_object\":{\"type\":2,\"content\":\"MTAwNjQ0IGJsb2IgYmIzODNmNTI0OWM2OGE0Y2M4YzgyYmRkMTIyOGI0YTg4ODNmZjZlOCAgICBmNzVhNjkwMDRhOTNiNGNjYzhjZTIxNWMxMjgwODYzNmMyYjc1Njc1\"},\"blobs\":[{\"type\":3,\"content\":\"ewogICJjb21tZW50cyI6IFsKICAgIHsKICAgICAgImtleSI6IHsKICAgICAgICAidXVpZCI6ICI5MGI1YWJmZl80ZjY3NTI2YSIsCiAgICAgICAgImZpbGVuYW1lIjogIi9DT01NSVRfTVNHIiwKICAgICAgICAicGF0Y2hTZXRJZCI6IDEKICAgICAgfSwKICAgICAgImxpbmVOYnIiOiA5LAogICAgICAiYXV0aG9yIjogewogICAgICAgICJpZCI6IDEwMDAwMDAKICAgICAgfSwKICAgICAgIndyaXR0ZW5PbiI6ICIyMDIxLTAxLTEzVDIyOjU3OjI4WiIsCiAgICAgICJzaWRlIjogMSwKICAgICAgIm1lc3NhZ2UiOiAidGVzdCBjb21tZW50IiwKICAgICAgInJhbmdlIjogewogICAgICAgICJzdGFydExpbmUiOiA5LAogICAgICAgICJzdGFydENoYXIiOiAyMSwKICAgICAgICAiZW5kTGluZSI6IDksCiAgICAgICAgImVuZENoYXIiOiAzNAogICAgICB9LAogICAgICAicmV2SWQiOiAiZjc1YTY5MDA0YTkzYjRjY2M4Y2UyMTVjMTI4MDg2MzZjMmI3NTY3NSIsCiAgICAgICJzZXJ2ZXJJZCI6ICI2OWVjMzhmMC0zNTBlLTRkOWMtOTZkNC1iYzk1NmYyZmFhYWMiLAogICAgICAidW5yZXNvbHZlZCI6IHRydWUKICAgIH0KICBdCn0\\u003d\"}]},\"async\":false}";
+      "{\"label\":\"Replication\",\"ref_name\":\"refs/heads/master\",\"revision_data\":{\"commit_object\":{\"type\":1,\"content\":\"dHJlZSA3NzgxNGQyMTZhNmNhYjJkZGI5ZjI4NzdmYmJkMGZlYmRjMGZhNjA4CnBhcmVudCA5ODNmZjFhM2NmNzQ3MjVhNTNhNWRlYzhkMGMwNjEyMjEyOGY1YThkCmF1dGhvciBHZXJyaXQgVXNlciAxMDAwMDAwIDwxMDAwMDAwQDY5ZWMzOGYwLTM1MGUtNGQ5Yy05NmQ0LWJjOTU2ZjJmYWFhYz4gMTYxMDU3ODY0OCArMDEwMApjb21taXR0ZXIgR2Vycml0IENvZGUgUmV2aWV3IDxyb290QG1hY3plY2gtWFBTLTE1PiAxNjEwNTc4NjQ4ICswMTAwCgpVcGRhdGUgcGF0Y2ggc2V0IDEKClBhdGNoIFNldCAxOgoKKDEgY29tbWVudCkKClBhdGNoLXNldDogMQo\\u003d\"},\"tree_object\":{\"type\":2,\"content\":\"MTAwNjQ0IGJsb2IgYmIzODNmNTI0OWM2OGE0Y2M4YzgyYmRkMTIyOGI0YTg4ODNmZjZlOCAgICBmNzVhNjkwMDRhOTNiNGNjYzhjZTIxNWMxMjgwODYzNmMyYjc1Njc1\"},\"blobs\":[{\"type\":3,\"content\":\"ewogICJjb21tZW50cyI6IFsKICAgIHsKICAgICAgImtleSI6IHsKICAgICAgICAidXVpZCI6ICI5MGI1YWJmZl80ZjY3NTI2YSIsCiAgICAgICAgImZpbGVuYW1lIjogIi9DT01NSVRfTVNHIiwKICAgICAgICAicGF0Y2hTZXRJZCI6IDEKICAgICAgfSwKICAgICAgImxpbmVOYnIiOiA5LAogICAgICAiYXV0aG9yIjogewogICAgICAgICJpZCI6IDEwMDAwMDAKICAgICAgfSwKICAgICAgIndyaXR0ZW5PbiI6ICIyMDIxLTAxLTEzVDIyOjU3OjI4WiIsCiAgICAgICJzaWRlIjogMSwKICAgICAgIm1lc3NhZ2UiOiAidGVzdCBjb21tZW50IiwKICAgICAgInJhbmdlIjogewogICAgICAgICJzdGFydExpbmUiOiA5LAogICAgICAgICJzdGFydENoYXIiOiAyMSwKICAgICAgICAiZW5kTGluZSI6IDksCiAgICAgICAgImVuZENoYXIiOiAzNAogICAgICB9LAogICAgICAicmV2SWQiOiAiZjc1YTY5MDA0YTkzYjRjY2M4Y2UyMTVjMTI4MDg2MzZjMmI3NTY3NSIsCiAgICAgICJzZXJ2ZXJJZCI6ICI2OWVjMzhmMC0zNTBlLTRkOWMtOTZkNC1iYzk1NmYyZmFhYWMiLAogICAgICAidW5yZXNvbHZlZCI6IHRydWUKICAgIH0KICBdCn0\\u003d\"}]}}";
   String commitObject =
       "tree 77814d216a6cab2ddb9f2877fbbd0febdc0fa608\n"
           + "parent 983ff1a3cf74725a53a5dec8d0c06122128f5a8d\n"
@@ -143,15 +148,17 @@
     when(credentialProvider.get(any(), any(CredentialItem.class))).thenReturn(true);
     when(credentials.create(anyString())).thenReturn(credentialProvider);
     when(replicationConfig.getConfig()).thenReturn(config);
+    when(config.getStringList("replication", null, "syncRefs")).thenReturn(new String[0]);
     when(source.getRemoteConfigName()).thenReturn("Replication");
     when(config.getString("replication", null, "instanceLabel")).thenReturn(label);
 
     HttpResult httpResult = new HttpResult(SC_CREATED, Optional.of("result message"));
     when(httpClient.execute(any(HttpPost.class), any(), any())).thenReturn(httpResult);
     when(httpClientFactory.create(any())).thenReturn(httpClient);
+    syncRefsFilter = new SyncRefsFilter(replicationConfig);
     objectUnderTest =
         new FetchRestApiClient(
-            credentials, httpClientFactory, replicationConfig, pluginName, source);
+            credentials, httpClientFactory, replicationConfig, syncRefsFilter, pluginName, source);
   }
 
   @Test
@@ -169,6 +176,67 @@
   }
 
   @Test
+  public void shouldByDefaultCallSyncFetchForAllRefs()
+      throws ClientProtocolException, IOException, URISyntaxException {
+
+    syncRefsFilter = new SyncRefsFilter(replicationConfig);
+    objectUnderTest =
+        new FetchRestApiClient(
+            credentials, httpClientFactory, replicationConfig, syncRefsFilter, pluginName, source);
+
+    objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
+
+    verify(httpClient, times(1)).execute(httpPostCaptor.capture(), any(), any());
+
+    HttpPost httpPost = httpPostCaptor.getValue();
+    assertThat(readPayload(httpPost)).isEqualTo(expectedPayload);
+  }
+
+  @Test
+  public void shouldCallAsyncFetchForAllRefs()
+      throws ClientProtocolException, IOException, URISyntaxException {
+
+    when(config.getStringList("replication", null, "syncRefs"))
+        .thenReturn(new String[] {"NO_SYNC_REFS"});
+    syncRefsFilter = new SyncRefsFilter(replicationConfig);
+    objectUnderTest =
+        new FetchRestApiClient(
+            credentials, httpClientFactory, replicationConfig, syncRefsFilter, pluginName, source);
+
+    objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
+
+    verify(httpClient, times(1)).execute(httpPostCaptor.capture(), any(), any());
+
+    HttpPost httpPost = httpPostCaptor.getValue();
+    assertThat(readPayload(httpPost)).isEqualTo(expectedAsyncPayload);
+  }
+
+  @Test
+  public void shouldCallSyncFetchOnlyForMetaRef()
+      throws ClientProtocolException, IOException, URISyntaxException {
+    String metaRefName = "refs/changes/01/101/meta";
+    String expectedMetaRefPayload =
+        "{\"label\":\"Replication\", \"ref_name\": \"" + metaRefName + "\", \"async\":false}";
+
+    when(config.getStringList("replication", null, "syncRefs"))
+        .thenReturn(new String[] {"^refs\\/changes\\/.*\\/meta"});
+    syncRefsFilter = new SyncRefsFilter(replicationConfig);
+    objectUnderTest =
+        new FetchRestApiClient(
+            credentials, httpClientFactory, replicationConfig, syncRefsFilter, pluginName, source);
+
+    objectUnderTest.callFetch(Project.nameKey("test_repo"), refName, new URIish(api));
+    verify(httpClient, times(1)).execute(httpPostCaptor.capture(), any(), any());
+    HttpPost httpPost = httpPostCaptor.getValue();
+    assertThat(readPayload(httpPost)).isEqualTo(expectedAsyncPayload);
+
+    objectUnderTest.callFetch(Project.nameKey("test_repo"), metaRefName, new URIish(api));
+    verify(httpClient, times(2)).execute(httpPostCaptor.capture(), any(), any());
+    httpPost = httpPostCaptor.getValue();
+    assertThat(readPayload(httpPost)).isEqualTo(expectedMetaRefPayload);
+  }
+
+  @Test
   public void shouldCallFetchEndpointWithPayload()
       throws ClientProtocolException, IOException, URISyntaxException {
 
@@ -241,7 +309,12 @@
         NullPointerException.class,
         () ->
             new FetchRestApiClient(
-                credentials, httpClientFactory, replicationConfig, pluginName, source));
+                credentials,
+                httpClientFactory,
+                replicationConfig,
+                syncRefsFilter,
+                pluginName,
+                source));
   }
 
   @Test
@@ -251,7 +324,12 @@
         NullPointerException.class,
         () ->
             new FetchRestApiClient(
-                credentials, httpClientFactory, replicationConfig, pluginName, source));
+                credentials,
+                httpClientFactory,
+                replicationConfig,
+                syncRefsFilter,
+                pluginName,
+                source));
   }
 
   @Test
@@ -261,7 +339,12 @@
         NullPointerException.class,
         () ->
             new FetchRestApiClient(
-                credentials, httpClientFactory, replicationConfig, pluginName, source));
+                credentials,
+                httpClientFactory,
+                replicationConfig,
+                syncRefsFilter,
+                pluginName,
+                source));
   }
 
   public String readPayload(HttpPost entity) throws UnsupportedOperationException, IOException {