Merge branch 'stable-2.15'

* stable-2.15:
  StartCommand: Reformat with google-java-format 1.6

Change-Id: I8375514117f8cf5476400c99973630cd2a931315
diff --git a/BUILD b/BUILD
index 41089c6..7ba3d26 100644
--- a/BUILD
+++ b/BUILD
@@ -14,7 +14,7 @@
     ],
     resources = glob(["src/main/resources/**/*"]),
     deps = [
-        "//lib:commons-io",
+        "//lib/commons:io",
     ],
 )
 
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/AutoReloadConfigDecorator.java b/src/main/java/com/googlesource/gerrit/plugins/replication/AutoReloadConfigDecorator.java
index 8b6b8fc..766be73 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/AutoReloadConfigDecorator.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/AutoReloadConfigDecorator.java
@@ -14,11 +14,13 @@
 package com.googlesource.gerrit.plugins.replication;
 
 import com.google.gerrit.common.FileUtil;
+import com.google.gerrit.extensions.annotations.PluginData;
 import com.google.gerrit.server.config.SitePaths;
 import com.google.gerrit.server.git.WorkQueue;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.List;
 import org.eclipse.jgit.errors.ConfigInvalidException;
 import org.slf4j.Logger;
@@ -33,13 +35,18 @@
   private final SitePaths site;
   private final WorkQueue workQueue;
   private final DestinationFactory destinationFactory;
+  private final Path pluginDataDir;
 
   @Inject
   public AutoReloadConfigDecorator(
-      SitePaths site, WorkQueue workQueue, DestinationFactory destinationFactory)
+      SitePaths site,
+      WorkQueue workQueue,
+      DestinationFactory destinationFactory,
+      @PluginData Path pluginDataDir)
       throws ConfigInvalidException, IOException {
     this.site = site;
     this.destinationFactory = destinationFactory;
+    this.pluginDataDir = pluginDataDir;
     this.currentConfig = loadConfig();
     this.currentConfigTs = getLastModified(currentConfig);
     this.workQueue = workQueue;
@@ -50,7 +57,7 @@
   }
 
   private ReplicationFileBasedConfig loadConfig() throws ConfigInvalidException, IOException {
-    return new ReplicationFileBasedConfig(site, destinationFactory);
+    return new ReplicationFileBasedConfig(site, destinationFactory, pluginDataDir);
   }
 
   private synchronized boolean isAutoReload() {
@@ -102,6 +109,11 @@
   }
 
   @Override
+  public Path getEventsDirectory() {
+    return currentConfig.getEventsDirectory();
+  }
+
+  @Override
   public synchronized int shutdown() {
     return currentConfig.shutdown();
   }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/Destination.java b/src/main/java/com/googlesource/gerrit/plugins/replication/Destination.java
index 0cee37c..b69bab6 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/Destination.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/Destination.java
@@ -24,9 +24,7 @@
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSet.Builder;
 import com.google.common.collect.Lists;
-import com.google.gerrit.common.EventDispatcher;
 import com.google.gerrit.common.data.GroupReference;
-import com.google.gerrit.extensions.client.ProjectState;
 import com.google.gerrit.extensions.config.FactoryModule;
 import com.google.gerrit.extensions.registration.DynamicItem;
 import com.google.gerrit.extensions.restapi.AuthException;
@@ -42,6 +40,7 @@
 import com.google.gerrit.server.account.GroupIncludeCache;
 import com.google.gerrit.server.account.ListGroupMembership;
 import com.google.gerrit.server.config.RequestScopedReviewDbProvider;
+import com.google.gerrit.server.events.EventDispatcher;
 import com.google.gerrit.server.git.GitRepositoryManager;
 import com.google.gerrit.server.git.PerThreadRequestScope;
 import com.google.gerrit.server.git.WorkQueue;
@@ -50,8 +49,8 @@
 import com.google.gerrit.server.permissions.ProjectPermission;
 import com.google.gerrit.server.permissions.RefPermission;
 import com.google.gerrit.server.project.NoSuchProjectException;
-import com.google.gerrit.server.project.PerRequestProjectControlCache;
-import com.google.gerrit.server.project.ProjectControl;
+import com.google.gerrit.server.project.ProjectCache;
+import com.google.gerrit.server.project.ProjectState;
 import com.google.gerrit.server.util.RequestContext;
 import com.google.inject.Injector;
 import com.google.inject.Provider;
@@ -86,9 +85,10 @@
   private final Map<URIish, PushOne> pending = new HashMap<>();
   private final Map<URIish, PushOne> inFlight = new HashMap<>();
   private final PushOne.Factory opFactory;
