Merge branch 'stable-3.0' into stable-3.1

* stable-3.0:
  Fix compile and test issues on stable-2.16
  Document Geo-located Gerrit master selection solutions
  Improve logging when catching a zookeeper error
  Expose replication lag metrics
  Fix cache eviction for projects cache
  Store project versioning

Change-Id: I98ffb462c2e26be3a5ced1f440879dca7fe374e5
diff --git a/DESIGN.md b/DESIGN.md
index 0995a14..270b7af 100644
--- a/DESIGN.md
+++ b/DESIGN.md
@@ -510,6 +510,60 @@
 set of refs in Read Only state across all the cluster if the RW node is failing after having
 sent the request to the Ref-DB but before persisting this request into its `git` layer.
 
+#### Geo located Gerrit master election
+
+Once you go multi-site multi-master you can improve the latency of your calls by
+serving traffic from the closest server to your user.
+
+Whether you are running your infrastructure in the cloud or on premise you have different solutions you can look at.
+
+##### AWS
+
+Route53 AWS DNS service offers the opportunity of doing [Geo Proximity](https://docs.aws.amazon.com/Route53/latest/DeveloperGuide/routing-policy.html#routing-policy-geoproximity)
+routing using [Traffic Flow](https://docs.aws.amazon.com/Route53/latest/DeveloperGuide/traffic-flow.html).
+
+Traffic flow is a tool which allows the definition of traffic policies and rules via a UI. Traffic rules are of different types, among which *Geoproximity* rules.
+
+When creating geoproximity rules for your resources you can specify one of the following values for each rule:
+
+* If you're using AWS resources, the AWS Region that you created the resource in
+* If you're using non-AWS resources, the latitude and longitude of the resource.
+
+This allows you to have an hybrid cloud-on premise infrastructure.
+
+You can define quite complex failover rules to ensure high availability of your system ([here](https://pasteboard.co/ILFSd5Y.png) an example).
+
+Overall the service provided is pretty much a smart reverse-proxy, if you want more
+complex routing strategies you will still need a proper Load Balancer.
+
+##### GCE
+
+GCE [doesn't offer](https://cloud.google.com/docs/compare/aws/networking#dns) a Geographical based routing, but it implicitly has geo-located DNS entries
+when distributing your application among different zones.
+
+The Load Balancer will balance the traffic to the [nearest available instance](https://cloud.google.com/load-balancing/docs/backend-service#backend_services_and_regions)
+, but this is not configurable and the app server has to be in GC.
+
+Hybrid architectures are supported but would make things more complicated,
+hence this solution is probably worthy only when the Gerrit instances are running in GC.
+
+##### On premise
+
+If you are going for an on premise solution and using HAProxy as Load Balancer,
+it is easy to define static ACL based on IP ranges and use them to route your traffic.
+
+This [blogpost](https://icicimov.github.io/blog/devops/Haproxy-GeoIP/) explains how to achieve it.
+
+On top of that, you want to define a DNS entry per zone and use the ACLs you just defined to
+issue redirection of the calls to most appropiate zone.
+
+You will have to add to your frontend definition your redirection strategy, i.e.:
+
+```
+http-request redirect code 307 prefix https://review-eu.gerrithub.io if acl_EU
+http-request redirect code 307 prefix https://review-am.gerrithub.io if acl_NA
+```
+
 # Next steps in the roadmap
 
 ## Step-1: Fill the gaps in multi-site Stage #7 implementation:
diff --git a/external_plugin_deps.bzl b/external_plugin_deps.bzl
index 2bbf1aa..18e407f 100644
--- a/external_plugin_deps.bzl
+++ b/external_plugin_deps.bzl
@@ -9,8 +9,8 @@
 
     maven_jar(
         name = "global-refdb",
-        artifact = "com.gerritforge:global-refdb:3.1.0-rc1",
-        sha1 = "61fc8defaed9c364e6bfa101563e434fcc70038f",
+        artifact = "com.gerritforge:global-refdb:3.1.2",
+        sha1 = "6ddee3de0f3fe9254453118ae1eca481ec03e957"
     )
 
     maven_jar(
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/SharedRefDatabaseWrapper.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/SharedRefDatabaseWrapper.java
index f15a4f2..9827002 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/SharedRefDatabaseWrapper.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/SharedRefDatabaseWrapper.java
@@ -21,6 +21,7 @@
 import com.google.gerrit.entities.Project;
 import com.google.gerrit.extensions.registration.DynamicItem;
 import com.google.inject.Inject;
+import java.util.Optional;
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.lib.Ref;
 
@@ -59,6 +60,12 @@
   }
 
   @Override
+  public <T> boolean compareAndPut(Project.NameKey project, String refName, T currValue, T newValue)
+      throws GlobalRefDbSystemError {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   public AutoCloseable lockRef(Project.NameKey project, String refName)
       throws GlobalRefDbLockException {
     AutoCloseable locker = sharedRefDb().lockRef(project, refName);
@@ -77,6 +84,12 @@
     sharedRefLogger.logProjectDelete(project.get());
   }
 
+  @Override
+  public <T> Optional<T> get(Project.NameKey nameKey, String s, Class<T> clazz)
+      throws GlobalRefDbSystemError {
+    return sharedRefDb().get(nameKey, s, clazz);
+  }
+
   private GlobalRefDatabase sharedRefDb() {
     return sharedRefDbDynamicItem.get();
   }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/consumer/AbstractSubcriber.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/consumer/AbstractSubcriber.java
index ec6072c..816edc9 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/consumer/AbstractSubcriber.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/consumer/AbstractSubcriber.java
@@ -74,6 +74,7 @@
         msgLog.log(Direction.CONSUME, topic, event);
         eventRouter.route(event.getEvent());
         subscriberMetrics.incrementSubscriberConsumedMessage();
+        subscriberMetrics.updateReplicationStatusMetrics(event);
       } catch (IOException e) {
         logger.atSevere().withCause(e).log(
             "Malformed event '%s': [Exception: %s]", event.getHeader());
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/consumer/SubscriberMetrics.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/consumer/SubscriberMetrics.java
index 36f618e..16c047a 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/consumer/SubscriberMetrics.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/consumer/SubscriberMetrics.java
@@ -14,25 +14,50 @@
 
 package com.googlesource.gerrit.plugins.multisite.consumer;
 
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.gerritforge.gerrit.eventbroker.EventMessage;
+import com.google.common.flogger.FluentLogger;
 import com.google.gerrit.metrics.Counter1;
 import com.google.gerrit.metrics.Description;
 import com.google.gerrit.metrics.MetricMaker;
+import com.google.gerrit.server.events.Event;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import com.googlesource.gerrit.plugins.multisite.MultiSiteMetrics;
+import com.googlesource.gerrit.plugins.multisite.validation.ProjectVersionRefUpdate;
+import com.googlesource.gerrit.plugins.replication.RefReplicatedEvent;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
 
 @Singleton
 public class SubscriberMetrics extends MultiSiteMetrics {
+  private static final FluentLogger logger = FluentLogger.forEnclosingClass();
   private static final String SUBSCRIBER_SUCCESS_COUNTER = "subscriber_msg_consumer_counter";
   private static final String SUBSCRIBER_FAILURE_COUNTER =
       "subscriber_msg_consumer_failure_counter";
+  private static final String PROJECT_REPLICATION_LAG_MS_PREFIX =
+      "multi_site/subscriber/subscriber_replication_status/ms_behind_for_";
 
   private final Counter1<String> subscriberSuccessCounter;
   private final Counter1<String> subscriberFailureCounter;
 
-  @Inject
-  public SubscriberMetrics(MetricMaker metricMaker) {
+  private MetricRegistry metricRegistry;
+  private MetricMaker metricMaker;
+  public Map<String, Long> replicationStatusPerProject = new HashMap<>();
 
+  private ProjectVersionRefUpdate projectVersionRefUpdate;
+
+  @Inject
+  public SubscriberMetrics(
+      MetricMaker metricMaker,
+      MetricRegistry metricRegistry,
+      ProjectVersionRefUpdate projectVersionRefUpdate) {
+
+    this.projectVersionRefUpdate = projectVersionRefUpdate;
+    this.metricMaker = metricMaker;
+    this.metricRegistry = metricRegistry;
     this.subscriberSuccessCounter =
         metricMaker.newCounter(
             "multi_site/subscriber/subscriber_message_consumer_counter",
@@ -56,4 +81,43 @@
   public void incrementSubscriberFailedToConsumeMessage() {
     subscriberFailureCounter.increment(SUBSCRIBER_FAILURE_COUNTER);
   }
+
+  public void updateReplicationStatusMetrics(EventMessage eventMessage) {
+    Event event = eventMessage.getEvent();
+    if (event instanceof RefReplicatedEvent) {
+      RefReplicatedEvent refReplicatedEvent = (RefReplicatedEvent) event;
+      String projectName = refReplicatedEvent.getProjectNameKey().get();
+      logger.atFine().log("Updating replication lag for %s", projectName);
+      Optional<Long> remoteVersion = projectVersionRefUpdate.getProjectRemoteVersion(projectName);
+      Optional<Long> localVersion = projectVersionRefUpdate.getProjectLocalVersion(projectName);
+      if (remoteVersion.isPresent() && localVersion.isPresent()) {
+        Long lag = remoteVersion.get() - localVersion.get();
+        logger.atFine().log("Calculated lag for project '%s' [%d]", projectName, lag);
+        replicationStatusPerProject.put(projectName, lag);
+        upsertMetricsForProject(projectName);
+      } else {
+        logger.atWarning().log(
+            "Didn't update metric for %s. Local [%b] or remote [%b] version is not defined",
+            projectName, localVersion.isPresent(), remoteVersion.isPresent());
+      }
+    } else {
+      logger.atInfo().log("Not a ref-replicated-event event [%s], skipping", event.type);
+    }
+  }
+
+  private void upsertMetricsForProject(String projectName) {
+    String metricName = PROJECT_REPLICATION_LAG_MS_PREFIX + projectName;
+    if (metricRegistry.getGauges(MetricFilter.contains(metricName)).isEmpty()) {
+      metricMaker.newCallbackMetric(
+          metricName,
+          Long.class,
+          new Description(String.format("%s replication lag (ms)", metricName))
+              .setGauge()
+              .setUnit(Description.Units.MILLISECONDS),
+          () -> replicationStatusPerProject.get(projectName));
+      logger.atFine().log("Added last replication timestamp callback metric for '%s'", projectName);
+    } else {
+      logger.atFine().log("Don't add metric since it already exists for project '%s'", projectName);
+    }
+  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/event/EventModule.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/event/EventModule.java
index 8889935..1c0c644 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/event/EventModule.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/event/EventModule.java
@@ -17,6 +17,7 @@
 import com.google.gerrit.extensions.registration.DynamicSet;
 import com.google.gerrit.lifecycle.LifecycleModule;
 import com.google.gerrit.server.events.EventListener;
+import com.googlesource.gerrit.plugins.multisite.validation.ProjectVersionRefUpdate;
 import java.util.concurrent.Executor;
 
 public class EventModule extends LifecycleModule {
@@ -26,5 +27,6 @@
     bind(Executor.class).annotatedWith(EventExecutor.class).toProvider(EventExecutorProvider.class);
     listener().to(EventExecutorProvider.class);
     DynamicSet.bind(binder(), EventListener.class).to(EventHandler.class);
+    DynamicSet.bind(binder(), EventListener.class).to(ProjectVersionRefUpdate.class);
   }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/GsonParser.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/GsonParser.java
index 922da1a..0bbdada 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/GsonParser.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/GsonParser.java
@@ -14,7 +14,7 @@
 
 package com.googlesource.gerrit.plugins.multisite.forwarder;
 
-import com.google.common.base.Strings;
+import com.google.common.base.MoreObjects;
 import com.google.gerrit.entities.Account;
 import com.google.gerrit.entities.AccountGroup;
 import com.google.gerrit.server.events.EventGson;
@@ -31,31 +31,43 @@
     this.gson = gson;
   }
 
-  public Object fromJson(String cacheName, String jsonString) {
-    JsonElement json = gson.fromJson(Strings.nullToEmpty(jsonString), JsonElement.class);
+  @SuppressWarnings("cast")
+  public Object fromJson(String cacheName, Object json) {
     Object key;
     // Need to add a case for 'adv_bases'
     switch (cacheName) {
       case Constants.ACCOUNTS:
-        key = Account.id(json.getAsJsonObject().get("id").getAsInt());
+        key = Account.id(jsonElement(json).getAsJsonObject().get("id").getAsInt());
         break;
       case Constants.GROUPS:
-        key = AccountGroup.id(json.getAsJsonObject().get("id").getAsInt());
+        key = AccountGroup.id(jsonElement(json).getAsJsonObject().get("id").getAsInt());
         break;
       case Constants.GROUPS_BYINCLUDE:
       case Constants.GROUPS_MEMBERS:
-        key = AccountGroup.uuid(json.getAsJsonObject().get("uuid").getAsString());
+        key = AccountGroup.uuid(jsonElement(json).getAsJsonObject().get("uuid").getAsString());
         break;
       case Constants.PROJECT_LIST:
-        key = gson.fromJson(json, Object.class);
+        key = gson.fromJson(nullToEmpty(json).toString(), Object.class);
         break;
       default:
-        try {
-          key = gson.fromJson(json, String.class);
-        } catch (Exception e) {
-          key = gson.fromJson(json, Object.class);
+        if (json instanceof String) {
+          key = (String) json;
+        } else {
+          try {
+            key = gson.fromJson(nullToEmpty(json).toString().trim(), String.class);
+          } catch (Exception e) {
+            key = gson.fromJson(nullToEmpty(json).toString(), Object.class);
+          }
         }
     }
     return key;
   }
+
+  private JsonElement jsonElement(Object json) {
+    return gson.fromJson(nullToEmpty(json), JsonElement.class);
+  }
+
+  private static String nullToEmpty(Object value) {
+    return MoreObjects.firstNonNull(value, "").toString().trim();
+  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/router/CacheEvictionEventRouter.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/router/CacheEvictionEventRouter.java
index 8c86c0c..0fb0c0a 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/router/CacheEvictionEventRouter.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/router/CacheEvictionEventRouter.java
@@ -34,8 +34,7 @@
 
   @Override
   public void route(CacheEvictionEvent cacheEvictionEvent) throws CacheNotFoundException {
-    Object parsedKey =
-        gsonParser.fromJson(cacheEvictionEvent.cacheName, cacheEvictionEvent.key.toString());
+    Object parsedKey = gsonParser.fromJson(cacheEvictionEvent.cacheName, cacheEvictionEvent.key);
     cacheEvictionHanlder.evict(CacheEntry.from(cacheEvictionEvent.cacheName, parsedKey));
   }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/ProjectVersionRefUpdate.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/ProjectVersionRefUpdate.java
new file mode 100644
index 0000000..c8ddc87
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/ProjectVersionRefUpdate.java
@@ -0,0 +1,293 @@
+// Copyright (C) 2020 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.multisite.validation;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.eclipse.jgit.lib.Constants.OBJ_BLOB;
+
+import com.gerritforge.gerrit.globalrefdb.GlobalRefDbSystemError;
+import com.google.common.base.CharMatcher;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.flogger.FluentLogger;
+import com.google.common.primitives.Ints;
+import com.google.gerrit.entities.Project;
+import com.google.gerrit.server.events.Event;
+import com.google.gerrit.server.events.EventListener;
+import com.google.gerrit.server.events.RefUpdatedEvent;
+import com.google.gerrit.server.git.GitRepositoryManager;
+import com.google.gerrit.server.notedb.IntBlob;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.googlesource.gerrit.plugins.multisite.SharedRefDatabaseWrapper;
+import com.googlesource.gerrit.plugins.multisite.forwarder.Context;
+import com.googlesource.gerrit.plugins.replication.RefReplicatedEvent;
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+import org.eclipse.jgit.lib.ObjectId;
+import org.eclipse.jgit.lib.ObjectIdRef;
+import org.eclipse.jgit.lib.ObjectInserter;
+import org.eclipse.jgit.lib.ObjectLoader;
+import org.eclipse.jgit.lib.ObjectReader;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.lib.RefUpdate;
+import org.eclipse.jgit.lib.Repository;
+import org.eclipse.jgit.revwalk.RevCommit;
+import org.eclipse.jgit.revwalk.RevWalk;
+import org.eclipse.jgit.transport.RemoteRefUpdate;
+
+@Singleton
+public class ProjectVersionRefUpdate implements EventListener {
+  private static final FluentLogger logger = FluentLogger.forEnclosingClass();
+  public static final String MULTI_SITE_VERSIONING_REF = "refs/multi-site/version";
+  public static final String SEQUENCE_REF_PREFIX = "refs/sequences/";
+  private static final Ref NULL_PROJECT_VERSION_REF =
+      new ObjectIdRef.Unpeeled(Ref.Storage.NETWORK, MULTI_SITE_VERSIONING_REF, ObjectId.zeroId());
+  private static final Set<RefUpdate.Result> SUCCESSFUL_RESULTS =
+      ImmutableSet.of(RefUpdate.Result.NEW, RefUpdate.Result.FORCED, RefUpdate.Result.NO_CHANGE);
+
+  protected final SharedRefDatabaseWrapper sharedRefDb;
+  private final GitRepositoryManager gitRepositoryManager;
+
+  @Inject
+  public ProjectVersionRefUpdate(
+      GitRepositoryManager gitRepositoryManager, SharedRefDatabaseWrapper sharedRefDb) {
+    this.gitRepositoryManager = gitRepositoryManager;
+    this.sharedRefDb = sharedRefDb;
+  }
+
+  @Override
+  public void onEvent(Event event) {
+    logger.atFine().log("Processing event type: " + event.type);
+    // Producer of the Event use RefUpdatedEvent to trigger the version update
+    if (!Context.isForwardedEvent() && event instanceof RefUpdatedEvent) {
+      updateProducerProjectVersionUpdate((RefUpdatedEvent) event);
+    }
+
+    // Consumers of the Event use RefReplicatedEvent to trigger the version update
+    if (Context.isForwardedEvent() && event instanceof RefReplicatedEvent) {
+      updateConsumerProjectVersion((RefReplicatedEvent) event);
+    }
+  }
+
+  private void updateConsumerProjectVersion(RefReplicatedEvent refReplicatedEvent) {
+    Project.NameKey projectNameKey = refReplicatedEvent.getProjectNameKey();
+    if (!refReplicatedEvent.refStatus.equals(RemoteRefUpdate.Status.OK)) {
+      logger.atFine().log(
+          String.format(
+              "Skipping version update for %s. RefReplicatedEvent failed with %s",
+              projectNameKey.get(), refReplicatedEvent.refStatus));
+      return;
+    }
+    if (refReplicatedEvent.getRefName().startsWith(SEQUENCE_REF_PREFIX)) {
+      logger.atFine().log("Found Sequence ref, skipping update for " + projectNameKey.get());
+      return;
+    }
+    try {
+      updateLocalProjectVersion(projectNameKey, refReplicatedEvent.getRefName());
+    } catch (LocalProjectVersionUpdateException e) {
+      logger.atSevere().withCause(e).log(
+          "Issue encountered when updating version for project " + projectNameKey);
+    }
+  }
+
+  private void updateProducerProjectVersionUpdate(RefUpdatedEvent refUpdatedEvent) {
+    if (refUpdatedEvent.getRefName().startsWith(SEQUENCE_REF_PREFIX)) {
+      logger.atFine().log(
+          "Found Sequence ref, skipping update for " + refUpdatedEvent.getProjectNameKey().get());
+      return;
+    }
+    try {
+      Project.NameKey projectNameKey = refUpdatedEvent.getProjectNameKey();
+      Ref currentProjectVersionRef = getLocalProjectVersionRef(refUpdatedEvent.getProjectNameKey());
+      ObjectId newProjectVersionObjectId =
+          updateLocalProjectVersion(projectNameKey, refUpdatedEvent.getRefName());
+      updateSharedProjectVersion(
+          projectNameKey, currentProjectVersionRef, newProjectVersionObjectId);
+    } catch (LocalProjectVersionUpdateException | SharedProjectVersionUpdateException e) {
+      logger.atSevere().withCause(e).log(
+          "Issue encountered when updating version for project "
+              + refUpdatedEvent.getProjectNameKey());
+    }
+  }
+
+  private RefUpdate getProjectVersionRefUpdate(Repository repository, Long version)
+      throws IOException {
+    RefUpdate refUpdate = repository.getRefDatabase().newUpdate(MULTI_SITE_VERSIONING_REF, false);
+    refUpdate.setNewObjectId(getNewId(repository, version));
+    refUpdate.setForceUpdate(true);
+    return refUpdate;
+  }
+
+  private ObjectId getNewId(Repository repository, Long version) throws IOException {
+    ObjectInserter ins = repository.newObjectInserter();
+    ObjectId newId = ins.insert(OBJ_BLOB, Long.toString(version).getBytes(UTF_8));
+    ins.flush();
+    return newId;
+  }
+
+  private Ref getLocalProjectVersionRef(Project.NameKey projectNameKey)
+      throws LocalProjectVersionUpdateException {
+    try (Repository repository = gitRepositoryManager.openRepository(projectNameKey)) {
+      Ref ref = repository.findRef(MULTI_SITE_VERSIONING_REF);
+      return ref != null ? ref : NULL_PROJECT_VERSION_REF;
+    } catch (IOException e) {
+      String message =
+          String.format("Error while getting current version for %s", projectNameKey.get());
+      logger.atSevere().withCause(e).log(message);
+      throw new LocalProjectVersionUpdateException(message);
+    }
+  }
+
+  private Long getLastRefUpdatedTimestamp(Project.NameKey projectNameKey, String refName)
+      throws LocalProjectVersionUpdateException {
+    logger.atFine().log(
+        String.format(
+            "Getting last ref updated time for project %s, ref %s", projectNameKey.get(), refName));
+    try (Repository repository = gitRepositoryManager.openRepository(projectNameKey)) {
+      Ref ref = repository.findRef(refName);
+      try (RevWalk walk = new RevWalk(repository)) {
+        RevCommit commit = walk.parseCommit(ref.getObjectId());
+        return Integer.toUnsignedLong(commit.getCommitTime());
+      }
+    } catch (IOException ioe) {
+      String message =
+          String.format(
+              "Error while getting last ref updated time for project %s, ref %s",
+              projectNameKey.get(), refName);
+      logger.atSevere().withCause(ioe).log(message);
+      throw new LocalProjectVersionUpdateException(message);
+    }
+  }
+
+  private void updateSharedProjectVersion(
+      Project.NameKey projectNameKey, Ref currentRef, ObjectId newObjectId)
+      throws SharedProjectVersionUpdateException {
+    logger.atFine().log(
+        String.format(
+            "Updating shared project version for %s. Current value %s, new value: %s",
+            projectNameKey.get(), currentRef.getObjectId(), newObjectId));
+    try {
+      boolean success = sharedRefDb.compareAndPut(projectNameKey, currentRef, newObjectId);
+      String message =
+          String.format(
+              "Project version update failed for %s. Current value %s, new value: %s",
+              projectNameKey.get(), currentRef.getObjectId(), newObjectId);
+      if (!success) {
+        logger.atSevere().log(message);
+        throw new SharedProjectVersionUpdateException(message);
+      }
+    } catch (GlobalRefDbSystemError refDbSystemError) {
+      String message =
+          String.format(
+              "Error while updating shared project version for %s. Current value %s, new value: %s. Error: %s",
+              projectNameKey.get(),
+              currentRef.getObjectId(),
+              newObjectId,
+              refDbSystemError.getMessage());
+      logger.atSevere().withCause(refDbSystemError).log(message);
+      throw new SharedProjectVersionUpdateException(message);
+    }
+  }
+
+  public Optional<Long> getProjectLocalVersion(String projectName) {
+    try (Repository repository =
+        gitRepositoryManager.openRepository(Project.NameKey.parse(projectName))) {
+      Optional<IntBlob> blob = IntBlob.parse(repository, MULTI_SITE_VERSIONING_REF);
+      if (blob.isPresent()) {
+        Long repoVersion = Integer.toUnsignedLong(blob.get().value());
+        logger.atInfo().log("Local project '%s' has version %d", projectName, repoVersion);
+        return Optional.of(repoVersion);
+      }
+    } catch (IOException e) {
+      logger.atSevere().withCause(e).log("Cannot read local project '%s' version", projectName);
+    }
+    return Optional.empty();
+  }
+
+  public Optional<Long> getProjectRemoteVersion(String projectName) {
+    Optional<ObjectId> remoteObjectId =
+        sharedRefDb.get(
+            Project.NameKey.parse(projectName), MULTI_SITE_VERSIONING_REF, ObjectId.class);
+    if (remoteObjectId.isPresent()) {
+      return getLongFromObjectId(projectName, remoteObjectId.get());
+    } else {
+      logger.atSevere().log("Didn't find remote version for %s", projectName);
+      return Optional.empty();
+    }
+  }
+
+  private Optional<Long> getLongFromObjectId(String projectName, ObjectId objectId) {
+    try (Repository repository =
+        gitRepositoryManager.openRepository(Project.NameKey.parse(projectName))) {
+      ObjectReader or = repository.newObjectReader();
+      ObjectLoader ol = or.open(objectId, OBJ_BLOB);
+      if (ol.getType() != OBJ_BLOB) {
+        // In theory this should be thrown by open but not all implementations may do it properly
+        // (certainly InMemoryRepository doesn't).
+        logger.atSevere().log("Incorrect object type loaded for objectId %s", objectId.toString());
+        return Optional.empty();
+      }
+      String str = CharMatcher.whitespace().trimFrom(new String(ol.getCachedBytes(), UTF_8));
+      Integer value = Ints.tryParse(str);
+      logger.atInfo().log(
+          "Found remote version for project %s, value: %s - %d",
+          projectName, objectId.toString(), value);
+      return Optional.of(Integer.toUnsignedLong(value));
+    } catch (IOException e) {
+      logger.atSevere().withCause(e).log("Cannot parse objectId %s", objectId.toString());
+      return Optional.empty();
+    }
+  }
+
+  private ObjectId updateLocalProjectVersion(Project.NameKey projectNameKey, String refName)
+      throws LocalProjectVersionUpdateException {
+    Long lastRefUpdatedTimestamp = getLastRefUpdatedTimestamp(projectNameKey, refName);
+    logger.atFine().log("Updating local version for project " + projectNameKey.get());
+    try (Repository repository = gitRepositoryManager.openRepository(projectNameKey)) {
+      RefUpdate refUpdate = getProjectVersionRefUpdate(repository, lastRefUpdatedTimestamp);
+      RefUpdate.Result result = refUpdate.update();
+      if (!isSuccessful(result)) {
+        String message =
+            String.format(
+                "RefUpdate failed with result %s for: project=%s, version=%d",
+                result.name(), projectNameKey.get(), lastRefUpdatedTimestamp);
+        logger.atSevere().log(message);
+        throw new LocalProjectVersionUpdateException(message);
+      }
+      return refUpdate.getNewObjectId();
+    } catch (IOException e) {
+      String message = "Cannot create versioning command for " + projectNameKey.get();
+      logger.atSevere().withCause(e).log(message);
+      throw new LocalProjectVersionUpdateException(message);
+    }
+  }
+
+  private Boolean isSuccessful(RefUpdate.Result result) {
+    return SUCCESSFUL_RESULTS.contains(result);
+  }
+
+  public static class LocalProjectVersionUpdateException extends Exception {
+    public LocalProjectVersionUpdateException(String projectName) {
+      super("Cannot update local project version of " + projectName);
+    }
+  }
+
+  public static class SharedProjectVersionUpdateException extends Exception {
+    public SharedProjectVersionUpdateException(String projectName) {
+      super("Cannot update shared project version of " + projectName);
+    }
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/RefUpdateValidator.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/RefUpdateValidator.java
index 312562c..dcc9030 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/RefUpdateValidator.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/RefUpdateValidator.java
@@ -86,7 +86,8 @@
   public RefUpdate.Result executeRefUpdate(
       RefUpdate refUpdate, NoParameterFunction<RefUpdate.Result> refUpdateFunction)
       throws IOException {
-    if (refEnforcement.getPolicy(projectName) == EnforcePolicy.IGNORED) {
+    if (isProjectVersionUpdate(refUpdate.getName())
+        || refEnforcement.getPolicy(projectName) == EnforcePolicy.IGNORED) {
       return refUpdateFunction.invoke();
     }
 
@@ -105,6 +106,13 @@
     return null;
   }
 
+  private Boolean isProjectVersionUpdate(String refName) {
+    Boolean isProjectVersionUpdate =
+        refName.equals(ProjectVersionRefUpdate.MULTI_SITE_VERSIONING_REF);
+    logger.atFine().log("Is project version update? " + isProjectVersionUpdate);
+    return isProjectVersionUpdate;
+  }
+
   private <T extends Throwable> void softFailBasedOnEnforcement(T e, EnforcePolicy policy)
       throws T {
     logger.atWarning().withCause(e).log(
@@ -153,6 +161,11 @@
           sharedRefDb.compareAndPut(
               Project.nameKey(projectName), refPair.compareRef, refPair.putValue);
     } catch (GlobalRefDbSystemError e) {
+      logger.atWarning().withCause(e).log(
+          "Not able to persist the data in Zookeeper for project '{}' and ref '{}', message: {}",
+          projectName,
+          refPair.getName(),
+          e.getMessage());
       throw new SharedDbSplitBrainException(errorMessage, e);
     }
 
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/dfsrefdb/NoopSharedRefDatabase.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/dfsrefdb/NoopSharedRefDatabase.java
index b82d7d9..1530838 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/dfsrefdb/NoopSharedRefDatabase.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/validation/dfsrefdb/NoopSharedRefDatabase.java
@@ -18,6 +18,7 @@
 import com.gerritforge.gerrit.globalrefdb.GlobalRefDbLockException;
 import com.gerritforge.gerrit.globalrefdb.GlobalRefDbSystemError;
 import com.google.gerrit.entities.Project;
+import java.util.Optional;
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.lib.Ref;
 
@@ -35,6 +36,12 @@
   }
 
   @Override
+  public <T> boolean compareAndPut(Project.NameKey project, String refName, T currValue, T newValue)
+      throws GlobalRefDbSystemError {
+    return false;
+  }
+
+  @Override
   public AutoCloseable lockRef(Project.NameKey project, String refName)
       throws GlobalRefDbLockException {
     return () -> {};
@@ -47,4 +54,10 @@
 
   @Override
   public void remove(Project.NameKey project) throws GlobalRefDbSystemError {}
+
+  @Override
+  public <T> Optional<T> get(Project.NameKey project, String refName, Class<T> clazz)
+      throws GlobalRefDbSystemError {
+    return Optional.empty();
+  }
 }
diff --git a/src/main/resources/Documentation/about.md b/src/main/resources/Documentation/about.md
index 42249b1..c1135bb 100644
--- a/src/main/resources/Documentation/about.md
+++ b/src/main/resources/Documentation/about.md
@@ -93,10 +93,6 @@
 
 `metric=plugins/multi-site/multi_site/subscriber/subscriber_message_consumer_failure_counter/subscriber_msg_consumer_poll_failure_counter, type=com.codahale.metrics.Meter`
 
-* Subscriber replication status (latest replication Epoch time in seconds) per instance
+* Subscriber replication status per project (ms behind the producer)
 
-`metric=plugins/multi-site/multi_site/subscriber/subscriber_replication_status/instance_latest_replication_epochtime_secs, type=com.google.gerrit.metrics.dropwizard.CallbackMetricImpl`
-
-* Subscriber replication status (latest replication Epoch time in seconds) per project
-
-`metric=plugins/multi-site/multi_site/subscriber/subscriber_replication_status/latest_replication_epochtime_secs_<projectName>, type=com.google.gerrit.metrics.dropwizard.CallbackMetricImpl`
+`metric=site/multi_site/subscriber/subscriber_replication_status/ms_behind_for_<projectName>, type=com.google.gerrit.metrics.dropwizard.CallbackMetricImpl`
\ No newline at end of file
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/event/CacheEvictionEventRouterTest.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/event/CacheEvictionEventRouterTest.java
index c919df8..a632b74 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/multisite/event/CacheEvictionEventRouterTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/event/CacheEvictionEventRouterTest.java
@@ -46,4 +46,13 @@
 
     verify(cacheEvictionHandler).evict(CacheEntry.from(event.cacheName, event.key));
   }
+
+  @Test
+  public void routerShouldSendEventsToTheAppropriateHandler_ProjectCacheEvictionWithSlash()
+      throws Exception {
+    final CacheEvictionEvent event = new CacheEvictionEvent("cache", "some/project");
+    router.route(event);
+
+    verify(cacheEvictionHandler).evict(CacheEntry.from(event.cacheName, event.key));
+  }
 }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/forwarder/GsonParserTest.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/forwarder/GsonParserTest.java
index 19ae621..12bdb74 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/multisite/forwarder/GsonParserTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/forwarder/GsonParserTest.java
@@ -53,8 +53,7 @@
   @Test
   public void stringParse() {
     String key = "key";
-    String json = gson.toJson(key);
-    assertThat(key).isEqualTo(gsonParser.fromJson(Constants.PROJECTS, json));
+    assertThat(key).isEqualTo(gsonParser.fromJson(Constants.PROJECTS, key));
   }
 
   @Test
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/validation/ProjectVersionRefUpdateTest.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/validation/ProjectVersionRefUpdateTest.java
new file mode 100644
index 0000000..776fec8
--- /dev/null
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/validation/ProjectVersionRefUpdateTest.java
@@ -0,0 +1,228 @@
+// Copyright (C) 2020 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.multisite.validation;
+
+import static com.google.common.truth.Truth.assertThat;
+import static com.googlesource.gerrit.plugins.multisite.validation.ProjectVersionRefUpdate.MULTI_SITE_VERSIONING_REF;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.*;
+
+import com.google.gerrit.entities.Project;
+import com.google.gerrit.server.events.RefUpdatedEvent;
+import com.google.gerrit.server.project.ProjectConfig;
+import com.google.gerrit.testing.InMemoryRepositoryManager;
+import com.google.gerrit.testing.InMemoryTestEnvironment;
+import com.google.inject.Inject;
+import com.googlesource.gerrit.plugins.multisite.SharedRefDatabaseWrapper;
+import com.googlesource.gerrit.plugins.multisite.forwarder.Context;
+import com.googlesource.gerrit.plugins.multisite.validation.dfsrefdb.RefFixture;
+import com.googlesource.gerrit.plugins.replication.RefReplicatedEvent;
+import com.googlesource.gerrit.plugins.replication.ReplicationState;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Optional;
+import org.apache.commons.io.IOUtils;
+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.ObjectLoader;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.revwalk.RevCommit;
+import org.eclipse.jgit.transport.RemoteRefUpdate;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class ProjectVersionRefUpdateTest implements RefFixture {
+
+  @Rule public InMemoryTestEnvironment testEnvironment = new InMemoryTestEnvironment();
+
+  @Mock RefUpdatedEvent refUpdatedEvent;
+  @Mock SharedRefDatabaseWrapper sharedRefDb;
+
+  @Inject private ProjectConfig.Factory projectConfigFactory;
+  @Inject private InMemoryRepositoryManager repoManager;
+
+  private TestRepository<InMemoryRepository> repo;
+  private ProjectConfig project;
+  private RevCommit masterCommit;
+
+  @Before
+  public void setUp() throws Exception {
+    InMemoryRepository inMemoryRepo = repoManager.createRepository(A_TEST_PROJECT_NAME_KEY);
+    project = projectConfigFactory.create(A_TEST_PROJECT_NAME_KEY);
+    project.load(inMemoryRepo);
+    repo = new TestRepository<>(inMemoryRepo);
+    masterCommit = repo.branch("master").commit().create();
+  }
+
+  @After
+  public void tearDown() {
+    Context.unsetForwardedEvent();
+  }
+
+  @Test
+  public void producerShouldUpdateProjectVersionUponRefUpdatedEvent() throws IOException {
+    Context.setForwardedEvent(false);
+    when(sharedRefDb.compareAndPut(any(Project.NameKey.class), any(Ref.class), any(ObjectId.class)))
+        .thenReturn(true);
+    when(refUpdatedEvent.getProjectNameKey()).thenReturn(A_TEST_PROJECT_NAME_KEY);
+    when(refUpdatedEvent.getRefName()).thenReturn(A_TEST_REF_NAME);
+
+    new ProjectVersionRefUpdate(repoManager, sharedRefDb).onEvent(refUpdatedEvent);
+
+    Ref ref = repo.getRepository().findRef(MULTI_SITE_VERSIONING_REF);
+
+    verify(sharedRefDb, atMost(1))
+        .compareAndPut(any(Project.NameKey.class), any(Ref.class), any(ObjectId.class));
+
+    assertThat(ref).isNotNull();
+
+    ObjectLoader loader = repo.getRepository().open(ref.getObjectId());
+    String storedVersion = IOUtils.toString(loader.openStream(), StandardCharsets.UTF_8.name());
+    assertThat(Long.parseLong(storedVersion)).isEqualTo(masterCommit.getCommitTime());
+  }
+
+  @Test
+  public void producerShouldNotUpdateProjectVersionUponSequenceRefUpdatedEvent()
+      throws IOException {
+    Context.setForwardedEvent(false);
+    when(refUpdatedEvent.getProjectNameKey()).thenReturn(A_TEST_PROJECT_NAME_KEY);
+    when(refUpdatedEvent.getRefName()).thenReturn("refs/sequences/changes");
+
+    new ProjectVersionRefUpdate(repoManager, sharedRefDb).onEvent(refUpdatedEvent);
+
+    Ref ref = repo.getRepository().findRef(MULTI_SITE_VERSIONING_REF);
+    assertThat(ref).isNull();
+  }
+
+  @Test
+  public void shouldNotUpdateProjectVersionWhenProjectDoesntExist() throws IOException {
+    Context.setForwardedEvent(false);
+    when(refUpdatedEvent.getProjectNameKey()).thenReturn(Project.nameKey("aNonExistentProject"));
+    when(refUpdatedEvent.getRefName()).thenReturn(A_TEST_REF_NAME);
+
+    new ProjectVersionRefUpdate(repoManager, sharedRefDb).onEvent(refUpdatedEvent);
+
+    Ref ref = repo.getRepository().findRef(MULTI_SITE_VERSIONING_REF);
+    assertThat(ref).isNull();
+  }
+
+  @Test
+  public void consumerShouldUpdateProjectVersionUponRefReplicatedEvent() throws IOException {
+    Context.setForwardedEvent(true);
+    RefReplicatedEvent refReplicatedEvent =
+        new RefReplicatedEvent(
+            A_TEST_PROJECT_NAME,
+            A_TEST_REF_NAME,
+            "targetNode",
+            ReplicationState.RefPushResult.SUCCEEDED,
+            RemoteRefUpdate.Status.OK);
+
+    new ProjectVersionRefUpdate(repoManager, sharedRefDb).onEvent(refReplicatedEvent);
+
+    Ref ref = repo.getRepository().findRef(MULTI_SITE_VERSIONING_REF);
+    assertThat(ref).isNotNull();
+
+    verify(sharedRefDb, never())
+        .compareAndPut(any(Project.NameKey.class), any(Ref.class), any(ObjectId.class));
+
+    ObjectLoader loader = repo.getRepository().open(ref.getObjectId());
+    String storedVersion = IOUtils.toString(loader.openStream(), StandardCharsets.UTF_8.name());
+    assertThat(Long.parseLong(storedVersion))
+        .isEqualTo(Integer.toUnsignedLong(masterCommit.getCommitTime()));
+  }
+
+  @Test
+  public void consumerShouldNotUpdateProjectVersionUponFailedRefReplicatedEvent()
+      throws IOException {
+    Context.setForwardedEvent(true);
+    RefReplicatedEvent refReplicatedEvent =
+        new RefReplicatedEvent(
+            A_TEST_PROJECT_NAME,
+            A_TEST_REF_NAME,
+            "targetNode",
+            ReplicationState.RefPushResult.SUCCEEDED,
+            RemoteRefUpdate.Status.REJECTED_OTHER_REASON);
+
+    new ProjectVersionRefUpdate(repoManager, sharedRefDb).onEvent(refReplicatedEvent);
+
+    Ref ref = repo.getRepository().findRef(MULTI_SITE_VERSIONING_REF);
+    assertThat(ref).isNull();
+  }
+
+  @Test
+  public void consumerShouldNotUpdateProjectVersionUponSequenceRefReplicatedEvent()
+      throws IOException {
+    Context.setForwardedEvent(true);
+    RefReplicatedEvent refReplicatedEvent =
+        new RefReplicatedEvent(
+            A_TEST_PROJECT_NAME,
+            "refs/sequences/groups",
+            "targetNode",
+            ReplicationState.RefPushResult.SUCCEEDED,
+            RemoteRefUpdate.Status.OK);
+
+    new ProjectVersionRefUpdate(repoManager, sharedRefDb).onEvent(refReplicatedEvent);
+
+    Ref ref = repo.getRepository().findRef(MULTI_SITE_VERSIONING_REF);
+    assertThat(ref).isNull();
+  }
+
+  @Test
+  public void getRemoteProjectVersionShouldReturnCorrectValue() throws IOException {
+    updateLocalVersion();
+    Ref ref = repo.getRepository().findRef(MULTI_SITE_VERSIONING_REF);
+    when(sharedRefDb.get(A_TEST_PROJECT_NAME_KEY, MULTI_SITE_VERSIONING_REF, ObjectId.class))
+        .thenReturn(Optional.of(ref.getObjectId()));
+
+    Optional<Long> version =
+        new ProjectVersionRefUpdate(repoManager, sharedRefDb)
+            .getProjectRemoteVersion(A_TEST_PROJECT_NAME);
+
+    assertThat(version.isPresent()).isTrue();
+    assertThat(version.get()).isEqualTo(masterCommit.getCommitTime());
+  }
+
+  @Test
+  public void getLocalProjectVersionShouldReturnCorrectValue() throws IOException {
+    updateLocalVersion();
+    Ref ref = repo.getRepository().findRef(MULTI_SITE_VERSIONING_REF);
+
+    Optional<Long> version =
+        new ProjectVersionRefUpdate(repoManager, sharedRefDb)
+            .getProjectLocalVersion(A_TEST_PROJECT_NAME);
+
+    assertThat(version.isPresent()).isTrue();
+    assertThat(version.get()).isEqualTo(masterCommit.getCommitTime());
+  }
+
+  private void updateLocalVersion() {
+    Context.setForwardedEvent(true);
+    RefReplicatedEvent refReplicatedEvent =
+        new RefReplicatedEvent(
+            A_TEST_PROJECT_NAME,
+            A_TEST_REF_NAME,
+            "targetNode",
+            ReplicationState.RefPushResult.SUCCEEDED,
+            RemoteRefUpdate.Status.OK);
+    new ProjectVersionRefUpdate(repoManager, sharedRefDb).onEvent(refReplicatedEvent);
+  }
+}
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/validation/dfsrefdb/MultisiteReplicationPushFilterTest.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/validation/dfsrefdb/MultisiteReplicationPushFilterTest.java
index 72e6431..5242ae8 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/multisite/validation/dfsrefdb/MultisiteReplicationPushFilterTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/validation/dfsrefdb/MultisiteReplicationPushFilterTest.java
@@ -31,6 +31,7 @@
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Optional;
 import java.util.Set;
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.lib.ObjectIdRef;
@@ -118,6 +119,13 @@
           }
 
           @Override
+          public <T> boolean compareAndPut(
+              Project.NameKey project, String refName, T currValue, T newValue)
+              throws GlobalRefDbSystemError {
+            return false;
+          }
+
+          @Override
           public AutoCloseable lockRef(Project.NameKey project, String refName)
               throws GlobalRefDbLockException {
             return null;
@@ -125,6 +133,12 @@
 
           @Override
           public void remove(Project.NameKey project) throws GlobalRefDbSystemError {}
+
+          @Override
+          public <T> Optional<T> get(Project.NameKey project, String refName, Class<T> clazz)
+              throws GlobalRefDbSystemError {
+            return Optional.empty();
+          }
         };
     return new SharedRefDatabaseWrapper(
         DynamicItem.itemOf(GlobalRefDatabase.class, sharedRefDatabase),