Merge branch 'stable-2.16' into stable-3.0

* stable-2.16:
  Honour index retries when indexing groups

Change-Id: I2509cb30a6e79f69848b9eae308a40f1e51f839f
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/ForwardedIndexGroupHandler.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/ForwardedIndexGroupHandler.java
index 368dffe..f9423b5 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/ForwardedIndexGroupHandler.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/ForwardedIndexGroupHandler.java
@@ -20,7 +20,12 @@
 import com.google.inject.Singleton;
 import com.googlesource.gerrit.plugins.multisite.Configuration;
 import com.googlesource.gerrit.plugins.multisite.forwarder.events.GroupIndexEvent;
+import com.googlesource.gerrit.plugins.multisite.index.ForwardedIndexExecutor;
+import com.googlesource.gerrit.plugins.multisite.index.GroupChecker;
 import java.util.Optional;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Index a group using {@link GroupIndexer}. This class is meant to be used on the receiving side of
@@ -31,17 +36,74 @@
 @Singleton
 public class ForwardedIndexGroupHandler extends ForwardedIndexingHandler<String, GroupIndexEvent> {
   private final GroupIndexer indexer;
+  private final GroupChecker groupChecker;
+  private final ScheduledExecutorService indexExecutor;
+  private final int retryInterval;
+  private final int maxTries;
 
   @Inject
-  ForwardedIndexGroupHandler(GroupIndexer indexer, Configuration config) {
+  ForwardedIndexGroupHandler(
+      GroupIndexer indexer,
+      Configuration config,
+      GroupChecker groupChecker,
+      @ForwardedIndexExecutor ScheduledExecutorService indexExecutor) {
     super(config.index().numStripedLocks());
     this.indexer = indexer;
+    this.groupChecker = groupChecker;
+    this.indexExecutor = indexExecutor;
+    Configuration.Index indexConfig = config.index();
+    this.retryInterval = indexConfig != null ? indexConfig.retryInterval() : 0;
+    this.maxTries = indexConfig != null ? indexConfig.maxTries() : 0;
   }
 
   @Override
   protected void doIndex(String uuid, Optional<GroupIndexEvent> event) {
+    doIndex(uuid, event, 0);
+  }
+
+  protected void doIndex(String uuid, Optional<GroupIndexEvent> groupIndexEvent, int retryCount) {
     indexer.index(new AccountGroup.UUID(uuid));
-    log.debug("Group {} successfully indexed", uuid);
+    if (groupChecker.isGroupUpToDate(groupIndexEvent)) {
+      if (retryCount > 0) {
+        log.warn("Group '{}' has been eventually indexed after {} attempt(s)", uuid, retryCount);
+      } else {
+        log.debug("Group '{}' successfully indexed", uuid);
+      }
+    } else {
+      log.debug("Group '{}' rescheduling indexing", uuid);
+      rescheduleIndex(uuid, groupIndexEvent, retryCount + 1);
+    }
+  }
+
+  private boolean rescheduleIndex(
+      String uuid, Optional<GroupIndexEvent> indexEvent, int retryCount) {
+    if (retryCount > maxTries) {
+      log.error(
+          "Group '{}' could not be indexed after {} retries. Group index could be stale.",
+          uuid,
+          retryCount);
+      return false;
+    }
+
+    log.warn(
+        "Retrying for the #{} time to index Group {} after {} msecs",
+        retryCount,
+        uuid,
+        retryInterval);
+    @SuppressWarnings("unused")
+    Future<?> possiblyIgnoredError =
+        indexExecutor.schedule(
+            () -> {
+              try {
+                Context.setForwardedEvent(true);
+                doIndex(uuid, indexEvent, retryCount);
+              } catch (Exception e) {
+                log.warn("Group {} could not be indexed", uuid, e);
+              }
+            },
+            retryInterval,
+            TimeUnit.MILLISECONDS);
+    return true;
   }
 
   @Override
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/events/GroupIndexEvent.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/events/GroupIndexEvent.java
index e452e27..4981b2f 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/events/GroupIndexEvent.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/forwarder/events/GroupIndexEvent.java
@@ -15,27 +15,30 @@
 package com.googlesource.gerrit.plugins.multisite.forwarder.events;
 
 import com.google.common.base.Objects;
+import com.google.gerrit.common.Nullable;
+import org.eclipse.jgit.lib.ObjectId;
 
 public class GroupIndexEvent extends IndexEvent {
   static final String TYPE = "group-index";
 
-  public String groupUUID;
+  public final String groupUUID;
+  public final ObjectId sha1;
 
-  public GroupIndexEvent(String groupUUID) {
+  public GroupIndexEvent(String groupUUID, @Nullable ObjectId sha1) {
     super(TYPE);
     this.groupUUID = groupUUID;
+    this.sha1 = sha1;
   }
 
-  @Override
   public boolean equals(Object o) {
     if (this == o) return true;
     if (o == null || getClass() != o.getClass()) return false;
     GroupIndexEvent that = (GroupIndexEvent) o;
-    return Objects.equal(groupUUID, that.groupUUID);
+    return Objects.equal(groupUUID, that.groupUUID) && Objects.equal(sha1, that.sha1);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hashCode(groupUUID);
+    return Objects.hashCode(groupUUID, sha1);
   }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/index/GroupChecker.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/index/GroupChecker.java
new file mode 100644
index 0000000..03ac30c
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/index/GroupChecker.java
@@ -0,0 +1,25 @@
+// 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.multisite.index;
+
+import com.googlesource.gerrit.plugins.multisite.forwarder.events.GroupIndexEvent;
+import java.util.Optional;
+import org.eclipse.jgit.lib.ObjectId;
+
+public interface GroupChecker {
+  boolean isGroupUpToDate(Optional<GroupIndexEvent> groupIndexEvent);
+
+  ObjectId getGroupHead(String groupUUID);
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/index/GroupCheckerImpl.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/index/GroupCheckerImpl.java
new file mode 100644
index 0000000..5e2a039
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/index/GroupCheckerImpl.java
@@ -0,0 +1,102 @@
+// 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.multisite.index;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.flogger.FluentLogger;
+import com.google.gerrit.reviewdb.client.AccountGroup;
+import com.google.gerrit.reviewdb.client.RefNames;
+import com.google.gerrit.server.config.AllUsersName;
+import com.google.gerrit.server.git.GitRepositoryManager;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.googlesource.gerrit.plugins.multisite.forwarder.events.GroupIndexEvent;
+import java.io.IOException;
+import java.util.Optional;
+import org.eclipse.jgit.errors.MissingObjectException;
+import org.eclipse.jgit.lib.ObjectId;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.lib.Repository;
+import org.eclipse.jgit.revwalk.RevWalk;
+
+@Singleton
+class GroupCheckerImpl implements GroupChecker {
+  private static final FluentLogger logger = FluentLogger.forEnclosingClass();
+  private final GitRepositoryManager repoManager;
+  private final AllUsersName allUsers;
+
+  @Inject
+  GroupCheckerImpl(GitRepositoryManager repoManager, AllUsersName allUsers) {
+    this.repoManager = repoManager;
+    this.allUsers = allUsers;
+  }
+
+  @Override
+  public boolean isGroupUpToDate(Optional<GroupIndexEvent> groupIndexEvent) {
+    if (!groupIndexEvent.isPresent()) {
+      logger.atWarning().log("Group Index empty, considering this group up-to-date");
+      return true;
+    }
+    GroupIndexEvent event = groupIndexEvent.get();
+    AccountGroup.UUID groupUUID = new AccountGroup.UUID(event.groupUUID);
+
+    if (event.sha1 == null) {
+      logger.atWarning().log(
+          "Event for group '%s' does not contain sha1, consider group up-to-date for compatibility.",
+          groupUUID);
+      return true;
+    }
+
+    try (Repository repo = repoManager.openRepository(allUsers)) {
+      if (commitExistsInRepo(repo, event.sha1)) {
+        logger.atInfo().log(
+            "Group '%s' up-to-date: sha1 '%s' exists in All-Users", groupUUID, event.sha1);
+        return true;
+      } else {
+        logger.atWarning().log(
+            "Group '%s' NOT up-to-date: sha1 '%s' still missing in All-Users",
+            groupUUID, event.sha1);
+      }
+    } catch (Exception e) {
+      logger.atSevere().withCause(e).log(
+          "Could not check whether Group '%s' is up-to-date", groupUUID);
+    }
+    return false;
+  }
+
+  @Override
+  public ObjectId getGroupHead(String groupUUID) {
+    try (Repository repo = repoManager.openRepository(allUsers)) {
+      return Optional.ofNullable(
+              repo.exactRef(RefNames.refsGroups(new AccountGroup.UUID(groupUUID))))
+          .map(Ref::getObjectId)
+          .orElse(ObjectId.zeroId());
+    } catch (Exception e) {
+      logger.atSevere().withCause(e).log("Fatal: could not get head of group %s.", groupUUID);
+      return ObjectId.zeroId();
+    }
+  }
+
+  @VisibleForTesting
+  boolean commitExistsInRepo(Repository repo, ObjectId sha1) throws IOException {
+    try (RevWalk revWalk = new RevWalk(repo)) {
+      revWalk.parseCommit(sha1);
+      return true;
+    } catch (MissingObjectException e) {
+      logger.atWarning().log("Commit %s does not exist in All-Users", sha1);
+    }
+    return false;
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/index/IndexEventHandler.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/index/IndexEventHandler.java
index ed33efa..649074b 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/index/IndexEventHandler.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/index/IndexEventHandler.java
@@ -44,15 +44,18 @@
   private final DynamicSet<IndexEventForwarder> forwarders;
   private final Set<IndexTask> queuedTasks = Collections.newSetFromMap(new ConcurrentHashMap<>());
   private final ChangeCheckerImpl.Factory changeChecker;
+  private final GroupChecker groupChecker;
 
   @Inject
   IndexEventHandler(
       @IndexExecutor Executor executor,
       DynamicSet<IndexEventForwarder> forwarders,
-      ChangeCheckerImpl.Factory changeChecker) {
+      ChangeCheckerImpl.Factory changeChecker,
+      GroupChecker groupChecker) {
     this.forwarders = forwarders;
     this.executor = executor;
     this.changeChecker = changeChecker;
+    this.groupChecker = groupChecker;
   }
 
   @Override
@@ -78,7 +81,8 @@
   @Override
   public void onGroupIndexed(String groupUUID) {
     if (!Context.isForwardedEvent()) {
-      IndexGroupTask task = new IndexGroupTask(new GroupIndexEvent(groupUUID));
+      IndexGroupTask task =
+          new IndexGroupTask(new GroupIndexEvent(groupUUID, groupChecker.getGroupHead(groupUUID)));
       if (queuedTasks.add(task)) {
         executor.execute(task);
       }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/multisite/index/IndexModule.java b/src/main/java/com/googlesource/gerrit/plugins/multisite/index/IndexModule.java
index 1f54385..075388f 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/multisite/index/IndexModule.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/multisite/index/IndexModule.java
@@ -39,6 +39,7 @@
     DynamicSet.bind(binder(), ProjectIndexedListener.class).to(IndexEventHandler.class);
 
     bind(ProjectChecker.class).to(ProjectCheckerImpl.class);
+    bind(GroupChecker.class).to(GroupCheckerImpl.class);
 
     install(
         new FactoryModuleBuilder()
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/event/IndexEventRouterTest.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/event/IndexEventRouterTest.java
index bf6f043..dd9b20d 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/multisite/event/IndexEventRouterTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/event/IndexEventRouterTest.java
@@ -34,6 +34,7 @@
 import com.googlesource.gerrit.plugins.multisite.forwarder.router.StreamEventRouter;
 import com.googlesource.gerrit.plugins.replication.RefReplicationDoneEvent;
 import java.util.Optional;
+import org.eclipse.jgit.lib.ObjectId;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -95,7 +96,7 @@
   @Test
   public void routerShouldSendEventsToTheAppropriateHandler_GroupIndex() throws Exception {
     final String groupId = "12";
-    final GroupIndexEvent event = new GroupIndexEvent(groupId);
+    final GroupIndexEvent event = new GroupIndexEvent(groupId, ObjectId.zeroId());
     router.route(event);
 
     verify(indexGroupHandler)
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/forwarder/ForwardedIndexGroupHandlerTest.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/forwarder/ForwardedIndexGroupHandlerTest.java
index 9013646..b333af0 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/multisite/forwarder/ForwardedIndexGroupHandlerTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/forwarder/ForwardedIndexGroupHandlerTest.java
@@ -16,6 +16,8 @@
 
 import static com.google.common.truth.Truth.assertThat;
 import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -24,8 +26,12 @@
 import com.google.gerrit.server.index.group.GroupIndexer;
 import com.googlesource.gerrit.plugins.multisite.Configuration;
 import com.googlesource.gerrit.plugins.multisite.forwarder.ForwardedIndexingHandler.Operation;
+import com.googlesource.gerrit.plugins.multisite.forwarder.events.GroupIndexEvent;
+import com.googlesource.gerrit.plugins.multisite.index.TestGroupChecker;
 import java.io.IOException;
 import java.util.Optional;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -40,16 +46,21 @@
 
   @Rule public ExpectedException exception = ExpectedException.none();
   @Mock private GroupIndexer indexerMock;
+  @Mock private ScheduledExecutorService indexExecutorMock;
   @Mock private Configuration config;
   @Mock private Configuration.Index index;
   private ForwardedIndexGroupHandler handler;
   private String uuid;
+  private static final int RETRY_INTERVAL = 1000;
+  private static final int MAX_TRIES = 2;
 
   @Before
   public void setUp() throws Exception {
     when(config.index()).thenReturn(index);
     when(index.numStripedLocks()).thenReturn(10);
-    handler = new ForwardedIndexGroupHandler(indexerMock, config);
+    when(index.retryInterval()).thenReturn(RETRY_INTERVAL);
+    when(index.maxTries()).thenReturn(MAX_TRIES);
+    handler = groupHandler(true);
     uuid = "123";
   }
 
@@ -108,4 +119,28 @@
 
     verify(indexerMock).index(new AccountGroup.UUID(uuid));
   }
+
+  @Test
+  public void shouldChangeIndexEventWheNotUpToDate() throws IOException {
+    ForwardedIndexGroupHandler groupHandlerWithOutdatedEvent = groupHandler(false);
+    groupHandlerWithOutdatedEvent.index(uuid, Operation.INDEX, groupIndexEvent(uuid));
+    verify(indexerMock).index(new AccountGroup.UUID(uuid));
+  }
+
+  @Test
+  public void shouldRescheduleGroupIndexingWhenItIsNotUpToDate() throws IOException {
+    ForwardedIndexGroupHandler groupHandlerWithOutdatedEvent = groupHandler(false);
+    groupHandlerWithOutdatedEvent.index(uuid, Operation.INDEX, groupIndexEvent(uuid));
+    verify(indexExecutorMock)
+        .schedule(any(Runnable.class), eq(new Long(RETRY_INTERVAL)), eq(TimeUnit.MILLISECONDS));
+  }
+
+  private ForwardedIndexGroupHandler groupHandler(boolean checkIsUpToDate) {
+    return new ForwardedIndexGroupHandler(
+        indexerMock, config, new TestGroupChecker(checkIsUpToDate), indexExecutorMock);
+  }
+
+  private Optional<GroupIndexEvent> groupIndexEvent(String uuid) {
+    return Optional.of(new GroupIndexEvent(uuid, null));
+  }
 }
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/index/GroupCheckerImplTest.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/index/GroupCheckerImplTest.java
new file mode 100644
index 0000000..653cffa
--- /dev/null
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/index/GroupCheckerImplTest.java
@@ -0,0 +1,126 @@
+// 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.multisite.index;
+
+import static com.google.common.truth.Truth.assertThat;
+import static org.mockito.Mockito.doReturn;
+
+import com.google.gerrit.common.Nullable;
+import com.google.gerrit.reviewdb.client.AccountGroup;
+import com.google.gerrit.reviewdb.client.RefNames;
+import com.google.gerrit.server.config.AllUsersName;
+import com.google.gerrit.server.config.AllUsersNameProvider;
+import com.google.gerrit.server.git.GitRepositoryManager;
+import com.googlesource.gerrit.plugins.multisite.forwarder.events.GroupIndexEvent;
+import java.io.IOException;
+import java.util.Optional;
+import java.util.UUID;
+import org.eclipse.jgit.lib.ObjectId;
+import org.eclipse.jgit.lib.ObjectIdRef;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.lib.RefDatabase;
+import org.eclipse.jgit.lib.Repository;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class GroupCheckerImplTest {
+  ObjectId AN_OBJECT_ID = ObjectId.fromString("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa");
+  AllUsersName allUsers = new AllUsersName(AllUsersNameProvider.DEFAULT);
+
+  GroupCheckerImpl objectUnderTest;
+  @Mock private GitRepositoryManager repoManagerMock;
+  @Mock private RefDatabase refDatabaseMock;
+  @Mock private Repository repoMock;
+
+  @Before
+  public void setUp() throws Exception {
+    doReturn(repoMock).when(repoManagerMock).openRepository(allUsers);
+    doReturn(refDatabaseMock).when(repoMock).getRefDatabase();
+    objectUnderTest = new GroupCheckerImpl(repoManagerMock, allUsers);
+  }
+
+  @Test
+  public void isGroupUpToDate_shouldReturnTrueWhenEventIsEmpty() {
+    assertThat(objectUnderTest.isGroupUpToDate(Optional.empty())).isTrue();
+  }
+
+  @Test
+  public void isGroupUpToDate_shouldReturnFalseWhenSha1DoesNotExistInAllUsers() {
+    setCommitExistsInRepo(false);
+    assertThat(
+            objectUnderTest.isGroupUpToDate(
+                groupIndexEvent(UUID.randomUUID().toString(), AN_OBJECT_ID)))
+        .isFalse();
+  }
+
+  @Test
+  public void isGroupUpToDate_shouldReturnFalseWhenSha1ExistsInAllUsers() {
+    setCommitExistsInRepo(true);
+    assertThat(
+            objectUnderTest.isGroupUpToDate(
+                groupIndexEvent(UUID.randomUUID().toString(), AN_OBJECT_ID)))
+        .isTrue();
+  }
+
+  @Test
+  public void isGroupUpToDate_shouldReturnTrueWhenSha1IsNotDefined() {
+    UUID groupUUID = UUID.randomUUID();
+    setCommitExistsInRepo(true);
+
+    assertThat(objectUnderTest.isGroupUpToDate(groupIndexEvent(groupUUID.toString(), null)))
+        .isTrue();
+  }
+
+  @Test
+  public void getGroupHead_shouldReturnTheExactReValueWhenDefined() throws IOException {
+    UUID groupUUID = UUID.randomUUID();
+    setupExactRefInGroup(groupUUID, AN_OBJECT_ID);
+
+    assertThat(objectUnderTest.getGroupHead(groupUUID.toString())).isEqualTo(AN_OBJECT_ID);
+  }
+
+  @Test
+  public void getGroupHead_shouldReturnObjectIdZeroWhenExactRefIsNull() throws IOException {
+    UUID groupUUID = UUID.randomUUID();
+    setupExactRefInGroup(groupUUID, null);
+
+    assertThat(objectUnderTest.getGroupHead(groupUUID.toString())).isEqualTo(ObjectId.zeroId());
+  }
+
+  private Optional<GroupIndexEvent> groupIndexEvent(String uuid, @Nullable ObjectId sha1) {
+    return Optional.of(new GroupIndexEvent(uuid, sha1));
+  }
+
+  private void setCommitExistsInRepo(boolean commitExists) {
+    objectUnderTest =
+        new GroupCheckerImpl(repoManagerMock, allUsers) {
+          @Override
+          boolean commitExistsInRepo(Repository repo, ObjectId sha1) {
+            return commitExists;
+          }
+        };
+  }
+
+  private void setupExactRefInGroup(UUID groupUUID, @Nullable ObjectId objectId)
+      throws IOException {
+    String groupRefName = RefNames.refsGroups(new AccountGroup.UUID(groupUUID.toString()));
+    ObjectIdRef.Unpeeled aRef = new ObjectIdRef.Unpeeled(Ref.Storage.LOOSE, groupRefName, objectId);
+    doReturn(objectId == null ? null : aRef).when(refDatabaseMock).exactRef(groupRefName);
+  }
+}
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/index/GroupEventIndexTest.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/index/GroupEventIndexTest.java
new file mode 100644
index 0000000..93cec05
--- /dev/null
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/index/GroupEventIndexTest.java
@@ -0,0 +1,45 @@
+// 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.multisite.index;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.gerritforge.gerrit.eventbroker.EventGsonProvider;
+import com.google.gson.Gson;
+import com.googlesource.gerrit.plugins.multisite.forwarder.events.GroupIndexEvent;
+import java.util.UUID;
+import org.eclipse.jgit.lib.ObjectId;
+import org.junit.Test;
+
+public class GroupEventIndexTest {
+  private static final Gson gson = new EventGsonProvider().get();
+
+  @Test
+  public void groupEventIndexRoundTripWithSha1() {
+    String aGroupUUID = UUID.randomUUID().toString();
+    ObjectId anObjectId = ObjectId.fromString("deadbeefdeadbeefdeadbeefdeadbeefdeadbeef");
+    GroupIndexEvent original = new GroupIndexEvent(aGroupUUID, anObjectId);
+
+    assertThat(gson.fromJson(gson.toJson(original), GroupIndexEvent.class)).isEqualTo(original);
+  }
+
+  @Test
+  public void groupEventIndexRoundTripWithoutSha1() {
+    String aGroupUUID = UUID.randomUUID().toString();
+    GroupIndexEvent original = new GroupIndexEvent(aGroupUUID, null);
+
+    assertThat(gson.fromJson(gson.toJson(original), GroupIndexEvent.class)).isEqualTo(original);
+  }
+}
diff --git a/src/test/java/com/googlesource/gerrit/plugins/multisite/index/TestGroupChecker.java b/src/test/java/com/googlesource/gerrit/plugins/multisite/index/TestGroupChecker.java
new file mode 100644
index 0000000..a6c2675
--- /dev/null
+++ b/src/test/java/com/googlesource/gerrit/plugins/multisite/index/TestGroupChecker.java
@@ -0,0 +1,42 @@
+// Copyright (C) 2018 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.index;
+
+import com.googlesource.gerrit.plugins.multisite.forwarder.events.GroupIndexEvent;
+import java.util.Optional;
+import org.eclipse.jgit.lib.ObjectId;
+import org.junit.Ignore;
+
+@Ignore
+public class TestGroupChecker implements GroupChecker {
+
+  private final boolean isUpToDate;
+
+  public TestGroupChecker(boolean isUpToDate) {
+    this.isUpToDate = isUpToDate;
+  }
+
+  private static final String someObjectId = "deadbeefdeadbeefdeadbeefdeadbeefdeadbeef";
+
+  @Override
+  public boolean isGroupUpToDate(Optional<GroupIndexEvent> groupIndexEvent) {
+    return isUpToDate;
+  }
+
+  @Override
+  public ObjectId getGroupHead(String groupUUID) {
+    return ObjectId.fromString(someObjectId);
+  }
+}