-  private final ProjectControl.Factory projectControlFactory;
   private final GitRepositoryManager gitManager;
   private final PermissionBackend permissionBackend;
+  private final Provider<CurrentUser> userProvider;
+  private final ProjectCache projectCache;
   private volatile ScheduledExecutorService pool;
   private final PerThreadRequestScope.Scoper threadScoper;
   private final DestinationConfiguration config;
@@ -117,6 +117,8 @@
       PluginUser pluginUser,
       GitRepositoryManager gitRepositoryManager,
       PermissionBackend permissionBackend,
+      Provider<CurrentUser> userProvider,
+      ProjectCache projectCache,
       GroupBackend groupBackend,
       ReplicationStateListener stateLog,
       GroupIncludeCache groupIncludeCache,
@@ -125,6 +127,8 @@
     this.eventDispatcher = eventDispatcher;
     gitManager = gitRepositoryManager;
     this.permissionBackend = permissionBackend;
+    this.userProvider = userProvider;
+    this.projectCache = projectCache;
     this.stateLog = stateLog;
 
     CurrentUser remoteUser;
@@ -151,7 +155,6 @@
               protected void configure() {
                 bindScope(RequestScoped.class, PerThreadRequestScope.REQUEST);
                 bind(PerThreadRequestScope.Propagator.class);
-                bind(PerRequestProjectControlCache.class).in(RequestScoped.class);
 
                 bind(Destination.class).toInstance(Destination.this);
                 bind(RemoteConfig.class).toInstance(config.getRemoteConfig());
@@ -183,7 +186,6 @@
               }
             });
 
-    projectControlFactory = child.getInstance(ProjectControl.Factory.class);
     opFactory = child.getInstance(PushOne.Factory.class);
     threadScoper = child.getInstance(PerThreadRequestScope.Scoper.class);
   }
@@ -223,15 +225,21 @@
     return cnt;
   }
 
