Delete refs as part of the replication queue execution
Do not delete refs synchronously in any case and use
instead the same queue for the regular fetch operations.
This allows to keep the ordering of the delete refs
against any other fetch.
Also remove the now unused async DeleteRefJob because the
deletion is managed in the same replication queue as any
other ref.
Bug: Issue 367651566
Change-Id: I3b43c0c5fb5e9e1189bf0e3990b7981e3e68cd92
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchOne.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchOne.java
index 84be78a..69c00f9 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchOne.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/FetchOne.java
@@ -33,6 +33,7 @@
import com.google.gerrit.server.util.IdGenerator;
import com.google.inject.Inject;
import com.google.inject.assistedinject.Assisted;
+import com.googlesource.gerrit.plugins.replication.pull.api.DeleteRefCommand;
import com.googlesource.gerrit.plugins.replication.pull.api.PullReplicationApiRequestMetrics;
import com.googlesource.gerrit.plugins.replication.pull.fetch.Fetch;
import com.googlesource.gerrit.plugins.replication.pull.fetch.FetchFactory;
@@ -72,6 +73,7 @@
private final ReplicationStateListener stateLog;
public static final String ALL_REFS = "..all..";
static final String ID_KEY = "fetchOneId";
+ private final DeleteRefCommand deleteRefCommand;
interface Factory {
FetchOne create(
@@ -118,6 +120,7 @@
FetchReplicationMetrics m,
FetchFactory fetchFactory,
FetchRefsDatabase fetchRefsDatabase,
+ DeleteRefCommand deleteRefCommand,
@Assisted Project.NameKey d,
@Assisted URIish u,
@Assisted Optional<PullReplicationApiRequestMetrics> apiRequestMetrics) {
@@ -126,6 +129,7 @@
config = c.getRemoteConfig();
threadScoper = ts;
this.fetchRefsDatabase = fetchRefsDatabase;
+ this.deleteRefCommand = deleteRefCommand;
projectName = d;
uri = u;
lockRetryCount = 0;
@@ -454,7 +458,21 @@
List<FetchRefSpec> fetchRefSpecs = getFetchRefSpecs();
try {
- updateStates(fetch.fetch(fetchRefSpecs));
+ List<FetchRefSpec> toFetch =
+ fetchRefSpecs.stream().filter(rs -> rs.getSource() != null).toList();
+ Set<String> toDelete =
+ fetchRefSpecs.stream()
+ .filter(rs -> rs.getSource() == null)
+ .map(RefSpec::getDestination)
+ .collect(Collectors.toSet());
+ updateStates(fetch.fetch(toFetch));
+
+ // JGit doesn't support a fetch of <empty> to a ref (e.g. :refs/to/delete) therefore we have
+ // manage them separately and remove them one by one.
+ if (!toDelete.isEmpty()) {
+ updateStates(
+ deleteRefCommand.deleteRefsSync(taskIdHex, projectName, toDelete, getRemoteName()));
+ }
} catch (InexistentRefTransportException e) {
String inexistentRef = e.getInexistentRef();
repLog.info(
@@ -571,9 +589,14 @@
private Optional<FetchRefSpec> refToFetchRefSpec(
FetchRefSpec refSpec, List<? extends RefSpec> configRefSpecs) {
for (RefSpec configRefSpec : configRefSpecs) {
- if (configRefSpec.matchSource(refSpec.refName())) {
- return Optional.of(
- FetchRefSpec.fromRefSpec(configRefSpec.expandFromSource(refSpec.refName())));
+ String refName = refSpec.refName();
+ if (configRefSpec.matchSource(refName)) {
+ if (refSpec.getSource() != null) {
+ return Optional.of(
+ FetchRefSpec.fromRefSpec(configRefSpec.expandFromSource(refSpec.getSource())));
+ } else {
+ return Optional.of(refSpec);
+ }
}
}
return Optional.empty();
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationModule.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationModule.java
index a4af8ca..7972fc4 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationModule.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationModule.java
@@ -40,7 +40,6 @@
import com.googlesource.gerrit.plugins.replication.ObservableQueue;
import com.googlesource.gerrit.plugins.replication.ReplicationConfigModule;
import com.googlesource.gerrit.plugins.replication.StartReplicationCapability;
-import com.googlesource.gerrit.plugins.replication.pull.api.DeleteRefJob;
import com.googlesource.gerrit.plugins.replication.pull.api.FetchApiCapability;
import com.googlesource.gerrit.plugins.replication.pull.api.FetchJob;
import com.googlesource.gerrit.plugins.replication.pull.auth.PullReplicationGroupModule;
@@ -83,7 +82,6 @@
bind(RevisionReader.class).in(Scopes.SINGLETON);
bind(ApplyObject.class);
install(new FactoryModuleBuilder().build(FetchJob.Factory.class));
- install(new FactoryModuleBuilder().build(DeleteRefJob.Factory.class));
install(new ApplyObjectCacheModule());
install(
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefCommand.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefCommand.java
index df13944..fe558e5 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefCommand.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefCommand.java
@@ -18,38 +18,31 @@
import com.google.common.flogger.FluentLogger;
import com.google.gerrit.entities.Project;
-import com.google.gerrit.extensions.registration.DynamicItem;
import com.google.gerrit.extensions.restapi.ResourceNotFoundException;
import com.google.gerrit.extensions.restapi.RestApiException;
-import com.google.gerrit.server.events.EventDispatcher;
import com.google.gerrit.server.git.GitRepositoryManager;
-import com.google.gerrit.server.permissions.PermissionBackendException;
import com.google.gerrit.server.project.ProjectCache;
import com.google.gerrit.server.project.ProjectState;
import com.google.inject.Inject;
-import com.googlesource.gerrit.plugins.replication.pull.Context;
-import com.googlesource.gerrit.plugins.replication.pull.FetchRefReplicatedEvent;
import com.googlesource.gerrit.plugins.replication.pull.LocalGitRepositoryManagerProvider;
import com.googlesource.gerrit.plugins.replication.pull.PullReplicationStateLogger;
import com.googlesource.gerrit.plugins.replication.pull.ReplicationState;
import com.googlesource.gerrit.plugins.replication.pull.Source;
import com.googlesource.gerrit.plugins.replication.pull.SourcesCollection;
-import com.googlesource.gerrit.plugins.replication.pull.api.exception.DeleteRefException;
import com.googlesource.gerrit.plugins.replication.pull.fetch.RefUpdateState;
import java.io.IOException;
+import java.util.List;
import java.util.Optional;
import java.util.Set;
import org.eclipse.jgit.lib.ObjectId;
import org.eclipse.jgit.lib.Ref;
import org.eclipse.jgit.lib.RefUpdate;
import org.eclipse.jgit.lib.Repository;
-import org.eclipse.jgit.transport.URIish;
public class DeleteRefCommand {
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
private final PullReplicationStateLogger fetchStateLog;
- private final DynamicItem<EventDispatcher> eventDispatcher;
private final ProjectCache projectCache;
private final SourcesCollection sourcesCollection;
private final GitRepositoryManager gitManager;
@@ -59,28 +52,35 @@
PullReplicationStateLogger fetchStateLog,
ProjectCache projectCache,
SourcesCollection sourcesCollection,
- DynamicItem<EventDispatcher> eventDispatcher,
LocalGitRepositoryManagerProvider gitManagerProvider) {
this.fetchStateLog = fetchStateLog;
this.projectCache = projectCache;
- this.eventDispatcher = eventDispatcher;
this.sourcesCollection = sourcesCollection;
this.gitManager = gitManagerProvider.get();
}
- public void deleteRefsSync(
- Project.NameKey name, Set<String> deletedRefNames, String sourceLabel) {
- deletedRefNames.forEach(
- r -> {
- try {
- deleteRef(name, r, sourceLabel);
- } catch (RestApiException | IOException e) {
- repLog.error("Could not delete ref {}:{} from source {}", name.get(), r, sourceLabel);
- }
- });
+ public List<RefUpdateState> deleteRefsSync(
+ String taskIdHex, Project.NameKey name, Set<String> deletedRefNames, String sourceLabel) {
+ return deletedRefNames.stream()
+ .map(
+ r -> {
+ try {
+ return deleteRef(taskIdHex, name, r, sourceLabel);
+ } catch (RestApiException | IOException e) {
+ repLog.error(
+ "[{}] Could not delete ref {}:{} from source {}",
+ taskIdHex,
+ name.get(),
+ r,
+ sourceLabel);
+ return new RefUpdateState(r, RefUpdate.Result.IO_FAILURE);
+ }
+ })
+ .toList();
}
- public void deleteRef(Project.NameKey name, String refName, String sourceLabel)
+ public RefUpdateState deleteRef(
+ String taskIdRef, Project.NameKey name, String refName, String sourceLabel)
throws IOException, RestApiException {
Source source =
sourcesCollection
@@ -91,73 +91,45 @@
String.format("Could not find URI for %s remote", sourceLabel)));
if (!source.isMirror()) {
repLog.info(
- "Ignoring ref {} deletion from project {}, as mirror option is false", refName, name);
- return;
+ "[{}] Ignoring ref {} deletion from project {}, as mirror option is false",
+ taskIdRef,
+ refName,
+ name);
+ return new RefUpdateState(refName, RefUpdate.Result.NO_CHANGE);
}
- try {
- repLog.info("Delete ref from {} for project {}, ref name {}", sourceLabel, name, refName);
- Optional<ProjectState> projectState = projectCache.get(name);
- if (!projectState.isPresent()) {
- throw new ResourceNotFoundException(String.format("Project %s was not found", name));
- }
- Optional<Ref> ref = getRef(name, refName);
- if (!ref.isPresent()) {
- logger.atFine().log("Ref %s was not found in project %s", refName, name);
- return;
- }
-
- URIish sourceUri = source.getURI(name);
-
- try {
-
- Context.setLocalEvent(true);
- RefUpdate.Result successResult = ensureSuccess(deleteRef(name, ref.get()));
-
- eventDispatcher
- .get()
- .postEvent(
- new FetchRefReplicatedEvent(
- name.get(),
- refName,
- sourceUri,
- ReplicationState.RefFetchResult.SUCCEEDED,
- successResult));
- } catch (PermissionBackendException e) {
- logger.atSevere().withCause(e).log(
- "Unexpected error while trying to delete ref '%s' on project %s and notifying it",
- refName, name);
- throw RestApiException.wrap(e.getMessage(), e);
- } catch (IOException e) {
- RefUpdate.Result refUpdateResult =
- e instanceof DeleteRefException
- ? ((DeleteRefException) e).getResult()
- : RefUpdate.Result.LOCK_FAILURE;
- eventDispatcher
- .get()
- .postEvent(
- new FetchRefReplicatedEvent(
- name.get(),
- refName,
- sourceUri,
- ReplicationState.RefFetchResult.FAILED,
- refUpdateResult));
- String message =
- String.format(
- "RefUpdate lock failure for: sourceLabel=%s, project=%s, refName=%s",
- sourceLabel, name, refName);
- logger.atSevere().withCause(e).log("%s", message);
- fetchStateLog.error(message);
- throw e;
- } finally {
- Context.unsetLocalEvent();
- }
-
- repLog.info(
- "Delete ref from {} for project {}, ref name {} completed", sourceLabel, name, refName);
- } catch (PermissionBackendException e) {
- throw RestApiException.wrap(e.getMessage(), e);
+ repLog.info(
+ "[{}] Delete ref from {} for project {}, ref name {}",
+ taskIdRef,
+ sourceLabel,
+ name,
+ refName);
+ Optional<ProjectState> projectState = projectCache.get(name);
+ if (!projectState.isPresent()) {
+ throw new ResourceNotFoundException(String.format("Project %s was not found", name));
}
+
+ Optional<Ref> ref = getRef(name, refName);
+ if (!ref.isPresent()) {
+ logger.atFine().log("[%s] Ref %s was not found in project %s", taskIdRef, refName, name);
+ return new RefUpdateState(refName, RefUpdate.Result.NO_CHANGE);
+ }
+
+ RefUpdateState deleteResult = deleteRef(name, ref.get());
+ ReplicationState.RefFetchResult deleteAsFetchResult =
+ isSuccess(deleteResult)
+ ? ReplicationState.RefFetchResult.SUCCEEDED
+ : ReplicationState.RefFetchResult.FAILED;
+
+ repLog.info(
+ "[{}] Delete ref from {} for project {}, ref name {}: {} ({})",
+ taskIdRef,
+ sourceLabel,
+ name,
+ refName,
+ deleteAsFetchResult,
+ deleteResult);
+ return deleteResult;
}
private Optional<Ref> getRef(Project.NameKey repo, String refName) throws IOException {
@@ -181,8 +153,7 @@
}
}
- private static RefUpdate.Result ensureSuccess(RefUpdateState refUpdateState)
- throws DeleteRefException {
+ private static boolean isSuccess(RefUpdateState refUpdateState) {
switch (refUpdateState.getResult()) {
case NOT_ATTEMPTED:
case REJECTED:
@@ -191,8 +162,8 @@
case LOCK_FAILURE:
case IO_FAILURE:
case REJECTED_OTHER_REASON:
- throw new DeleteRefException("Failed ref deletion", refUpdateState.getResult());
+ return false;
}
- return refUpdateState.getResult();
+ return true;
}
}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefJob.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefJob.java
deleted file mode 100644
index acc40cb..0000000
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefJob.java
+++ /dev/null
@@ -1,45 +0,0 @@
-// Copyright (C) 2024 The Android Open Source Project
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package com.googlesource.gerrit.plugins.replication.pull.api;
-
-import com.google.gerrit.entities.Project;
-import com.google.inject.Inject;
-import com.google.inject.assistedinject.Assisted;
-import com.googlesource.gerrit.plugins.replication.pull.api.FetchAction.BatchInput;
-
-public class DeleteRefJob implements Runnable {
- public interface Factory {
- DeleteRefJob create(Project.NameKey project, BatchInput input);
- }
-
- private final DeleteRefCommand command;
- private final Project.NameKey project;
- private final BatchInput batchInput;
-
- @Inject
- public DeleteRefJob(
- DeleteRefCommand command,
- @Assisted Project.NameKey project,
- @Assisted BatchInput batchInput) {
- this.command = command;
- this.project = project;
- this.batchInput = batchInput;
- }
-
- @Override
- public void run() {
- command.deleteRefsSync(project, batchInput.getDeletedRefNames(), batchInput.label);
- }
-}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchAction.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchAction.java
index 2da66af..1d5ecec 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchAction.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchAction.java
@@ -53,29 +53,23 @@
@Singleton
public class FetchAction implements RestModifyView<ProjectResource, Input> {
private final FetchCommand command;
- private final DeleteRefCommand deleteRefCommand;
private final WorkQueue workQueue;
private final DynamicItem<UrlFormatter> urlFormatter;
private final FetchPreconditions preConditions;
private final Factory fetchJobFactory;
- private final DeleteRefJob.Factory deleteJobFactory;
@Inject
public FetchAction(
FetchCommand command,
- DeleteRefCommand deleteRefCommand,
WorkQueue workQueue,
DynamicItem<UrlFormatter> urlFormatter,
FetchPreconditions preConditions,
- FetchJob.Factory fetchJobFactory,
- DeleteRefJob.Factory deleteJobFactory) {
+ FetchJob.Factory fetchJobFactory) {
this.command = command;
- this.deleteRefCommand = deleteRefCommand;
this.workQueue = workQueue;
this.urlFormatter = urlFormatter;
this.preConditions = preConditions;
this.fetchJobFactory = fetchJobFactory;
- this.deleteJobFactory = deleteJobFactory;
}
public static class Input {
@@ -125,24 +119,14 @@
return batchInput;
}
- private Stream<String> getFilteredRefNames(Predicate<RefInput> filterFunc) {
- return refInputs.stream().filter(filterFunc).map(RefInput::refName);
- }
-
- private Stream<FetchRefSpec> getFilteredRefSpecs(Predicate<RefInput> filterFunc) {
- return getFilteredRefNames(filterFunc).map(FetchRefSpec::fromRef);
- }
-
- public Set<FetchRefSpec> getNonDeletedRefSpecs() {
- return getFilteredRefSpecs(RefInput.IS_DELETE.negate()).collect(Collectors.toSet());
- }
-
- public boolean hasDeletedRefSpecs() {
- return refInputs.stream().anyMatch(RefInput.IS_DELETE);
- }
-
- public Set<String> getDeletedRefNames() {
- return getFilteredRefNames(RefInput.IS_DELETE).collect(Collectors.toSet());
+ public Set<FetchRefSpec> getRefSpecs() {
+ return refInputs.stream()
+ .map(
+ ri ->
+ ri.isDelete()
+ ? FetchRefSpec.fromRef(":" + ri.refName())
+ : FetchRefSpec.fromRef(ri.refName()))
+ .collect(Collectors.toSet());
}
}
@@ -190,16 +174,7 @@
private Response<?> applySync(Project.NameKey project, BatchInput input)
throws InterruptedException, ExecutionException, RemoteConfigurationMissingException,
TimeoutException, TransportException {
- command.fetchSync(project, input.label, input.getNonDeletedRefSpecs());
-
- /* git fetches and deletes cannot be handled atomically within the same transaction.
- Here we choose to handle fetches first and then deletes:
- - If the fetch fails delete is not even attempted.
- - If the delete fails after the fetch then the client is left with some extra refs.
- */
- if (input.hasDeletedRefSpecs()) {
- deleteRefCommand.deleteRefsSync(project, input.getDeletedRefNames(), input.label);
- }
+ command.fetchSync(project, input.label, input.getRefSpecs());
return Response.created(input);
}
@@ -216,10 +191,6 @@
urlFormatter
.get()
.getRestUrl("a/config/server/tasks/" + HexFormat.fromInt(task.getTaskId()));
-
- if (batchInput.hasDeletedRefSpecs()) {
- workQueue.getDefaultQueue().submit(deleteJobFactory.create(project, batchInput));
- }
// We're in a HTTP handler, so must be present.
checkState(url.isPresent());
return Response.accepted(url.get());
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchJob.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchJob.java
index 3b0ba93..f5ee3a1 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchJob.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchJob.java
@@ -52,7 +52,7 @@
@Override
public void run() {
try {
- command.fetchAsync(project, batchInput.label, batchInput.getNonDeletedRefSpecs(), metrics);
+ command.fetchAsync(project, batchInput.label, batchInput.getRefSpecs(), metrics);
} catch (InterruptedException
| ExecutionException
| RemoteConfigurationMissingException
@@ -60,7 +60,7 @@
| TransportException e) {
log.atSevere().withCause(e).log(
"Exception during the async fetch call for project %s, label %s and ref(s) name(s) %s",
- project.get(), batchInput.label, batchInput.getNonDeletedRefSpecs());
+ project.get(), batchInput.label, batchInput.getRefSpecs());
}
}
}
diff --git a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListener.java b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListener.java
index 74bf51d..6ca79e6 100644
--- a/src/main/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListener.java
+++ b/src/main/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListener.java
@@ -26,7 +26,6 @@
import com.google.gerrit.entities.RefNames;
import com.google.gerrit.extensions.restapi.AuthException;
import com.google.gerrit.extensions.restapi.ResourceNotFoundException;
-import com.google.gerrit.extensions.restapi.RestApiException;
import com.google.gerrit.extensions.restapi.UnprocessableEntityException;
import com.google.gerrit.server.config.GerritInstanceId;
import com.google.gerrit.server.data.RefUpdateAttribute;
@@ -45,7 +44,6 @@
import com.googlesource.gerrit.plugins.replication.pull.FetchOne;
import com.googlesource.gerrit.plugins.replication.pull.Source;
import com.googlesource.gerrit.plugins.replication.pull.SourcesCollection;
-import com.googlesource.gerrit.plugins.replication.pull.api.DeleteRefCommand;
import com.googlesource.gerrit.plugins.replication.pull.api.FetchAction;
import com.googlesource.gerrit.plugins.replication.pull.api.FetchJob;
import com.googlesource.gerrit.plugins.replication.pull.api.FetchJob.Factory;
@@ -61,7 +59,6 @@
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
private static final String ZERO_ID_NAME = ObjectId.zeroId().name();
- private final DeleteRefCommand deleteCommand;
private final ExcludedRefsFilter refsFilter;
private final Factory fetchJobFactory;
private final UpdateHeadCommand updateHeadCommand;
@@ -75,7 +72,6 @@
@Inject
public StreamEventListener(
@Nullable @GerritInstanceId String instanceId,
- DeleteRefCommand deleteCommand,
UpdateHeadCommand updateHeadCommand,
ProjectInitializationAction projectInitializationAction,
WorkQueue workQueue,
@@ -85,7 +81,6 @@
ExcludedRefsFilter excludedRefsFilter,
@Named(APPLY_OBJECTS_CACHE) Cache<ApplyObjectsCacheKey, Long> refUpdatesSucceededCache) {
this.instanceId = instanceId;
- this.deleteCommand = deleteCommand;
this.updateHeadCommand = updateHeadCommand;
this.projectInitializationAction = projectInitializationAction;
this.workQueue = workQueue;
@@ -133,15 +128,6 @@
return;
}
- if (isProjectDelete(refUpdatedEvent)) {
- return;
- }
-
- if (isRefDelete(refUpdatedEvent)) {
- deleteRef(refUpdatedEvent);
- return;
- }
-
if (isApplyObjectsCacheHit(refUpdatedEvent)) {
logger.atFine().log(
"Skipping refupdate '%s' '%s'=>'%s' (eventCreatedOn=%d) for project '%s' because has been already replicated via apply-object",
@@ -157,6 +143,7 @@
refUpdatedEvent.getRefName(),
refUpdatedEvent.instanceId,
refUpdatedEvent.getProjectNameKey(),
+ isRefDelete(refUpdatedEvent),
metrics);
} else if (event instanceof ProjectCreatedEvent) {
ProjectCreatedEvent projectCreatedEvent = (ProjectCreatedEvent) event;
@@ -166,6 +153,7 @@
FetchOne.ALL_REFS,
projectCreatedEvent.instanceId,
projectCreatedEvent.getProjectNameKey(),
+ false,
metrics);
} catch (AuthException | PermissionBackendException | IOException e) {
logger.atSevere().withCause(e).log(
@@ -184,19 +172,6 @@
}
}
- private void deleteRef(RefUpdatedEvent refUpdatedEvent) {
- try {
- deleteCommand.deleteRef(
- refUpdatedEvent.getProjectNameKey(),
- refUpdatedEvent.getRefName(),
- refUpdatedEvent.instanceId);
- } catch (IOException | RestApiException e) {
- logger.atSevere().withCause(e).log(
- "Cannot delete ref %s project:%s",
- refUpdatedEvent.getRefName(), refUpdatedEvent.getProjectNameKey());
- }
- }
-
private boolean isRefToBeReplicated(String refName) {
return !refsFilter.match(refName);
}
@@ -239,10 +214,12 @@
String refName,
String sourceInstanceId,
NameKey projectNameKey,
+ boolean isDelete,
PullReplicationApiRequestMetrics metrics) {
FetchAction.Input input = new FetchAction.Input();
input.refName = refName;
input.label = sourceInstanceId;
+ input.isDelete = isDelete;
workQueue
.getDefaultQueue()
.submit(fetchJobFactory.create(projectNameKey, fromInput(input), metrics));
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchOneTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchOneTest.java
index 8903b2b..49e8d6f 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchOneTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/FetchOneTest.java
@@ -30,6 +30,7 @@
import com.google.gerrit.server.git.GitRepositoryManager;
import com.google.gerrit.server.git.PerThreadRequestScope;
import com.google.gerrit.server.util.IdGenerator;
+import com.googlesource.gerrit.plugins.replication.pull.api.DeleteRefCommand;
import com.googlesource.gerrit.plugins.replication.pull.api.PullReplicationApiRequestMetrics;
import com.googlesource.gerrit.plugins.replication.pull.fetch.Fetch;
import com.googlesource.gerrit.plugins.replication.pull.fetch.FetchFactory;
@@ -86,6 +87,7 @@
@Mock private RemoteConfig remoteConfig;
@Mock private DynamicItem<ReplicationFetchFilter> replicationFilter;
@Mock private FetchRefsDatabase fetchRefsDatabase;
+ @Mock private DeleteRefCommand deleteRefCommand;
@Mock private Transport transport;
@@ -126,6 +128,7 @@
fetchReplicationMetrics,
fetchFactory,
fetchRefsDatabase,
+ deleteRefCommand,
PROJECT_NAME,
urIish,
Optional.of(pullReplicationApiRequestMetrics));
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationITBase.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationITBase.java
index 0754870..f3381e6 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationITBase.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/PullReplicationITBase.java
@@ -65,6 +65,7 @@
config.setString("remote", remoteName, "apiUrl", adminRestSession.url());
config.setString("remote", remoteName, "fetch", "+refs/*:refs/*");
config.setInt("remote", remoteName, "timeout", 600);
+ config.setBoolean("remote", remoteName, "mirror", true);
config.setInt("remote", remoteName, "replicationDelay", TEST_REPLICATION_DELAY);
project.ifPresent(prj -> config.setString("remote", remoteName, "projects", prj));
config.setBoolean("gerrit", null, "autoReload", true);
@@ -139,6 +140,39 @@
}
@Test
+ @GerritConfig(name = "gerrit.instanceId", value = TEST_REPLICATION_REMOTE)
+ public void shouldReplicateBranchDeleted() throws Exception {
+ String testProjectName = project + TEST_REPLICATION_SUFFIX;
+ createTestProject(testProjectName);
+
+ String newBranch = "refs/heads/mybranch";
+ String master = "refs/heads/master";
+ BranchInput input = new BranchInput();
+ input.revision = master;
+ gApi.projects().name(project.get()).branch(newBranch).create(input);
+ String branchRevision = gApi.projects().name(project.get()).branch(newBranch).get().revision;
+
+ ReplicationQueue pullReplicationQueue =
+ plugin.getSysInjector().getInstance(ReplicationQueue.class);
+ ProjectEvent event =
+ generateUpdateEvent(
+ project,
+ newBranch,
+ branchRevision,
+ ObjectId.zeroId().getName(),
+ TEST_REPLICATION_REMOTE);
+ pullReplicationQueue.onEvent(event);
+
+ try (Repository repo = repoManager.openRepository(project);
+ Repository sourceRepo = repoManager.openRepository(project)) {
+ waitUntil(() -> checkedGetRef(repo, newBranch) == null);
+
+ Ref targetBranchRef = getRef(repo, newBranch);
+ assertThat(targetBranchRef).isNull();
+ }
+ }
+
+ @Test
@UseLocalDisk
@GerritConfig(name = "gerrit.instanceId", value = TEST_REPLICATION_REMOTE)
public void shouldReplicateForceUpdatedBranch() throws Exception {
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefCommandTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefCommandTest.java
index 5ede40b..4903965 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefCommandTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/DeleteRefCommandTest.java
@@ -15,82 +15,55 @@
package com.googlesource.gerrit.plugins.replication.pull.api;
import static com.google.common.truth.Truth.assertThat;
-import static com.google.gerrit.testing.GerritJUnit.assertThrows;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyString;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoInteractions;
import static org.mockito.Mockito.when;
import com.google.gerrit.entities.Project;
import com.google.gerrit.entities.Project.NameKey;
-import com.google.gerrit.extensions.registration.DynamicItem;
-import com.google.gerrit.server.events.Event;
-import com.google.gerrit.server.events.EventDispatcher;
import com.google.gerrit.server.git.LocalDiskRepositoryManager;
-import com.google.gerrit.server.permissions.PermissionBackend.ForProject;
-import com.google.gerrit.server.permissions.PermissionBackend.ForRef;
-import com.google.gerrit.server.permissions.PermissionBackend.WithUser;
import com.google.gerrit.server.project.ProjectCache;
import com.google.gerrit.server.project.ProjectState;
-import com.googlesource.gerrit.plugins.replication.pull.FetchRefReplicatedEvent;
import com.googlesource.gerrit.plugins.replication.pull.LocalGitRepositoryManagerProvider;
import com.googlesource.gerrit.plugins.replication.pull.PullReplicationStateLogger;
-import com.googlesource.gerrit.plugins.replication.pull.ReplicationState;
import com.googlesource.gerrit.plugins.replication.pull.Source;
import com.googlesource.gerrit.plugins.replication.pull.SourcesCollection;
-import com.googlesource.gerrit.plugins.replication.pull.fetch.ApplyObject;
-import java.io.IOException;
+import com.googlesource.gerrit.plugins.replication.pull.fetch.RefUpdateState;
import java.util.Optional;
import org.eclipse.jgit.lib.Ref;
import org.eclipse.jgit.lib.RefUpdate;
import org.eclipse.jgit.lib.RefUpdate.Result;
import org.eclipse.jgit.lib.Repository;
-import org.eclipse.jgit.transport.URIish;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Captor;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(MockitoJUnitRunner.class)
public class DeleteRefCommandTest {
+ private static final String TEST_TASK_ID = "task-id";
private static final String TEST_SOURCE_LABEL = "test-source-label";
private static final String TEST_REF_NAME = "refs/changes/01/1/1";
- private static final String NON_EXISTING_REF_NAME = "refs/changes/01/11101/1";
private static final NameKey TEST_PROJECT_NAME = Project.nameKey("test-project");
- private static URIish TEST_REMOTE_URI;
@Mock private PullReplicationStateLogger fetchStateLog;
- @Mock private DynamicItem<EventDispatcher> eventDispatcherDataItem;
- @Mock private EventDispatcher eventDispatcher;
@Mock private ProjectCache projectCache;
- @Mock private ApplyObject applyObject;
@Mock private ProjectState projectState;
@Mock private SourcesCollection sourceCollection;
@Mock private Source source;
- @Mock private WithUser currentUser;
- @Mock private ForProject forProject;
- @Mock private ForRef forRef;
@Mock private LocalDiskRepositoryManager gitManager;
@Mock private RefUpdate refUpdate;
@Mock private Repository repository;
@Mock private Ref currentRef;
- @Captor ArgumentCaptor<Event> eventCaptor;
private DeleteRefCommand objectUnderTest;
@Before
public void setup() throws Exception {
- when(eventDispatcherDataItem.get()).thenReturn(eventDispatcher);
when(projectCache.get(any())).thenReturn(Optional.of(projectState));
when(sourceCollection.getByRemoteName(TEST_SOURCE_LABEL)).thenReturn(Optional.of(source));
- TEST_REMOTE_URI = new URIish("git://some.remote.uri");
- when(source.getURI(TEST_PROJECT_NAME)).thenReturn(TEST_REMOTE_URI);
when(gitManager.openRepository(any())).thenReturn(repository);
when(repository.updateRef(any())).thenReturn(refUpdate);
when(repository.exactRef(anyString())).thenReturn(currentRef);
@@ -101,58 +74,17 @@
fetchStateLog,
projectCache,
sourceCollection,
- eventDispatcherDataItem,
new LocalGitRepositoryManagerProvider(gitManager));
}
@Test
- public void shouldSendEventWhenDeletingRef() throws Exception {
- when(source.isMirror()).thenReturn(true);
-
- objectUnderTest.deleteRef(TEST_PROJECT_NAME, TEST_REF_NAME, TEST_SOURCE_LABEL);
-
- assertFetchReplicatedEvent(ReplicationState.RefFetchResult.SUCCEEDED, Result.FORCED);
- }
-
- @Test
- public void shouldNotSendNotSendEventWhenMirroringIsDisabled() throws Exception {
- when(source.isMirror()).thenReturn(false);
-
- objectUnderTest.deleteRef(TEST_PROJECT_NAME, TEST_REF_NAME, TEST_SOURCE_LABEL);
-
- verifyNoInteractions(eventDispatcher);
- }
-
- @Test
- public void shouldHandleNonExistingRef() throws Exception {
- when(source.isMirror()).thenReturn(true);
- when(repository.exactRef(anyString())).thenReturn(null);
- objectUnderTest.deleteRef(TEST_PROJECT_NAME, NON_EXISTING_REF_NAME, TEST_SOURCE_LABEL);
-
- verify(eventDispatcher, never()).postEvent(any());
- }
-
- @Test
- public void shouldThrowWhenRefDeletionFails() throws Exception {
+ public void shouldReturnLockFailureWhenRefDeletionFails() throws Exception {
when(source.isMirror()).thenReturn(true);
when(refUpdate.delete()).thenReturn(Result.LOCK_FAILURE);
- assertThrows(
- IOException.class,
- () -> objectUnderTest.deleteRef(TEST_PROJECT_NAME, TEST_REF_NAME, TEST_SOURCE_LABEL));
-
- assertFetchReplicatedEvent(ReplicationState.RefFetchResult.FAILED, Result.LOCK_FAILURE);
- }
-
- private void assertFetchReplicatedEvent(
- ReplicationState.RefFetchResult refFetchResult, RefUpdate.Result result) throws Exception {
- verify(eventDispatcher).postEvent(eventCaptor.capture());
- Event sentEvent = eventCaptor.getValue();
- assertThat(sentEvent).isInstanceOf(FetchRefReplicatedEvent.class);
- FetchRefReplicatedEvent fetchEvent = (FetchRefReplicatedEvent) sentEvent;
- assertThat(fetchEvent.getProjectNameKey()).isEqualTo(TEST_PROJECT_NAME);
- assertThat(fetchEvent.getRefName()).isEqualTo(TEST_REF_NAME);
- assertThat(fetchEvent.getStatus()).isEqualTo(refFetchResult.toString());
- assertThat(fetchEvent.getRefUpdateResult()).isEqualTo(result);
+ RefUpdateState deleteRefResult =
+ objectUnderTest.deleteRef(
+ TEST_TASK_ID, TEST_PROJECT_NAME, TEST_REF_NAME, TEST_SOURCE_LABEL);
+ assertThat(deleteRefResult.getResult()).isEqualTo(Result.LOCK_FAILURE);
}
}
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchActionTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchActionTest.java
index a4642ba..cb5758f 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchActionTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/api/FetchActionTest.java
@@ -19,9 +19,7 @@
import static org.apache.http.HttpStatus.SC_CREATED;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyString;
-import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import com.google.gerrit.extensions.registration.DynamicItem;
@@ -63,9 +61,7 @@
@Mock FetchCommand fetchCommand;
@Mock DeleteRefCommand deleteRefCommand;
@Mock FetchJob fetchJob;
- @Mock DeleteRefJob deleteRefJob;
@Mock FetchJob.Factory fetchJobFactory;
- @Mock DeleteRefJob.Factory deleteRefJobFactory;
@Mock ProjectResource projectResource;
@Mock WorkQueue workQueue;
@Mock ScheduledExecutorService exceutorService;
@@ -77,7 +73,6 @@
@Before
public void setup() throws Exception {
when(fetchJobFactory.create(any(), any(), any())).thenReturn(fetchJob);
- when(deleteRefJobFactory.create(any(), any())).thenReturn(deleteRefJob);
when(workQueue.getDefaultQueue()).thenReturn(exceutorService);
when(urlFormatter.getRestUrl(anyString())).thenReturn(Optional.of(location));
when(exceutorService.submit(any(Runnable.class)))
@@ -94,13 +89,7 @@
fetchAction =
new FetchAction(
- fetchCommand,
- deleteRefCommand,
- workQueue,
- urlFormatterDynamicItem,
- preConditions,
- fetchJobFactory,
- deleteRefJobFactory);
+ fetchCommand, workQueue, urlFormatterDynamicItem, preConditions, fetchJobFactory);
}
@Test
@@ -126,18 +115,6 @@
}
@Test
- public void shouldDeleteRefSync() throws Exception {
- FetchAction.BatchInput batchInputParams = new FetchAction.BatchInput();
- batchInputParams.label = label;
- batchInputParams.refInputs = Set.of(RefInput.create(refName, true));
-
- Response<?> response = fetchAction.apply(projectResource, batchInputParams);
- verify(deleteRefCommand).deleteRefsSync(any(), eq(Set.of(refName)), eq(label));
-
- assertThat(response.statusCode()).isEqualTo(SC_CREATED);
- }
-
- @Test
public void shouldDeleteRefAsync() throws Exception {
FetchAction.BatchInput batchInputParams = new FetchAction.BatchInput();
batchInputParams.label = label;
@@ -145,8 +122,6 @@
batchInputParams.refInputs = Set.of(RefInput.create(refName, true));
Response<?> response = fetchAction.apply(projectResource, batchInputParams);
- verify(deleteRefJobFactory).create(any(), eq(batchInputParams));
-
assertThat(response.statusCode()).isEqualTo(SC_ACCEPTED);
}
diff --git a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListenerTest.java b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListenerTest.java
index c81820e..a60fed6 100644
--- a/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListenerTest.java
+++ b/src/test/java/com/googlesource/gerrit/plugins/replication/pull/event/StreamEventListenerTest.java
@@ -25,7 +25,6 @@
import com.google.common.cache.CacheBuilder;
import com.google.common.collect.Lists;
import com.google.gerrit.entities.Project;
-import com.google.gerrit.entities.RefNames;
import com.google.gerrit.server.data.RefUpdateAttribute;
import com.google.gerrit.server.events.Event;
import com.google.gerrit.server.events.ProjectCreatedEvent;
@@ -36,7 +35,6 @@
import com.googlesource.gerrit.plugins.replication.pull.FetchOne;
import com.googlesource.gerrit.plugins.replication.pull.Source;
import com.googlesource.gerrit.plugins.replication.pull.SourcesCollection;
-import com.googlesource.gerrit.plugins.replication.pull.api.DeleteRefCommand;
import com.googlesource.gerrit.plugins.replication.pull.api.FetchAction;
import com.googlesource.gerrit.plugins.replication.pull.api.FetchJob;
import com.googlesource.gerrit.plugins.replication.pull.api.ProjectInitializationAction;
@@ -69,7 +67,6 @@
@Mock private FetchJob fetchJob;
@Mock private FetchJob.Factory fetchJobFactory;
@Mock private UpdateHeadCommand updateHeadCommand;
- @Mock private DeleteRefCommand deleteRefCommand;
@Captor ArgumentCaptor<FetchAction.BatchInput> batchInputCaptor;
@Mock private PullReplicationApiRequestMetrics metrics;
@Mock private SourcesCollection sources;
@@ -94,7 +91,6 @@
objectUnderTest =
new StreamEventListener(
INSTANCE_ID,
- deleteRefCommand,
updateHeadCommand,
projectInitializationAction,
workQueue,
@@ -116,22 +112,6 @@
}
@Test
- public void shouldSkipFetchForProjectDeleteEvent() {
- RefUpdatedEvent event = new RefUpdatedEvent();
- RefUpdateAttribute refUpdate = new RefUpdateAttribute();
- refUpdate.refName = RefNames.REFS_CONFIG;
- refUpdate.newRev = ObjectId.zeroId().getName();
- refUpdate.project = TEST_PROJECT;
-
- event.instanceId = REMOTE_INSTANCE_ID;
- event.refUpdate = () -> refUpdate;
-
- objectUnderTest.onEvent(event);
-
- verify(executor, never()).submit(any(Runnable.class));
- }
-
- @Test
public void shouldSkipEventWhenNotOnAllowedProjectsList() {
when(source.wouldFetchProject(any())).thenReturn(false);
@@ -151,7 +131,7 @@
}
@Test
- public void shouldDeleteRefForRefDeleteEvent() throws Exception {
+ public void shouldScheduleJobForRefDeleteEvent() throws Exception {
RefUpdatedEvent event = new RefUpdatedEvent();
RefUpdateAttribute refUpdate = new RefUpdateAttribute();
refUpdate.refName = TEST_REF_NAME;
@@ -163,8 +143,15 @@
objectUnderTest.onEvent(event);
- verify(deleteRefCommand)
- .deleteRef(Project.nameKey(TEST_PROJECT), refUpdate.refName, REMOTE_INSTANCE_ID);
+ verify(fetchJobFactory)
+ .create(eq(Project.nameKey(TEST_PROJECT)), batchInputCaptor.capture(), any());
+
+ FetchAction.BatchInput batchInput = batchInputCaptor.getValue();
+ assertThat(batchInput.label).isEqualTo(REMOTE_INSTANCE_ID);
+ FetchAction.RefInput deletedRefInput = FetchAction.RefInput.create(TEST_REF_NAME, true);
+ assertThat(batchInput.refInputs).contains(deletedRefInput);
+
+ verify(executor).submit(any(FetchJob.class));
}
@Test