-  private boolean shouldReplicate(ProjectControl ctl) throws PermissionBackendException {
-    if (!config.replicateHiddenProjects() && ctl.getProject().getState() == ProjectState.HIDDEN) {
+  private boolean shouldReplicate(ProjectState state, CurrentUser user)
+      throws PermissionBackendException {
+    if (!config.replicateHiddenProjects()
+        && state.getProject().getState()
+            == com.google.gerrit.extensions.client.ProjectState.HIDDEN) {
       return false;
     }
+
+    // Hidden projects(permitsRead = false) should only be accessible by the project owners.
+    // READ_CONFIG is checked here because it's only allowed to project owners(ACCESS may also
+    // be allowed for other users).
+    ProjectPermission permissionToCheck =
+        state.statePermitsRead() ? ProjectPermission.ACCESS : ProjectPermission.READ_CONFIG;
     try {
-      permissionBackend
-          .user(ctl.getUser())
-          .project(ctl.getProject().getNameKey())
-          .check(ProjectPermission.ACCESS);
+      permissionBackend.user(user).project(state.getNameKey()).check(permissionToCheck);
       return true;
     } catch (AuthException e) {
       return false;
@@ -246,8 +254,19 @@
               new Callable<Boolean>() {
                 @Override
                 public Boolean call() throws NoSuchProjectException, PermissionBackendException {
-                  ProjectControl projectControl = controlFor(project);
-                  if (!shouldReplicate(projectControl)) {
+                  ProjectState projectState;
+                  try {
+                    projectState = projectCache.checkedGet(project);
+                  } catch (IOException e) {
+                    return false;
+                  }
+                  if (projectState == null) {
+                    throw new NoSuchProjectException(project);
+                  }
+                  if (!projectState.statePermitsRead()) {
+                    return false;
+                  }
+                  if (!shouldReplicate(projectState, userProvider.get())) {
                     return false;
                   }
                   if (PushOne.ALL_REFS.equals(ref)) {
@@ -255,7 +274,7 @@
                   }
                   try {
                     permissionBackend
-                        .user(projectControl.getUser())
+                        .user(userProvider.get())
                         .project(project)
                         .ref(ref)
                         .check(RefPermission.READ);
@@ -282,7 +301,16 @@
               new Callable<Boolean>() {
                 @Override
                 public Boolean call() throws NoSuchProjectException, PermissionBackendException {
-                  return shouldReplicate(controlFor(project));
+                  ProjectState projectState;
+                  try {
+                    projectState = projectCache.checkedGet(project);
+                  } catch (IOException e) {
+                    return false;
+                  }
+                  if (projectState == null) {
+                    throw new NoSuchProjectException(project);
+                  }
+                  return shouldReplicate(projectState, userProvider.get());
                 }
               })
           .call();
@@ -456,10 +484,6 @@
     }
   }
 
-  ProjectControl controlFor(Project.NameKey project) throws NoSuchProjectException {
-    return projectControlFactory.controlFor(project);
-  }
-
   boolean requestRunway(PushOne op) {
     synchronized (stateLock) {
       if (op.wasCanceled()) {
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/DestinationFactory.java b/src/main/java/com/googlesource/gerrit/plugins/replication/DestinationFactory.java
index 83eab86..5e41e12 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/DestinationFactory.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/DestinationFactory.java
@@ -14,15 +14,18 @@
 
 package com.googlesource.gerrit.plugins.replication;
 
-import com.google.gerrit.common.EventDispatcher;
 import com.google.gerrit.extensions.registration.DynamicItem;
+import com.google.gerrit.server.CurrentUser;
 import com.google.gerrit.server.PluginUser;
 import com.google.gerrit.server.account.GroupBackend;
 import com.google.gerrit.server.account.GroupIncludeCache;
+import com.google.gerrit.server.events.EventDispatcher;
 import com.google.gerrit.server.git.GitRepositoryManager;
 import com.google.gerrit.server.permissions.PermissionBackend;
+import com.google.gerrit.server.project.ProjectCache;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
+import com.google.inject.Provider;
 import com.google.inject.Singleton;
 
 @Singleton
@@ -32,6 +35,8 @@
   private final PluginUser pluginUser;
   private final GitRepositoryManager gitRepositoryManager;
   private final PermissionBackend permissionBackend;
+  private final Provider<CurrentUser> userProvider;
+  private final ProjectCache projectCache;
   private final GroupBackend groupBackend;
   private final ReplicationStateListener stateLog;
   private final GroupIncludeCache groupIncludeCache;
@@ -44,6 +49,8 @@
       PluginUser pluginUser,
       GitRepositoryManager gitRepositoryManager,
       PermissionBackend permissionBackend,
+      Provider<CurrentUser> userProvider,
+      ProjectCache projectCache,
       GroupBackend groupBackend,
       ReplicationStateListener stateLog,
       GroupIncludeCache groupIncludeCache,
@@ -53,6 +60,8 @@
     this.pluginUser = pluginUser;
     this.gitRepositoryManager = gitRepositoryManager;
     this.permissionBackend = permissionBackend;
+    this.userProvider = userProvider;
+    this.projectCache = projectCache;
     this.groupBackend = groupBackend;
     this.stateLog = stateLog;
     this.groupIncludeCache = groupIncludeCache;
@@ -67,6 +76,8 @@
         pluginUser,
         gitRepositoryManager,
         permissionBackend,
+        userProvider,
+        projectCache,
         groupBackend,
         stateLog,
         groupIncludeCache,
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/EventsStorage.java b/src/main/java/com/googlesource/gerrit/plugins/replication/EventsStorage.java
new file mode 100644
index 0000000..d28a1c0
--- /dev/null
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/EventsStorage.java
@@ -0,0 +1,111 @@
+// 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.replication;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import com.google.common.hash.Hashing;
+import com.google.gson.Gson;
+import com.google.inject.Inject;
+import com.google.inject.ProvisionException;
+import com.google.inject.Singleton;
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import org.eclipse.jgit.lib.ObjectId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Singleton
+public class EventsStorage {
+  private static final Logger log = LoggerFactory.getLogger(EventsStorage.class);
+
+  public static class ReplicateRefUpdate {
+    public String project;
+    public String ref;
+  }
+
+  private static Gson GSON = new Gson();
+
+  private final Path refUpdates;
+
+  @Inject
+  EventsStorage(ReplicationConfig config) {
+    refUpdates = config.getEventsDirectory().resolve("ref-updates");
+  }
+
+  public String persist(String project, String ref) {
+    ReplicateRefUpdate r = new ReplicateRefUpdate();
+    r.project = project;
+    r.ref = ref;
+
+    String json = GSON.toJson(r) + "\n";
+    String eventKey = sha1(json).name();
+    Path file = refUpdates().resolve(eventKey);
+
+    if (Files.exists(file)) {
+      return eventKey;
+    }
+
+    try {
+      Files.write(file, json.getBytes(UTF_8));
+    } catch (IOException e) {
+      log.warn("Couldn't persist event {}", json);
+    }
+    return eventKey;
+  }
+
+  public void delete(String eventKey) {
+    if (eventKey != null) {
+      try {
+        Files.delete(refUpdates().resolve(eventKey));
+      } catch (IOException e) {
+        log.error("Error while deleting event {}", eventKey);
+      }
+    }
+  }
+
+  public List<ReplicateRefUpdate> list() {
+    ArrayList<ReplicateRefUpdate> result = new ArrayList<>();
+    try (DirectoryStream<Path> events = Files.newDirectoryStream(refUpdates())) {
+      for (Path e : events) {
+        if (Files.isRegularFile(e)) {
+          String json = new String(Files.readAllBytes(e), UTF_8);
+          result.add(GSON.fromJson(json, ReplicateRefUpdate.class));
+          Files.delete(e);
+        }
+      }
+    } catch (IOException e) {
+      log.error("Error when firing pending events", e);
+    }
+    return result;
+  }
+
+  @SuppressWarnings("deprecation")
+  private ObjectId sha1(String s) {
+    return ObjectId.fromRaw(Hashing.sha1().hashString(s, UTF_8).asBytes());
+  }
+
+  private Path refUpdates() {
+    try {
+      return Files.createDirectories(refUpdates);
+    } catch (IOException e) {
+      throw new ProvisionException(String.format("Couldn't create %s", refUpdates), e);
+    }
+  }
+}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/OnStartStop.java b/src/main/java/com/googlesource/gerrit/plugins/replication/OnStartStop.java
index 227804d..ac0262d 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/OnStartStop.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/OnStartStop.java
@@ -15,12 +15,13 @@
 package com.googlesource.gerrit.plugins.replication;
 
 import com.google.common.util.concurrent.Atomics;
-import com.google.gerrit.common.EventDispatcher;
 import com.google.gerrit.extensions.events.LifecycleListener;
 import com.google.gerrit.extensions.registration.DynamicItem;
 import com.google.gerrit.extensions.systemstatus.ServerInformation;
+import com.google.gerrit.server.events.EventDispatcher;
 import com.google.inject.Inject;
 import com.googlesource.gerrit.plugins.replication.PushResultProcessing.GitUpdateProcessing;
+import com.googlesource.gerrit.plugins.replication.ReplicationState.Factory;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
@@ -29,32 +30,31 @@
   private final AtomicReference<Future<?>> pushAllFuture;
   private final ServerInformation srvInfo;
   private final PushAll.Factory pushAll;
-  private final ReplicationQueue queue;
   private final ReplicationConfig config;
   private final DynamicItem<EventDispatcher> eventDispatcher;
+  private final Factory replicationStateFactory;
 
   @Inject
   protected OnStartStop(
       ServerInformation srvInfo,
       PushAll.Factory pushAll,
-      ReplicationQueue queue,
       ReplicationConfig config,
-      DynamicItem<EventDispatcher> eventDispatcher) {
+      DynamicItem<EventDispatcher> eventDispatcher,
+      ReplicationState.Factory replicationStateFactory) {
     this.srvInfo = srvInfo;
     this.pushAll = pushAll;
-    this.queue = queue;
     this.config = config;
     this.eventDispatcher = eventDispatcher;
+    this.replicationStateFactory = replicationStateFactory;
     this.pushAllFuture = Atomics.newReference();
   }
 
   @Override
   public void start() {
-    queue.start();
-
     if (srvInfo.getState() == ServerInformation.State.STARTUP
         && config.isReplicateAllOnPluginStart()) {
-      ReplicationState state = new ReplicationState(new GitUpdateProcessing(eventDispatcher.get()));
+      ReplicationState state =
+          replicationStateFactory.create(new GitUpdateProcessing(eventDispatcher.get()));
       pushAllFuture.set(
           pushAll
               .create(null, ReplicationFilter.all(), state, false)
@@ -68,6 +68,5 @@
     if (f != null) {
       f.cancel(true);
     }
-    queue.stop();
   }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/PushOne.java b/src/main/java/com/googlesource/gerrit/plugins/replication/PushOne.java
index 1efad4f..220ccad 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/PushOne.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/PushOne.java
@@ -23,19 +23,20 @@
 import com.google.common.collect.Sets;
 import com.google.gerrit.extensions.events.GitReferenceUpdatedListener;
 import com.google.gerrit.extensions.restapi.AuthException;
+import com.google.gerrit.extensions.restapi.ResourceConflictException;
 import com.google.gerrit.metrics.Timer1;
 import com.google.gerrit.reviewdb.client.Project;
 import com.google.gerrit.reviewdb.client.RefNames;
 import com.google.gerrit.server.git.GitRepositoryManager;
 import com.google.gerrit.server.git.PerThreadRequestScope;
 import com.google.gerrit.server.git.ProjectRunnable;
-import com.google.gerrit.server.git.VisibleRefFilter;
 import com.google.gerrit.server.git.WorkQueue.CanceledWhileRunning;
 import com.google.gerrit.server.permissions.PermissionBackend;
+import com.google.gerrit.server.permissions.PermissionBackend.RefFilterOptions;
 import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.google.gerrit.server.permissions.ProjectPermission;
-import com.google.gerrit.server.project.NoSuchProjectException;
-import com.google.gerrit.server.project.ProjectControl;
+import com.google.gerrit.server.project.ProjectCache;
+import com.google.gerrit.server.project.ProjectState;
 import com.google.gerrit.server.util.IdGenerator;
 import com.google.inject.Inject;
 import com.google.inject.assistedinject.Assisted;
@@ -92,7 +93,6 @@
   private final RemoteConfig config;
   private final CredentialsProvider credentialsProvider;
   private final PerThreadRequestScope.Scoper threadScoper;
-  private final VisibleRefFilter.Factory refFilterFactory;
   private final ReplicationQueue replicationQueue;
 
   private final Project.NameKey projectName;
@@ -110,6 +110,7 @@
   private final int id;
   private final long createdAt;
   private final ReplicationMetrics metrics;
+  private final ProjectCache projectCache;
   private final AtomicBoolean canceledWhileRunning;
 
   @Inject
@@ -118,20 +119,19 @@
       PermissionBackend permissionBackend,
       Destination p,
       RemoteConfig c,
-      VisibleRefFilter.Factory rff,
       CredentialsFactory cpFactory,
       PerThreadRequestScope.Scoper ts,
       ReplicationQueue rq,
       IdGenerator ig,
       ReplicationStateListener sl,
       ReplicationMetrics m,
+      ProjectCache pc,
       @Assisted Project.NameKey d,
       @Assisted URIish u) {
     gitManager = grm;
     this.permissionBackend = permissionBackend;
     pool = p;
     config = c;
-    refFilterFactory = rff;
     credentialsProvider = cpFactory.create(c.getName());
     threadScoper = ts;
     replicationQueue = rq;
@@ -143,6 +143,7 @@
     stateLog = sl;
     createdAt = System.nanoTime();
     metrics = m;
+    projectCache = pc;
     canceledWhileRunning = new AtomicBoolean(false);
     maxRetries = p.getMaxRetries();
   }
@@ -452,19 +453,19 @@
 
   private List<RemoteRefUpdate> generateUpdates(Transport tn)
       throws IOException, PermissionBackendException {
-    ProjectControl pc;
-    try {
-      pc = pool.controlFor(projectName);
-    } catch (NoSuchProjectException e) {
+    ProjectState projectState = projectCache.checkedGet(projectName);
+    if (projectState == null) {
       return Collections.emptyList();
     }
 
     Map<String, Ref> local = git.getAllRefs();
     boolean filter;
+    PermissionBackend.ForProject forProject = permissionBackend.currentUser().project(projectName);
     try {
-      permissionBackend.user(pc.getUser()).project(projectName).check(ProjectPermission.READ);
+      projectState.checkStatePermitsRead();
+      forProject.check(ProjectPermission.READ);
       filter = false;
-    } catch (AuthException e) {
+    } catch (AuthException | ResourceConflictException e) {
       filter = true;
     }
     if (filter) {
@@ -481,7 +482,7 @@
         }
         local = n;
       }
-      local = refFilterFactory.create(pc.getProjectState(), git).filter(local, true);
+      local = forProject.filter(local, git, RefFilterOptions.builder().setFilterMeta(true).build());
     }
 
     return pushAllRefs ? doPushAll(tn, local) : doPushDelta(local);
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/PushResultProcessing.java b/src/main/java/com/googlesource/gerrit/plugins/replication/PushResultProcessing.java
index 654cd1f..d9efa95 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/PushResultProcessing.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/PushResultProcessing.java
@@ -14,7 +14,7 @@
 
 package com.googlesource.gerrit.plugins.replication;
 
-import com.google.gerrit.common.EventDispatcher;
+import com.google.gerrit.server.events.EventDispatcher;
 import com.google.gerrit.server.events.RefEvent;
 import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.google.gwtorm.server.OrmException;
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/RemoteSiteUser.java b/src/main/java/com/googlesource/gerrit/plugins/replication/RemoteSiteUser.java
index 91fce7f..31d10b6 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/RemoteSiteUser.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/RemoteSiteUser.java
@@ -35,4 +35,10 @@
   public GroupMembership getEffectiveGroups() {
     return effectiveGroups;
   }
+
+  @Override
+  public Object getCacheKey() {
+    // Never cache a remote user
+    return new Object();
+  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationConfig.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationConfig.java
index e94abbd..9693e2d 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationConfig.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationConfig.java
@@ -14,6 +14,7 @@
 package com.googlesource.gerrit.plugins.replication;
 
 import com.google.gerrit.server.git.WorkQueue;
+import java.nio.file.Path;
 import java.util.List;
 
 public interface ReplicationConfig {
@@ -32,6 +33,8 @@
 
   boolean isEmpty();
 
+  Path getEventsDirectory();
+
   int shutdown();
 
   void startup(WorkQueue workQueue);
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationFileBasedConfig.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationFileBasedConfig.java
index bec4f20..ac078e1 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationFileBasedConfig.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationFileBasedConfig.java
@@ -15,8 +15,10 @@
 
 import static java.util.stream.Collectors.toList;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.gerrit.extensions.annotations.PluginData;
 import com.google.gerrit.server.config.SitePaths;
 import com.google.gerrit.server.git.WorkQueue;
 import com.google.inject.Inject;
@@ -42,17 +44,22 @@
 public class ReplicationFileBasedConfig implements ReplicationConfig {
   static final Logger log = LoggerFactory.getLogger(ReplicationFileBasedConfig.class);
   private List<Destination> destinations;
+  private final SitePaths site;
   private Path cfgPath;
   private boolean replicateAllOnPluginStart;
   private boolean defaultForceUpdate;
   private final FileBasedConfig config;
+  private final Path pluginDataDir;
 
   @Inject
-  public ReplicationFileBasedConfig(SitePaths site, DestinationFactory destinationFactory)
+  public ReplicationFileBasedConfig(
+      SitePaths site, DestinationFactory destinationFactory, @PluginData Path pluginDataDir)
       throws ConfigInvalidException, IOException {
+    this.site = site;
     this.cfgPath = site.etc_dir.resolve("replication.config");
     this.config = new FileBasedConfig(cfgPath.toFile(), FS.DETECTED);
     this.destinations = allDestinations(destinationFactory);
+    this.pluginDataDir = pluginDataDir;
   }
 
   /*
@@ -100,7 +107,7 @@
           String.format("Cannot read %s: %s", config.getFile(), e.getMessage()), e);
     }
 
-    replicateAllOnPluginStart = config.getBoolean("gerrit", "replicateOnStartup", true);
+    replicateAllOnPluginStart = config.getBoolean("gerrit", "replicateOnStartup", false);
 
     defaultForceUpdate = config.getBoolean("gerrit", "defaultForceUpdate", false);
 
@@ -180,6 +187,15 @@
     return destinations.isEmpty();
   }
 
+  @Override
+  public Path getEventsDirectory() {
+    String eventsDirectory = config.getString("replication", null, "eventsDirectory");
+    if (!Strings.isNullOrEmpty(eventsDirectory)) {
+      return site.resolve(eventsDirectory);
+    }
+    return pluginDataDir;
+  }
+
   Path getCfgPath() {
     return cfgPath;
   }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationModule.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationModule.java
index f30e13d..b989827 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationModule.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationModule.java
@@ -36,6 +36,9 @@
   protected void configure() {
     bind(DestinationFactory.class).in(Scopes.SINGLETON);
     bind(ReplicationQueue.class).in(Scopes.SINGLETON);
+    bind(LifecycleListener.class)
+        .annotatedWith(UniqueAnnotations.create())
+        .to(ReplicationQueue.class);
 
     DynamicSet.bind(binder(), GitReferenceUpdatedListener.class).to(ReplicationQueue.class);
     DynamicSet.bind(binder(), NewProjectCreatedListener.class).to(ReplicationQueue.class);
@@ -56,6 +59,7 @@
 
     install(new FactoryModuleBuilder().build(PushAll.Factory.class));
     install(new FactoryModuleBuilder().build(RemoteSiteUser.Factory.class));
+    install(new FactoryModuleBuilder().build(ReplicationState.Factory.class));
 
     bind(ReplicationConfig.class).to(AutoReloadConfigDecorator.class);
     bind(ReplicationStateListener.class).to(ReplicationStateLogger.class);
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationQueue.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationQueue.java
index 30aff44..8e74a5f 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationQueue.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationQueue.java
@@ -15,7 +15,6 @@
 package com.googlesource.gerrit.plugins.replication;
 
 import com.google.common.base.Strings;
-import com.google.gerrit.common.EventDispatcher;
 import com.google.gerrit.extensions.events.GitReferenceUpdatedListener;
 import com.google.gerrit.extensions.events.HeadUpdatedListener;
 import com.google.gerrit.extensions.events.LifecycleListener;
@@ -23,6 +22,7 @@
 import com.google.gerrit.extensions.events.ProjectDeletedListener;
 import com.google.gerrit.extensions.registration.DynamicItem;
 import com.google.gerrit.reviewdb.client.Project;
+import com.google.gerrit.server.events.EventDispatcher;
 import com.google.gerrit.server.git.WorkQueue;
 import com.google.inject.Inject;
 import com.googlesource.gerrit.plugins.replication.PushResultProcessing.GitUpdateProcessing;
@@ -72,6 +72,8 @@
   private final DynamicItem<EventDispatcher> dispatcher;
   private final ReplicationConfig config;
   private final GerritSshApi gerritAdmin;
+  private final ReplicationState.Factory replicationStateFactory;
+  private final EventsStorage eventsStorage;
   private volatile boolean running;
 
   @Inject
@@ -81,19 +83,24 @@
       GerritSshApi ga,
       ReplicationConfig rc,
       DynamicItem<EventDispatcher> dis,
-      ReplicationStateListener sl) {
+      ReplicationStateListener sl,
+      ReplicationState.Factory rsf,
+      EventsStorage es) {
     workQueue = wq;
     sshHelper = sh;
     dispatcher = dis;
     config = rc;
     stateLog = sl;
     gerritAdmin = ga;
+    replicationStateFactory = rsf;
+    eventsStorage = es;
   }
 
   @Override
   public void start() {
     config.startup(workQueue);
     running = true;
+    firePendingEvents();
   }
 
   @Override
@@ -127,23 +134,37 @@
 
   @Override
   public void onGitReferenceUpdated(GitReferenceUpdatedListener.Event event) {
-    ReplicationState state = new ReplicationState(new GitUpdateProcessing(dispatcher.get()));
+    onGitReferenceUpdated(event.getProjectName(), event.getRefName());
+  }
+
+  private void onGitReferenceUpdated(String projectName, String refName) {
+    ReplicationState state =
+        replicationStateFactory.create(new GitUpdateProcessing(dispatcher.get()));
     if (!running) {
       stateLog.warn("Replication plugin did not finish startup before event", state);
       return;
     }
 
-    Project.NameKey project = new Project.NameKey(event.getProjectName());
+    Project.NameKey project = new Project.NameKey(projectName);
     for (Destination cfg : config.getDestinations(FilterType.ALL)) {
-      if (cfg.wouldPushProject(project) && cfg.wouldPushRef(event.getRefName())) {
+      if (cfg.wouldPushProject(project) && cfg.wouldPushRef(refName)) {
+        String eventKey = eventsStorage.persist(projectName, refName);
+        state.setEventKey(eventKey);
         for (URIish uri : cfg.getURIs(project, null)) {
-          cfg.schedule(project, event.getRefName(), uri, state);
+          cfg.schedule(project, refName, uri, state);
         }
       }
     }
     state.markAllPushTasksScheduled();
   }
 
+  private void firePendingEvents() {
+    for (EventsStorage.ReplicateRefUpdate e : eventsStorage.list()) {
+      repLog.info("Firing pending event {}", e);
+      onGitReferenceUpdated(e.project, e.ref);
+    }
+  }
+
   @Override
   public void onNewProjectCreated(NewProjectCreatedListener.Event event) {
     Project.NameKey projectName = new Project.NameKey(event.getProjectName());
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationState.java b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationState.java
index 86557e2..6f0803a 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationState.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/ReplicationState.java
@@ -16,6 +16,8 @@
 
 import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.Table;
+import com.google.inject.assistedinject.Assisted;
+import com.google.inject.assistedinject.AssistedInject;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
@@ -23,7 +25,13 @@
 import org.eclipse.jgit.transport.URIish;
 
 public class ReplicationState {
+
+  public interface Factory {
+    ReplicationState create(PushResultProcessing processing);
+  }
+
   private boolean allScheduled;
+  private final EventsStorage eventsStorage;
   private final PushResultProcessing pushResultProcessing;
 
   private final Lock countingLock = new ReentrantLock();
@@ -49,7 +57,11 @@
   private int totalPushTasksCount;
   private int finishedPushTasksCount;
 
-  public ReplicationState(PushResultProcessing processing) {
+  private String eventKey;
+
+  @AssistedInject
+  ReplicationState(EventsStorage storage, @Assisted PushResultProcessing processing) {
+    eventsStorage = storage;
     pushResultProcessing = processing;
     statusByProjectRef = HashBasedTable.create();
   }
@@ -74,6 +86,7 @@
       URIish uri,
       RefPushResult status,
       RemoteRefUpdate.Status refUpdateStatus) {
+    deleteEvent();
     pushResultProcessing.onRefReplicatedToOneNode(project, ref, uri, status, refUpdateStatus);
 
     RefReplicationStatus completedRefStatus = null;
@@ -103,6 +116,12 @@
     }
   }
 
+  private void deleteEvent() {
+    if (eventKey != null) {
+      eventsStorage.delete(eventKey);
+    }
+  }
+
   public void markAllPushTasksScheduled() {
     countingLock.lock();
     try {
@@ -173,4 +192,8 @@
       return name().toLowerCase().replace("_", "-");
     }
   }
+
+  public void setEventKey(String eventKey) {
+    this.eventKey = eventKey;
+  }
 }
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/StartCommand.java b/src/main/java/com/googlesource/gerrit/plugins/replication/StartCommand.java
index 5aa0861..77bc285 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/StartCommand.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/StartCommand.java
@@ -51,13 +51,15 @@
 
   @Inject private PushAll.Factory pushFactory;
 
+  @Inject private ReplicationState.Factory replicationStateFactory;
+
   @Override
   protected void run() throws Failure {
     if (all && projectPatterns.size() > 0) {
       throw new UnloggedFailure(1, "error: cannot combine --all and PROJECT");
     }
 
-    ReplicationState state = new ReplicationState(new CommandProcessing(this));
+    ReplicationState state = replicationStateFactory.create(new CommandProcessing(this));
     Future<?> future = null;
 
     ReplicationFilter projectFilter;
diff --git a/src/main/resources/Documentation/config.md b/src/main/resources/Documentation/config.md
index c066513..646d49c 100644
--- a/src/main/resources/Documentation/config.md
+++ b/src/main/resources/Documentation/config.md
@@ -65,7 +65,7 @@
 
 gerrit.replicateOnStartup
 :	If true, replicates to all remotes on startup to ensure they
-	are in-sync with this server.  By default, true.
+	are in-sync with this server.  By default, false.
 
 gerrit.autoReload
 :	If true, automatically reloads replication destinations and settings
@@ -116,6 +116,17 @@
 
 	By default, pushes are retried indefinitely.
 
+replication.eventsDirectory
+: Directory where replication events are persisted
+
+	When scheduling a replication, the replication event is persisted
+	under this directory. When the replication is done, the event is deleted.
+	If plugin is stopped before all scheduled replications are done, the
+	persisted events will not be deleted. When the plugin is started again,
+	it will trigger all replications found under this directory.
+
+	When not set, defaults to the plugin's data directory.
+
 remote.NAME.url
 :	Address of the remote server to push to.  Multiple URLs may be
 	specified within a single remote block, listing different
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/GitUpdateProcessingTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/GitUpdateProcessingTest.java
index 337bd1d..5fa7b98 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/GitUpdateProcessingTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/GitUpdateProcessingTest.java
@@ -22,8 +22,8 @@
 import static org.easymock.EasyMock.reset;
 import static org.easymock.EasyMock.verify;
 
-import com.google.gerrit.common.EventDispatcher;
 import com.google.gerrit.reviewdb.server.ReviewDb;
+import com.google.gerrit.server.events.EventDispatcher;
 import com.google.gerrit.server.permissions.PermissionBackendException;
 import com.google.gwtorm.client.KeyUtil;
 import com.google.gwtorm.server.OrmException;
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/ReplicationStateTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/ReplicationStateTest.java
index 193af1e..cf6715e 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/ReplicationStateTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/ReplicationStateTest.java
@@ -32,12 +32,15 @@
 
   private ReplicationState replicationState;
   private PushResultProcessing pushResultProcessingMock;
+  private EventsStorage eventsStorage;
 
   @Before
   public void setUp() throws Exception {
     pushResultProcessingMock = createNiceMock(PushResultProcessing.class);
     replay(pushResultProcessingMock);
-    replicationState = new ReplicationState(pushResultProcessingMock);
+    eventsStorage = createNiceMock(EventsStorage.class);
+    replay(eventsStorage);
+    replicationState = new ReplicationState(eventsStorage, pushResultProcessingMock);
   }
 
   @Test