MergeOp: Operate on ChangeSets instead of branches

Operating on the set of changes instead of the set
of target branches, comes closer to what the user wants
and may have less racing errors.

Two new classes `ChangeSet` and `MergeSuperSet` are introduced,
the former being an auto value object which is just used for passing
around which changes are meant to go in together. The latter is a helper
class used by MergeOp to calculate the superset of the given change set
to be merged.

There is no queue any more. We either succeed or we don't
In case of failure we tell the user by throwing an exception
which translates into a 409 status code on the client side. This
is a behavioral change, so we have changes in tests as well.

Also we want to regain the possibility to submit a branch without
using a topic to group a set of changes. Before when the merge
queue was still there, you would submit the changes starting at
the head of the branch and backwards. When submitting the first
change in the branch all the changes would move from the SUBMITTED
state into the merged state as then the each change has all its
dependencies met.

To make this possible without the intermediate submitted state,
you only need to submit the latest change and all its dependencies
will be automatically submitted as well.

Change-Id: Ie30ac2974facc85eddde0f17e11466266342a357
This commit is contained in:
Stefan Beller
2015-06-18 15:48:01 -07:00
parent 52227e2f32
commit c5993446eb
12 changed files with 739 additions and 362 deletions

View File

@@ -266,6 +266,13 @@ public class PushOneCommit {
assertStatus(Status.REJECTED_OTHER_REASON, expectedMessage);
}
public void assertErrorStatus() {
RemoteRefUpdate refUpdate = result.getRemoteUpdate(ref);
assertThat(refUpdate.getStatus())
.named(message(refUpdate))
.isEqualTo(Status.REJECTED_OTHER_REASON);
}
private void assertStatus(Status expectedStatus, String expectedMessage) {
RemoteRefUpdate refUpdate = result.getRemoteUpdate(ref);
assertThat(refUpdate.getStatus())

View File

@@ -123,7 +123,7 @@ public class SubmitOnPushIT extends AbstractDaemonTest {
grant(Permission.SUBMIT, project, "refs/for/refs/heads/master");
PushOneCommit.Result r =
push("refs/for/master%submit", "other change", "a.txt", "other content");
r.assertOkStatus();
r.assertErrorStatus();
r.assertChange(Change.Status.NEW, null, admin);
r.assertMessage(CommitMergeStatus.PATH_CONFLICT.getMessage());
}

View File

@@ -277,7 +277,9 @@ public class SubmitByCherryPickIT extends AbstractSubmit {
// Merge fails; change3 contains the delta "b1" -> "b2", which cannot be
// applied against tip.
submitWithConflict(change3.getChangeId());
// As change4 sits on top of change 3 we need to trigger submission there
// to include it into the mergeing
submitWithConflict(change4.getChangeId());
// change4 is a clean merge, so should succeed in the same run where change3
// failed.

View File

@@ -43,6 +43,33 @@ public class SubmitByFastForwardIT extends AbstractSubmit {
assertSubmitter(change.getChangeId(), 1);
}
@Test
public void submitTwoChangesWithFastForward() throws Exception {
PushOneCommit.Result change = createChange();
PushOneCommit.Result change2 = createChange();
approve(change.getChangeId());
submit(change2.getChangeId());
RevCommit head = getRemoteHead();
assertThat(head.getId()).isEqualTo(change2.getCommitId());
assertThat(head.getParent(0).getId()).isEqualTo(change.getCommitId());
assertSubmitter(change.getChangeId(), 1);
assertSubmitter(change2.getChangeId(), 1);
}
@Test
public void submitTwoChangesWithFastForward_missingDependency() throws Exception {
RevCommit oldHead = getRemoteHead();
PushOneCommit.Result change = createChange();
PushOneCommit.Result change2 = createChange();
submitWithConflict(change2.getChangeId());
RevCommit head = getRemoteHead();
assertThat(head.getId()).isEqualTo(oldHead.getId());
}
@Test
public void submitFastForwardNotPossible_Conflict() throws Exception {
RevCommit initialHead = getRemoteHead();

View File

@@ -53,9 +53,9 @@ import com.google.gerrit.server.IdentifiedUser;
import com.google.gerrit.server.ProjectUtil;
import com.google.gerrit.server.account.AccountsCollection;
import com.google.gerrit.server.config.GerritServerConfig;
import com.google.gerrit.server.git.ChangeSet;
import com.google.gerrit.server.git.GitRepositoryManager;
import com.google.gerrit.server.git.LabelNormalizer;
import com.google.gerrit.server.git.MergeException;
import com.google.gerrit.server.git.MergeOp;
import com.google.gerrit.server.git.VersionedMetaData.BatchMetaDataUpdate;
import com.google.gerrit.server.index.ChangeIndexer;
@@ -212,16 +212,12 @@ public class Submit implements RestModifyView<RevisionResource, SubmitInput>,
rsrc.getPatchSet().getRevision().get()));
}
List<Change> submittedChanges = submit(rsrc, caller, false);
ChangeSet submittedChanges = ChangeSet.create(submit(rsrc, caller, false));
try {
for (Change c : submittedChanges) {
// TODO(sbeller): We should make schedule return a Future, then we
// could do these all in parallel and still block until they're done.
mergeOpFactory.create(c.getDest()).merge();
}
mergeOpFactory.create(submittedChanges, caller).merge(true);
change = dbProvider.get().changes().get(change.getId());
} catch (MergeException | NoSuchChangeException e) {
} catch (NoSuchChangeException e) {
throw new OrmException("Submission failed", e);
}
@@ -691,7 +687,7 @@ public class Submit implements RestModifyView<RevisionResource, SubmitInput>,
return new RevisionResource(changes.parse(target), rsrc.getPatchSet());
}
static boolean wholeTopicEnabled(Config config) {
public static boolean wholeTopicEnabled(Config config) {
return config.getBoolean("change", null, "submitWholeTopic" , false);
}

View File

@@ -0,0 +1,66 @@
// Copyright (C) 2015 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.google.gerrit.server.git;
import com.google.auto.value.AutoValue;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSetMultimap;
import com.google.gerrit.reviewdb.client.Branch;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.client.Project;
/** A set of changes grouped together to be submitted atomically.*/
@AutoValue
public abstract class ChangeSet {
public static ChangeSet create(Iterable<Change> changes) {
ImmutableSet.Builder<Project.NameKey> pb = ImmutableSet.builder();
ImmutableSet.Builder<Branch.NameKey> bb = ImmutableSet.builder();
ImmutableSet.Builder<Change.Id> ib = ImmutableSet.builder();
ImmutableSetMultimap.Builder<Project.NameKey, Branch.NameKey> pbb =
ImmutableSetMultimap.builder();
ImmutableSetMultimap.Builder<Project.NameKey, Change.Id> pcb =
ImmutableSetMultimap.builder();
for (Change c : changes) {
Project.NameKey project = c.getDest().getParentKey();
pb.add(project);
bb.add(c.getDest());
ib.add(c.getId());
pbb.put(project, c.getDest());
pcb.put(project, c.getId());
}
return new AutoValue_ChangeSet(pb.build(), bb.build(),
ib.build(), pbb.build(), pcb.build());
}
public static ChangeSet create(Change change) {
return create(ImmutableList.of(change));
}
public abstract ImmutableSet<Project.NameKey> projects();
public abstract ImmutableSet<Branch.NameKey> branches();
public abstract ImmutableSet<Change.Id> ids();
public abstract ImmutableSetMultimap<Project.NameKey, Branch.NameKey>
branchesByProject();
public abstract ImmutableSetMultimap<Project.NameKey, Change.Id>
changesByProject();
@Override
public int hashCode() {
return ids().hashCode();
}
}

View File

@@ -1,30 +0,0 @@
// Copyright (C) 2014 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.google.gerrit.server.git;
/**
* Thrown in inserting change or patchset, e.g. OrmException or IOException.
*/
public class InsertException extends Exception {
private static final long serialVersionUID = 1L;
InsertException(final String msg) {
super(msg, null);
}
InsertException(final String msg, final Throwable why) {
super(msg, why);
}
}

View File

@@ -14,19 +14,22 @@
package com.google.gerrit.server.git;
import static java.util.concurrent.TimeUnit.HOURS;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static com.google.common.base.Preconditions.checkState;
import static org.eclipse.jgit.lib.RefDatabase.ALL;
import com.google.common.base.Optional;
import com.google.common.base.Predicate;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import com.google.gerrit.common.ChangeHooks;
import com.google.gerrit.common.Nullable;
import com.google.gerrit.common.TimeUtil;
import com.google.gerrit.common.data.SubmitRecord;
import com.google.gerrit.common.data.SubmitTypeRecord;
import com.google.gerrit.extensions.client.SubmitType;
import com.google.gerrit.extensions.restapi.ResourceConflictException;
import com.google.gerrit.reviewdb.client.Account;
import com.google.gerrit.reviewdb.client.Branch;
import com.google.gerrit.reviewdb.client.Change;
@@ -43,6 +46,9 @@ import com.google.gerrit.server.CurrentUser;
import com.google.gerrit.server.IdentifiedUser;
import com.google.gerrit.server.RemotePeer;
import com.google.gerrit.server.account.AccountCache;
import com.google.gerrit.server.change.ChangeResource;
import com.google.gerrit.server.change.RevisionResource;
import com.google.gerrit.server.change.Submit;
import com.google.gerrit.server.config.GerritRequestModule;
import com.google.gerrit.server.config.RequestScopedReviewDbProvider;
import com.google.gerrit.server.extensions.events.GitReferenceUpdated;
@@ -51,7 +57,6 @@ import com.google.gerrit.server.git.strategy.SubmitStrategyFactory;
import com.google.gerrit.server.git.validators.MergeValidationException;
import com.google.gerrit.server.git.validators.MergeValidators;
import com.google.gerrit.server.index.ChangeIndexer;
import com.google.gerrit.server.mail.MergeFailSender;
import com.google.gerrit.server.mail.MergedSender;
import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.notedb.ChangeUpdate;
@@ -95,13 +100,13 @@ import org.eclipse.jgit.revwalk.RevCommit;
import org.eclipse.jgit.revwalk.RevFlag;
import org.eclipse.jgit.revwalk.RevSort;
import org.eclipse.jgit.revwalk.RevWalk;
import org.joda.time.format.ISODateTimeFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.SocketAddress;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@@ -127,15 +132,11 @@ import java.util.concurrent.Callable;
*/
public class MergeOp {
public interface Factory {
MergeOp create(Branch.NameKey branch);
MergeOp create(ChangeSet changes, IdentifiedUser caller);
}
private static final Logger log = LoggerFactory.getLogger(MergeOp.class);
private static final long MAX_SUBMIT_WINDOW =
MILLISECONDS.convert(12, HOURS);
private final AccountCache accountCache;
private final ApprovalsUtil approvalsUtil;
private final ChangeControl.GenericFactory changeControlFactory;
@@ -143,40 +144,40 @@ public class MergeOp {
private final ChangeHooks hooks;
private final ChangeIndexer indexer;
private final ChangeMessagesUtil cmUtil;
private final ChangeNotes.Factory notesFactory;
private final ChangeUpdate.Factory updateFactory;
private final GitReferenceUpdated gitRefUpdated;
private final GitRepositoryManager repoManager;
private final IdentifiedUser.GenericFactory identifiedUserFactory;
private final MergedSender.Factory mergedSenderFactory;
private final MergeFailSender.Factory mergeFailSenderFactory;
private final MergeSuperSet mergeSuperSet;
private final MergeValidators.Factory mergeValidatorsFactory;
private final PatchSetInfoFactory patchSetInfoFactory;
private final ProjectCache projectCache;
private final Provider<InternalChangeQuery> queryProvider;
private final SchemaFactory<ReviewDb> schemaFactory;
private final Submit submit;
private final SubmitStrategyFactory submitStrategyFactory;
private final SubmoduleOp.Factory subOpFactory;
private final TagCache tagCache;
private final WorkQueue workQueue;
private final String logPrefix;
private final Branch.NameKey destBranch;
private final ListMultimap<SubmitType, CodeReviewCommit> toMerge;
private final List<CodeReviewCommit> potentiallyStillSubmittable;
private final Map<Change.Id, CodeReviewCommit> commits;
private final List<Change> toUpdate;
private final PerThreadRequestScope.Scoper threadScoper;
private final ChangeSet changes;
private final IdentifiedUser caller;
private final String logPrefix;
private ProjectState destProject;
private ReviewDb db;
private Repository repo;
private RevWalk rw;
private RevFlag canMergeFlag;
private CodeReviewCommit branchTip;
private MergeTip mergeTip;
private ObjectInserter inserter;
private PersonIdent refLogIdent;
private Map<Branch.NameKey, RefUpdate> pendingRefUpdates;
private Map<Branch.NameKey, CodeReviewCommit> openBranches;
private Map<Branch.NameKey, MergeTip> mergeTips;
@Inject
MergeOp(AccountCache accountCache,
@@ -187,23 +188,24 @@ public class MergeOp {
ChangeIndexer indexer,
Injector injector,
ChangeMessagesUtil cmUtil,
ChangeNotes.Factory notesFactory,
ChangeUpdate.Factory updateFactory,
GitReferenceUpdated gitRefUpdated,
GitRepositoryManager repoManager,
IdentifiedUser.GenericFactory identifiedUserFactory,
MergedSender.Factory mergedSenderFactory,
MergeFailSender.Factory mergeFailSenderFactory,
MergeSuperSet mergeSuperSet,
MergeValidators.Factory mergeValidatorsFactory,
PatchSetInfoFactory patchSetInfoFactory,
ProjectCache projectCache,
Provider<InternalChangeQuery> queryProvider,
SchemaFactory<ReviewDb> schemaFactory,
Submit submit,
SubmitStrategyFactory submitStrategyFactory,
SubmoduleOp.Factory subOpFactory,
TagCache tagCache,
WorkQueue workQueue,
@Assisted Branch.NameKey branch) {
@Assisted ChangeSet changes,
@Assisted IdentifiedUser caller) {
this.accountCache = accountCache;
this.approvalsUtil = approvalsUtil;
this.changeControlFactory = changeControlFactory;
@@ -211,29 +213,31 @@ public class MergeOp {
this.hooks = hooks;
this.indexer = indexer;
this.cmUtil = cmUtil;
this.notesFactory = notesFactory;
this.updateFactory = updateFactory;
this.gitRefUpdated = gitRefUpdated;
this.repoManager = repoManager;
this.identifiedUserFactory = identifiedUserFactory;
this.mergedSenderFactory = mergedSenderFactory;
this.mergeFailSenderFactory = mergeFailSenderFactory;
this.mergeSuperSet = mergeSuperSet;
this.mergeValidatorsFactory = mergeValidatorsFactory;
this.patchSetInfoFactory = patchSetInfoFactory;
this.projectCache = projectCache;
this.queryProvider = queryProvider;
this.schemaFactory = schemaFactory;
this.submit = submit;
this.submitStrategyFactory = submitStrategyFactory;
this.subOpFactory = subOpFactory;
this.tagCache = tagCache;
this.workQueue = workQueue;
logPrefix = String.format("[%s@%s]: ", branch.toString(),
ISODateTimeFormat.hourMinuteSecond().print(TimeUtil.nowMs()));
destBranch = branch;
toMerge = ArrayListMultimap.create();
potentiallyStillSubmittable = new ArrayList<>();
this.changes = changes;
this.caller = caller;
commits = new HashMap<>();
toUpdate = Lists.newArrayList();
logPrefix = String.format("[%s]: ", String.valueOf(changes.hashCode()));
pendingRefUpdates = new HashMap<>();
openBranches = new HashMap<>();
mergeTips = new HashMap<>();
Injector child = injector.createChildInjector(new AbstractModule() {
@Override
@@ -285,7 +289,7 @@ public class MergeOp {
threadScoper = child.getInstance(PerThreadRequestScope.Scoper.class);
}
private void setDestProject() throws MergeException {
private void setDestProject(Branch.NameKey destBranch) throws MergeException {
destProject = projectCache.get(destBranch.getParentKey());
if (destProject == null) {
throw new MergeException("No such project: " + destBranch.getParentKey());
@@ -298,89 +302,267 @@ public class MergeOp {
}
}
public void merge()
throws MergeException, NoSuchChangeException {
logDebug("Beginning merge attempt on {}", destBranch);
setDestProject();
private static Optional<SubmitRecord> findOkRecord(Collection<SubmitRecord> in) {
return Iterables.tryFind(in, new Predicate<SubmitRecord>() {
@Override
public boolean apply(SubmitRecord input) {
return input.status == SubmitRecord.Status.OK;
}
});
}
private List<SubmitRecord> checkSubmitRule(ChangeData cd)
throws ResourceConflictException, OrmException {
PatchSet patchSet = cd.currentPatchSet();
List<SubmitRecord> results = new SubmitRuleEvaluator(cd)
.setPatchSet(patchSet)
.evaluate();
Optional<SubmitRecord> ok = findOkRecord(results);
if (ok.isPresent()) {
// Rules supplied a valid solution.
return ImmutableList.of(ok.get());
} else if (results.isEmpty()) {
throw new IllegalStateException(String.format(
"SubmitRuleEvaluator.evaluate returned empty list for %s in %s",
patchSet.getId(),
cd.change().getProject().get()));
}
for (SubmitRecord record : results) {
switch (record.status) {
case CLOSED:
throw new ResourceConflictException("change is closed");
case RULE_ERROR:
throw new ResourceConflictException(String.format(
"rule error: %s",
record.errorMessage));
case NOT_READY:
StringBuilder msg = new StringBuilder();
for (SubmitRecord.Label lbl : record.labels) {
switch (lbl.status) {
case OK:
case MAY:
continue;
case REJECT:
if (msg.length() > 0) {
msg.append("; ");
}
msg.append("blocked by ").append(lbl.label);
continue;
case NEED:
if (msg.length() > 0) {
msg.append("; ");
}
msg.append("needs ").append(lbl.label);
continue;
case IMPOSSIBLE:
if (msg.length() > 0) {
msg.append("; ");
}
msg.append("needs ").append(lbl.label)
.append(" (check project access)");
continue;
default:
throw new IllegalStateException(String.format(
"Unsupported SubmitRecord.Label %s for %s in %s",
lbl.toString(),
patchSet.getId(),
cd.change().getProject().get()));
}
}
throw new ResourceConflictException(msg.toString());
default:
throw new IllegalStateException(String.format(
"Unsupported SubmitRecord %s for %s in %s",
record,
patchSet.getId().getId(),
cd.change().getProject().get()));
}
}
throw new IllegalStateException();
}
private void checkPermissions(ChangeSet cs)
throws ResourceConflictException, OrmException {
for (Change.Id id : cs.ids()) {
ChangeData cd = changeDataFactory.create(db, id);
if (cd.change().getStatus() != Change.Status.NEW
&& cd.change().getStatus() != Change.Status.SUBMITTED){
throw new OrmException("Change " + cd.change().getChangeId()
+ " is in state " + cd.change().getStatus());
} else {
checkSubmitRule(cd);
}
}
}
// For historic reasons we will first go into the submitted state
// TODO(sbeller): remove this when we get rid of Change.Status.SUBMITTED
private void submitAllChanges(ChangeSet cs, boolean force)
throws OrmException, ResourceConflictException, IOException {
for (Change.Id id : cs.ids()) {
ChangeData cd = changeDataFactory.create(db, id);
switch (cd.change().getStatus()) {
case ABANDONED:
throw new ResourceConflictException("Change " + cd.getId() +
" was abandoned while processing this change set");
case DRAFT:
throw new ResourceConflictException("Cannot submit draft " + cd.getId());
case NEW:
RevisionResource rsrc =
new RevisionResource(new ChangeResource(cd.changeControl(), null),
cd.currentPatchSet());
logDebug("Submitting change id {}", cd.change().getId());
submit.submit(rsrc, caller, force);
break;
case MERGED:
// we're racing here, but having it already merged is fine.
case SUBMITTED:
// ok
}
}
}
public void merge(boolean checkPermissions) throws NoSuchChangeException,
OrmException, ResourceConflictException {
logDebug("Beginning merge of {}", changes);
try {
openSchema();
openRepository();
RefUpdate branchUpdate = openBranch();
boolean reopen = false;
ListMultimap<SubmitType, Change> toSubmit =
validateChangeList(queryProvider.get().submitted(destBranch));
ListMultimap<SubmitType, CodeReviewCommit> toMergeNextTurn =
ArrayListMultimap.create();
while (!toMerge.isEmpty()) {
logDebug("Beginning merge iteration with {} left to merge",
toMerge.size());
toMergeNextTurn.clear();
Set<SubmitType> submitTypes = new HashSet<>(toMerge.keySet());
for (SubmitType submitType : submitTypes) {
if (reopen) {
logDebug("Reopening branch");
branchUpdate = openBranch();
ChangeSet cs = mergeSuperSet.completeChangeSet(db, changes);
logDebug("Calculated to merge {}", cs);
if (checkPermissions) {
logDebug("Submitting all calculated changes while "
+ "enforcing submit rules");
submitAllChanges(cs, false);
logDebug("Checking permissions");
checkPermissions(cs);
} else {
logDebug("Submitting all calculated changes ignoring submit rules");
submitAllChanges(cs, true);
}
SubmitStrategy strategy = createStrategy(submitType);
MergeTip mergeTip = preMerge(strategy, toMerge.get(submitType));
RefUpdate update = updateBranch(branchUpdate);
reopen = true;
updateChangeStatus(toSubmit.get(submitType), mergeTip);
updateSubscriptions(toSubmit.get(submitType));
if (update != null) {
fireRefUpdated(update);
try {
integrateIntoHistory(cs);
} catch (MergeException e) {
logError("Merge Conflict", e);
throw new ResourceConflictException("Merge Conflict", e);
}
}
toMerge.clear();
toMerge.putAll(toMergeNextTurn);
logDebug("Adding {} changes to merge on next run", toMerge.size());
}
updateChangeStatus(toUpdate, mergeTip);
} catch (NoSuchProjectException noProject) {
logWarn("Project " + destBranch.getParentKey() + " no longer exists,"
+ " abandoning open changes");
abandonAllOpenChanges();
} catch (OrmException e) {
throw new MergeException("Cannot query the database", e);
} catch (IOException e) {
// Anything before the merge attempt is an error
throw new OrmException(e);
} finally {
if (inserter != null) {
inserter.close();
}
if (rw != null) {
rw.close();
}
if (repo != null) {
repo.close();
}
if (db != null) {
db.close();
}
}
}
private void integrateIntoHistory(ChangeSet cs)
throws MergeException, NoSuchChangeException, ResourceConflictException {
logDebug("Beginning merge attempt on {}", changes);
Map<Branch.NameKey, ListMultimap<SubmitType, Change>> toSubmit =
new HashMap<>();
try {
openSchema();
logDebug("Perform the merges");
for (Project.NameKey project : cs.projects()) {
openRepository(project);
for (Branch.NameKey branch : cs.branchesByProject().get(project)) {
setDestProject(branch);
ListMultimap<SubmitType, Change> submitting =
validateChangeList(queryProvider.get().submitted(branch));
toSubmit.put(branch, submitting);
Set<SubmitType> submitTypes = new HashSet<>(submitting.keySet());
for (SubmitType submitType : submitTypes) {
SubmitStrategy strategy = createStrategy(branch, submitType,
getBranchTip(branch));
MergeTip mergeTip = preMerge(strategy, submitting.get(submitType),
getBranchTip(branch));
mergeTips.put(branch, mergeTip);
if (submitType != SubmitType.CHERRY_PICK) {
// For cherry picking we have relaxed atomic guarantees
// as traditionally Gerrit kept going cherry picking if one
// failed. We want to keep it for now.
updateChangeStatus(submitting.get(submitType), branch, true);
}
}
inserter.flush();
}
closeRepository();
}
logDebug("Write out the new branch tips");
for (Project.NameKey project : cs.projects()) {
openRepository(project);
for (Branch.NameKey branch : cs.branchesByProject().get(project)) {
RefUpdate update = updateBranch(branch);
pendingRefUpdates.remove(branch);
setDestProject(branch);
ListMultimap<SubmitType, Change> submitting = toSubmit.get(branch);
for (SubmitType submitType : submitting.keySet()) {
updateChangeStatus(submitting.get(submitType), branch, false);
updateSubscriptions(branch, submitting.get(submitType),
getBranchTip(branch));
}
if (update != null) {
fireRefUpdated(branch, update);
}
}
closeRepository();
}
checkState(pendingRefUpdates.isEmpty(), "programmer error: "
+ "pending ref update list not emptied");
} catch (NoSuchProjectException noProject) {
logWarn("Project " + noProject.project() + " no longer exists, "
+ "abandoning open changes");
abandonAllOpenChanges(noProject.project());
} catch (OrmException e) {
throw new MergeException("Cannot query the database", e);
} catch (IOException e) {
throw new MergeException("Cannot query the database", e);
} finally {
closeRepository();
}
}
private MergeTip preMerge(SubmitStrategy strategy,
List<CodeReviewCommit> toMerge) throws MergeException {
logDebug("Running submit strategy {} for {} commits",
strategy.getClass().getSimpleName(), toMerge.size());
mergeTip = strategy.run(branchTip, toMerge);
List<Change> submitted, CodeReviewCommit branchTip)
throws MergeException {
logDebug("Running submit strategy {} for {} commits {}",
strategy.getClass().getSimpleName(), submitted.size(), submitted);
List<CodeReviewCommit> toMerge = new ArrayList<>(submitted.size());
for (Change c : submitted) {
CodeReviewCommit commit = commits.get(c.getId());
checkState(commit != null,
"commit for %s not found by validateChangeList", c.getId());
toMerge.add(commit);
}
MergeTip mergeTip = strategy.run(branchTip, toMerge);
refLogIdent = strategy.getRefLogIdent();
logDebug("Produced {} new commits", strategy.getNewCommits().size());
commits.putAll(strategy.getNewCommits());
return mergeTip;
}
private SubmitStrategy createStrategy(SubmitType submitType)
private SubmitStrategy createStrategy(Branch.NameKey destBranch,
SubmitType submitType, CodeReviewCommit branchTip)
throws MergeException, NoSuchProjectException {
return submitStrategyFactory.create(submitType, db, repo, rw, inserter,
canMergeFlag, getAlreadyAccepted(branchTip), destBranch);
}
private void openRepository() throws MergeException, NoSuchProjectException {
Project.NameKey name = destBranch.getParentKey();
private void openRepository(Project.NameKey name)
throws MergeException, NoSuchProjectException {
try {
repo = repoManager.openRepository(name);
} catch (RepositoryNotFoundException notFound) {
@@ -398,10 +580,30 @@ public class MergeOp {
inserter = repo.newObjectInserter();
}
private RefUpdate openBranch()
throws MergeException, OrmException, NoSuchChangeException {
private void closeRepository() {
if (inserter != null) {
inserter.close();
}
if (rw != null) {
rw.close();
}
if (repo != null) {
repo.close();
}
}
private RefUpdate getPendingRefUpdate(Branch.NameKey destBranch)
throws MergeException {
if (pendingRefUpdates.containsKey(destBranch)) {
logDebug("Access cached open branch {}: {}", destBranch.get(),
openBranches.get(destBranch));
return pendingRefUpdates.get(destBranch);
}
try {
RefUpdate branchUpdate = repo.updateRef(destBranch.get());
CodeReviewCommit branchTip;
if (branchUpdate.getOldObjectId() != null) {
branchTip =
(CodeReviewCommit) rw.parseCommit(branchUpdate.getOldObjectId());
@@ -409,23 +611,29 @@ public class MergeOp {
branchTip = null;
branchUpdate.setExpectedOldObjectId(ObjectId.zeroId());
} else {
for (ChangeData cd : queryProvider.get().submitted(destBranch)) {
try {
Change c = cd.change();
setNew(c, message(c, "Change could not be merged, "
+ "because the destination branch does not exist anymore."));
} catch (OrmException e) {
log.error("Error setting change new", e);
}
}
throw new MergeException("The destination branch " + destBranch.get()
+ " does not exist anymore.");
}
logDebug("Opened branch {}: {}", destBranch.get(), branchTip);
pendingRefUpdates.put(destBranch, branchUpdate);
openBranches.put(destBranch, branchTip);
return branchUpdate;
} catch (IOException e) {
throw new MergeException("Cannot open branch", e);
}
}
private CodeReviewCommit getBranchTip(Branch.NameKey destBranch)
throws MergeException {
if (openBranches.containsKey(destBranch)) {
return openBranches.get(destBranch);
} else {
getPendingRefUpdate(destBranch);
return openBranches.get(destBranch);
}
}
private Set<RevCommit> getAlreadyAccepted(CodeReviewCommit branchTip)
throws MergeException {
Set<RevCommit> alreadyAccepted = new HashSet<>();
@@ -491,6 +699,7 @@ public class MergeOp {
}
PatchSet ps;
Branch.NameKey destBranch = chg.getDest();
try {
ps = cd.currentPatchSet();
} catch (OrmException e) {
@@ -536,8 +745,7 @@ public class MergeOp {
try {
commit = (CodeReviewCommit) rw.parseCommit(id);
} catch (IOException e) {
logError(
"Invalid commit " + idstr + " on patch set " + ps.getId(), e);
logError("Invalid commit " + idstr + " on patch set " + ps.getId(), e);
commits.put(changeId, CodeReviewCommit.revisionGone(ctl));
toUpdate.add(chg);
continue;
@@ -560,28 +768,6 @@ public class MergeOp {
continue;
}
if (branchTip != null) {
// If this commit is already merged its a bug in the queuing code
// that we got back here. Just mark it complete and move on. It's
// merged and that is all that mattered to the requestor.
//
try {
if (rw.isMergedInto(commit, branchTip)) {
logDebug("Revision {} of patch set {} is already merged",
idstr, ps.getId());
commit.setStatusCode(CommitMergeStatus.ALREADY_MERGED);
try {
setMerged(chg, null, commit);
} catch (OrmException e) {
logError("Cannot mark change " + chg.getId() + " merged", e);
}
continue;
}
} catch (IOException err) {
throw new MergeException("Cannot perform merge base test", err);
}
}
SubmitType submitType;
submitType = getSubmitType(commit.getControl(), ps);
if (submitType == null) {
@@ -593,7 +779,6 @@ public class MergeOp {
}
commit.add(canMergeFlag);
toMerge.put(submitType, commit);
toSubmit.put(submitType, chg);
}
logDebug("Submitting on this run: {}", toSubmit);
@@ -616,8 +801,13 @@ public class MergeOp {
}
}
private RefUpdate updateBranch(RefUpdate branchUpdate)
private RefUpdate updateBranch(Branch.NameKey destBranch)
throws MergeException {
RefUpdate branchUpdate = getPendingRefUpdate(destBranch);
CodeReviewCommit branchTip = getBranchTip(destBranch);
MergeTip mergeTip = mergeTips.get(destBranch);
CodeReviewCommit currentTip =
mergeTip != null ? mergeTip.getCurrentTip() : null;
if (Objects.equals(branchTip, currentTip)) {
@@ -682,11 +872,12 @@ public class MergeOp {
}
}
private void fireRefUpdated(RefUpdate branchUpdate) {
private void fireRefUpdated(Branch.NameKey destBranch,
RefUpdate branchUpdate) {
logDebug("Firing ref updated hooks for {}", branchUpdate.getName());
gitRefUpdated.fire(destBranch.getParentKey(), branchUpdate);
hooks.doRefUpdatedHook(destBranch, branchUpdate,
getAccount(mergeTip.getCurrentTip()));
getAccount(mergeTips.get(destBranch).getCurrentTip()));
}
private Account getAccount(CodeReviewCommit codeReviewCommit) {
@@ -707,9 +898,16 @@ public class MergeOp {
return "";
}
private void updateChangeStatus(List<Change> submitted, MergeTip mergeTip)
throws NoSuchChangeException {
private void updateChangeStatus(List<Change> submitted,
Branch.NameKey destBranch, boolean dryRun)
throws NoSuchChangeException, MergeException, ResourceConflictException {
if (!dryRun) {
logDebug("Updating change status for {} changes", submitted.size());
} else {
logDebug("Checking change state for {} changes in a dry run",
submitted.size());
}
MergeTip mergeTip = mergeTips.get(destBranch);
for (Change c : submitted) {
CodeReviewCommit commit = commits.get(c.getId());
CommitMergeStatus s = commit != null ? commit.getStatusCode() : null;
@@ -729,20 +927,27 @@ public class MergeOp {
ObjectId mergeResultRev =
mergeTip != null ? mergeTip.getMergeResults().get(commit) : null;
try {
ChangeMessage msg;
switch (s) {
case CLEAN_MERGE:
if (!dryRun) {
setMerged(c, message(c, txt + getByAccountName(commit)),
mergeResultRev);
}
break;
case CLEAN_REBASE:
case CLEAN_PICK:
if (!dryRun) {
setMerged(c, message(c, txt + " as " + commit.name()
+ getByAccountName(commit)), mergeResultRev);
}
break;
case ALREADY_MERGED:
if (!dryRun) {
setMerged(c, null, mergeResultRev);
}
break;
case PATH_CONFLICT:
@@ -755,17 +960,18 @@ public class MergeOp {
case INVALID_PROJECT_CONFIGURATION_PARENT_PROJECT_NOT_FOUND:
case INVALID_PROJECT_CONFIGURATION_ROOT_PROJECT_CANNOT_HAVE_PARENT:
case SETTING_PARENT_PROJECT_ONLY_ALLOWED_BY_ADMIN:
setNew(commit, message(c, txt));
break;
setNew(commit.notes(), message(c, txt));
throw new ResourceConflictException("Cannot merge " + commit.name()
+ "\n" + s.getMessage());
case MISSING_DEPENDENCY:
logDebug("Change {} is missing dependency", c.getId());
potentiallyStillSubmittable.add(commit);
break;
throw new MergeException("Cannot merge " + commit.name() + "\n"
+ s.getMessage());
case REVISION_GONE:
logDebug("Commit not found for change {}", c.getId());
ChangeMessage msg = new ChangeMessage(
msg = new ChangeMessage(
new ChangeMessage.Key(
c.getId(),
ChangeUtil.messageUUID(db)),
@@ -773,13 +979,13 @@ public class MergeOp {
TimeUtil.nowTs(),
c.currentPatchSetId());
msg.setMessage("Failed to read commit for this patch set");
sendMergeFail(commit.notes(), msg, false);
break;
setNew(commit.notes(), msg);
throw new MergeException(msg.getMessage());
default:
setNew(commit,
message(c, "Unspecified merge failure: " + s.name()));
break;
msg = message(c, "Unspecified merge failure: " + s.name());
setNew(commit.notes(), msg);
throw new MergeException(msg.getMessage());
}
} catch (OrmException | IOException err) {
logWarn("Error updating change status for " + c.getId(), err);
@@ -787,7 +993,9 @@ public class MergeOp {
}
}
private void updateSubscriptions(List<Change> submitted) {
private void updateSubscriptions(Branch.NameKey destBranch,
List<Change> submitted, CodeReviewCommit branchTip) {
MergeTip mergeTip = mergeTips.get(destBranch);
if (mergeTip != null
&& (branchTip == null || branchTip != mergeTip.getCurrentTip())) {
logDebug("Updating submodule subscriptions for {} changes",
@@ -799,8 +1007,8 @@ public class MergeOp {
try {
subOp.update();
} catch (SubmoduleException e) {
logError(
"The gitLinks were not updated according to the subscriptions" , e);
logError("The gitLinks were not updated according to the subscriptions",
e);
}
}
}
@@ -938,88 +1146,10 @@ public class MergeOp {
c, identifiedUserFactory.create(c.getOwner()));
}
private void setNew(CodeReviewCommit c, ChangeMessage msg)
private void setNew(ChangeNotes notes, final ChangeMessage msg)
throws NoSuchChangeException, IOException {
sendMergeFail(c.notes(), msg, true);
}
Change c = notes.getChange();
private void setNew(Change c, ChangeMessage msg)
throws NoSuchChangeException, IOException {
sendMergeFail(notesFactory.create(c), msg, true);
}
private enum RetryStatus {
UNSUBMIT, RETRY_NO_MESSAGE, RETRY_ADD_MESSAGE
}
private RetryStatus getRetryStatus(
@Nullable PatchSetApproval submitter,
ChangeMessage msg,
ChangeNotes notes) {
Change.Id id = notes.getChangeId();
if (submitter != null) {
long sinceMs = TimeUtil.nowMs() - submitter.getGranted().getTime();
if (sinceMs > MAX_SUBMIT_WINDOW) {
logDebug("Change {} submitted {}ms ago, unsubmitting", id, sinceMs);
return RetryStatus.UNSUBMIT;
} else {
logDebug("Change {} submitted {}ms ago, within window", id, sinceMs);
}
} else {
logDebug("No submitter for change {}", id);
}
try {
ChangeMessage last = Iterables.getLast(cmUtil.byChange(db, notes));
if (last != null) {
if (Objects.equals(last.getAuthor(), msg.getAuthor())
&& Objects.equals(last.getMessage(), msg.getMessage())) {
long lastMs = last.getWrittenOn().getTime();
long msgMs = msg.getWrittenOn().getTime();
long sinceMs = msgMs - lastMs;
if (sinceMs > MAX_SUBMIT_WINDOW) {
logDebug("Last message for change {} was {}ms ago, unsubmitting",
id, sinceMs);
return RetryStatus.UNSUBMIT;
} else {
logDebug("Last message for change {} was {}ms ago, within window",
id, sinceMs);
return RetryStatus.RETRY_NO_MESSAGE;
}
} else {
logDebug("Last message for change {} differed, adding message", id);
}
}
return RetryStatus.RETRY_ADD_MESSAGE;
} catch (OrmException err) {
logWarn("Cannot check previous merge failure, unsubmitting", err);
return RetryStatus.UNSUBMIT;
}
}
private void sendMergeFail(ChangeNotes notes, final ChangeMessage msg,
boolean makeNew) throws NoSuchChangeException, IOException {
logDebug("Possibly sending merge failure notification for {}",
notes.getChangeId());
PatchSetApproval submitter = null;
try {
submitter = approvalsUtil.getSubmitter(
db, notes, notes.getChange().currentPatchSetId());
} catch (Exception e) {
logError("Cannot get submitter for change " + notes.getChangeId(), e);
}
if (!makeNew) {
RetryStatus retryStatus = getRetryStatus(submitter, msg, notes);
if (retryStatus == RetryStatus.RETRY_NO_MESSAGE) {
return;
} else if (retryStatus == RetryStatus.UNSUBMIT) {
makeNew = true;
}
}
final boolean setStatusNew = makeNew;
final Change c = notes.getChange();
Change change = null;
ChangeUpdate update = null;
try {
@@ -1031,9 +1161,7 @@ public class MergeOp {
@Override
public Change update(Change c) {
if (c.getStatus().isOpen()) {
if (setStatusNew) {
c.setStatus(Change.Status.NEW);
}
ChangeUtil.updated(c);
}
return c;
@@ -1056,55 +1184,15 @@ public class MergeOp {
if (update != null) {
update.commit();
}
indexer.index(db, change);
final PatchSetApproval from = submitter;
try {
threadScoper.scope(new Callable<Void>() {
@Override
public Void call() throws Exception {
workQueue.getDefaultQueue()
.submit(new Runnable() {
@Override
public void run() {
PatchSet patchSet;
try {
ReviewDb reviewDb = schemaFactory.open();
try {
patchSet = reviewDb.patchSets().get(c.currentPatchSetId());
} finally {
reviewDb.close();
}
} catch (Exception e) {
logError("Cannot send email notifications about merge failure", e);
return;
}
PatchSetApproval submitter = null;
try {
MergeFailSender cm = mergeFailSenderFactory.create(c.getId());
if (from != null) {
cm.setFrom(from.getAccountId());
}
cm.setPatchSet(patchSet);
cm.setChangeMessage(msg);
cm.send();
submitter = approvalsUtil.getSubmitter(
db, notes, notes.getChange().currentPatchSetId());
} catch (Exception e) {
logError("Cannot send email notifications about merge failure", e);
logError("Cannot get submitter for change " + notes.getChangeId(), e);
}
}
@Override
public String toString() {
return "send-email merge-failed";
}
});
return null;
}
}).call();
} catch (Exception e) {
logError("internal server error", e);
}
if (submitter != null) {
try {
hooks.doMergeFailedHook(c,
@@ -1116,24 +1204,18 @@ public class MergeOp {
}
}
private void abandonAllOpenChanges() throws NoSuchChangeException {
Exception err = null;
private void abandonAllOpenChanges(Project.NameKey destProject)
throws NoSuchChangeException {
try {
openSchema();
for (ChangeData cd
: queryProvider.get().byProjectOpen(destBranch.getParentKey())) {
: queryProvider.get().byProjectOpen(destProject)) {
abandonOneChange(cd.change());
}
db.close();
db = null;
} catch (IOException e) {
err = e;
} catch (OrmException e) {
err = e;
}
if (err != null) {
logWarn("Cannot abandon changes for deleted project "
+ destBranch.getParentKey().get(), err);
} catch (IOException | OrmException e) {
logWarn("Cannot abandon changes for deleted project ", e);
}
}
@@ -1200,13 +1282,15 @@ public class MergeOp {
private void logError(String msg, Throwable t) {
if (log.isErrorEnabled()) {
if (t != null) {
log.error(logPrefix + msg, t);
} else {
log.error(logPrefix + msg);
}
}
}
private void logError(String msg) {
if (log.isErrorEnabled()) {
log.error(logPrefix + msg);
}
logError(msg, null);
}
}

View File

@@ -0,0 +1,196 @@
// Copyright (C) 2015 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.google.gerrit.server.git;
import com.google.common.base.Strings;
import com.google.gerrit.common.data.SubmitTypeRecord;
import com.google.gerrit.extensions.client.SubmitType;
import com.google.gerrit.reviewdb.client.Branch;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.client.PatchSet;
import com.google.gerrit.reviewdb.client.Project;
import com.google.gerrit.reviewdb.server.ReviewDb;
import com.google.gerrit.server.change.Submit;
import com.google.gerrit.server.config.GerritServerConfig;
import com.google.gerrit.server.project.SubmitRuleEvaluator;
import com.google.gerrit.server.query.change.ChangeData;
import com.google.gerrit.server.query.change.InternalChangeQuery;
import com.google.gwtorm.server.OrmException;
import com.google.inject.Inject;
import com.google.inject.Provider;
import com.google.inject.Singleton;
import org.eclipse.jgit.errors.IncorrectObjectTypeException;
import org.eclipse.jgit.errors.MissingObjectException;
import org.eclipse.jgit.lib.Config;
import org.eclipse.jgit.lib.ObjectId;
import org.eclipse.jgit.lib.Ref;
import org.eclipse.jgit.lib.Repository;
import org.eclipse.jgit.revwalk.RevCommit;
import org.eclipse.jgit.revwalk.RevSort;
import org.eclipse.jgit.revwalk.RevWalk;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
/**
* Calculates the minimal superset of changes required to be merged.
* <p>
* This includes all parents between a change and the tip of its target
* branch for the merging/rebasing submit strategies. For the cherry-pick
* strategy no additional changes are included.
* <p>
* If change.submitWholeTopic is enabled, also all changes of the topic
* and their parents are included.
*/
@Singleton
public class MergeSuperSet {
private static final Logger log = LoggerFactory.getLogger(MergeOp.class);
private final ChangeData.Factory changeDataFactory;
private final Provider<InternalChangeQuery> queryProvider;
private final GitRepositoryManager repoManager;
private final Config cfg;
@Inject
MergeSuperSet(@GerritServerConfig Config cfg,
ChangeData.Factory changeDataFactory,
Provider<InternalChangeQuery> queryProvider,
GitRepositoryManager repoManager) {
this.cfg = cfg;
this.changeDataFactory = changeDataFactory;
this.queryProvider = queryProvider;
this.repoManager = repoManager;
}
public ChangeSet completeChangeSet(ReviewDb db, ChangeSet changes)
throws MissingObjectException, IncorrectObjectTypeException, IOException,
OrmException {
if (Submit.wholeTopicEnabled(cfg)) {
return completeChangeSetIncludingTopics(db, changes);
} else {
return completeChangeSetWithoutTopic(db, changes);
}
}
private ChangeSet completeChangeSetWithoutTopic(ReviewDb db, ChangeSet changes)
throws MissingObjectException, IncorrectObjectTypeException, IOException,
OrmException {
List<Change> ret = new ArrayList<>();
for (Project.NameKey project : changes.projects()) {
try (Repository repo = repoManager.openRepository(project);
RevWalk rw = CodeReviewCommit.newRevWalk(repo)) {
for (Change.Id cId : changes.changesByProject().get(project)) {
ChangeData cd = changeDataFactory.create(db, cId);
SubmitTypeRecord r = new SubmitRuleEvaluator(cd).getSubmitType();
if (r.status != SubmitTypeRecord.Status.OK) {
logErrorAndThrow("Failed to get submit type for " + cd.getId());
}
if (r.type == SubmitType.CHERRY_PICK) {
ret.add(cd.change());
continue;
}
// Get the underlying git commit object
PatchSet ps = cd.currentPatchSet();
String objIdStr = ps.getRevision().get();
RevCommit commit = rw.parseCommit(ObjectId.fromString(objIdStr));
// Collect unmerged ancestors
Branch.NameKey destBranch = cd.change().getDest();
repo.getRefDatabase().refresh();
Ref ref = repo.getRefDatabase().getRef(destBranch.get());
if (ref == null) {
ret.add(cd.change());
// A new empty branch doesn't have additional changes
continue;
}
rw.reset();
rw.sort(RevSort.TOPO);
rw.markStart(commit);
RevCommit head = rw.parseCommit(ref.getObjectId());
rw.markUninteresting(head);
List<String> hashes = new ArrayList<>();
for (RevCommit c : rw) {
hashes.add(c.name());
}
if (!hashes.isEmpty()) {
// Merged changes are ok to exclude
List<ChangeData> destChanges = queryProvider.get()
.byCommitsOnBranchNotMerged(cd.change().getDest(), hashes);
for (ChangeData chd : destChanges) {
Change chg = chd.change();
ret.add(chg);
}
}
}
}
}
return ChangeSet.create(ret);
}
private ChangeSet completeChangeSetIncludingTopics(
ReviewDb db, ChangeSet changes) throws MissingObjectException,
IncorrectObjectTypeException, IOException, OrmException {
Set<String> topicsTraversed = new HashSet<>();
boolean done = false;
ChangeSet newCs = completeChangeSetWithoutTopic(db, changes);
while (!done) {
List<Change> chgs = new ArrayList<>();
done = true;
for (Change.Id cId : newCs.ids()) {
// TODO(sbeller): Cache the change data here and in completeChangeSet
// There is no need to reread it a few times.
ChangeData cd = changeDataFactory.create(db, cId);
chgs.add(cd.change());
String topic = cd.change().getTopic();
if (!Strings.isNullOrEmpty(topic) && !topicsTraversed.contains(topic)) {
for (ChangeData addCd : queryProvider.get().byTopicOpen(topic)) {
chgs.add(addCd.change());
}
done = false;
topicsTraversed.add(topic);
}
}
changes = ChangeSet.create(chgs);
newCs = completeChangeSetWithoutTopic(db, changes);
}
return newCs;
}
private void logError(String msg) {
if (log.isErrorEnabled()) {
log.error(msg);
}
}
private void logErrorAndThrow(String msg) throws OrmException {
logError(msg);
throw new OrmException(msg);
}
}

View File

@@ -65,6 +65,7 @@ import com.google.gerrit.common.data.PermissionRule;
import com.google.gerrit.extensions.registration.DynamicMap;
import com.google.gerrit.extensions.registration.DynamicMap.Entry;
import com.google.gerrit.extensions.restapi.ResourceConflictException;
import com.google.gerrit.extensions.restapi.RestApiException;
import com.google.gerrit.reviewdb.client.Account;
import com.google.gerrit.reviewdb.client.Branch;
import com.google.gerrit.reviewdb.client.Change;
@@ -258,17 +259,14 @@ public class ReceiveCommits {
}
}
private static final Function<Exception, InsertException> INSERT_EXCEPTION =
new Function<Exception, InsertException>() {
private static final Function<Exception, RestApiException> INSERT_EXCEPTION =
new Function<Exception, RestApiException>() {
@Override
public InsertException apply(Exception input) {
if (input instanceof OrmException) {
return new InsertException("ORM error", input);
public RestApiException apply(Exception input) {
if (input instanceof RestApiException) {
return (RestApiException) input;
}
if (input instanceof IOException) {
return new InsertException("IO error", input);
}
return new InsertException("Error inserting change/patchset", input);
return new RestApiException("Error inserting change/patchset", input);
}
};
@@ -736,7 +734,7 @@ public class ReceiveCommits {
if (replace.insertPatchSet().checkedGet() != null) {
replace.inputCommand.setResult(OK);
}
} catch (IOException | InsertException err) {
} catch (IOException | RestApiException err) {
reject(replace.inputCommand, "internal server error");
log.error(String.format(
"Cannot add patch set to %d of %s",
@@ -785,7 +783,7 @@ public class ReceiveCommits {
}
try {
List<CheckedFuture<?, InsertException>> futures = Lists.newArrayList();
List<CheckedFuture<?, RestApiException>> futures = Lists.newArrayList();
for (ReplaceRequest replace : replaceByChange.values()) {
if (magicBranch != null && replace.inputCommand == magicBranch.cmd) {
futures.add(replace.insertPatchSet());
@@ -800,13 +798,19 @@ public class ReceiveCommits {
futures.add(update.updateGroups());
}
for (CheckedFuture<?, InsertException> f : futures) {
for (CheckedFuture<?, RestApiException> f : futures) {
f.checkedGet();
}
magicBranch.cmd.setResult(OK);
} catch (InsertException err) {
log.error("Can't insert change/patchset for " + project.getName(), err);
reject(magicBranch.cmd, "internal server error");
} catch (RestApiException err) {
log.error("Can't insert change/patchset for " + project.getName()
+ ". " + err.getMessage(), err);
String rejection = "internal server error";
if (err.getCause() != null) {
rejection += ": " + err.getCause().getMessage();
}
reject(magicBranch.cmd, rejection);
} catch (IOException err) {
log.error("Can't read commits for " + project.getName(), err);
reject(magicBranch.cmd, "internal server error");
@@ -1703,14 +1707,15 @@ public class ReceiveCommits {
ins.getPatchSet().getRefName());
}
CheckedFuture<Void, InsertException> insertChange() throws IOException {
CheckedFuture<Void, RestApiException> insertChange() throws IOException {
rp.getRevWalk().parseBody(commit);
final Thread caller = Thread.currentThread();
ListenableFuture<Void> future = changeUpdateExector.submit(
requestScopePropagator.wrap(new Callable<Void>() {
@Override
public Void call() throws OrmException, IOException {
public Void call() throws OrmException, IOException,
ResourceConflictException {
if (caller == Thread.currentThread()) {
insertChange(db);
} else {
@@ -1727,7 +1732,8 @@ public class ReceiveCommits {
return Futures.makeChecked(future, INSERT_EXCEPTION);
}
private void insertChange(ReviewDb db) throws OrmException, IOException {
private void insertChange(ReviewDb db) throws OrmException, IOException,
ResourceConflictException {
final PatchSet ps = ins.setGroups(groups).getPatchSet();
final Account.Id me = currentUser.getAccountId();
final List<FooterLine> footerLines = commit.getFooterLines();
@@ -1763,7 +1769,7 @@ public class ReceiveCommits {
}
private void submit(ChangeControl changeCtl, PatchSet ps)
throws OrmException, IOException{
throws OrmException, IOException, ResourceConflictException {
Submit submit = submitProvider.get();
RevisionResource rsrc = new RevisionResource(changes.parse(changeCtl), ps);
List<Change> changes;
@@ -1773,13 +1779,14 @@ public class ReceiveCommits {
} catch (ResourceConflictException e) {
throw new IOException(e);
}
addMessage("");
for (Change c : changes) {
try {
mergeFactory.create(c.getDest()).merge();
} catch (MergeException | NoSuchChangeException e) {
mergeFactory.create(ChangeSet.create(changes),
(IdentifiedUser) changeCtl.getCurrentUser()).merge(false);
} catch (NoSuchChangeException e) {
throw new OrmException(e);
}
addMessage("");
for (Change c : changes) {
c = db.changes().get(c.getId());
switch (c.getStatus()) {
case SUBMITTED:
@@ -2074,7 +2081,7 @@ public class ReceiveCommits {
newPatchSet.getRefName());
}
CheckedFuture<PatchSet.Id, InsertException> insertPatchSet()
CheckedFuture<PatchSet.Id, RestApiException> insertPatchSet()
throws IOException {
rp.getRevWalk().parseBody(newCommit);
@@ -2082,7 +2089,8 @@ public class ReceiveCommits {
ListenableFuture<PatchSet.Id> future = changeUpdateExector.submit(
requestScopePropagator.wrap(new Callable<PatchSet.Id>() {
@Override
public PatchSet.Id call() throws OrmException, IOException, NoSuchChangeException {
public PatchSet.Id call() throws OrmException, IOException,
NoSuchChangeException, ResourceConflictException {
try {
if (magicBranch != null && magicBranch.edit) {
return upsertEdit();
@@ -2133,7 +2141,8 @@ public class ReceiveCommits {
return newPatchSet.getId();
}
PatchSet.Id insertPatchSet(ReviewDb db) throws OrmException, IOException {
PatchSet.Id insertPatchSet(ReviewDb db) throws OrmException, IOException,
ResourceConflictException {
final Account.Id me = currentUser.getAccountId();
final List<FooterLine> footerLines = newCommit.getFooterLines();
final MailRecipients recipients = new MailRecipients();
@@ -2331,7 +2340,7 @@ public class ReceiveCommits {
}
}
CheckedFuture<Void, InsertException> updateGroups() {
CheckedFuture<Void, RestApiException> updateGroups() {
final Thread caller = Thread.currentThread();
ListenableFuture<Void> future = changeUpdateExector.submit(
requestScopePropagator.wrap(new Callable<Void>() {
@@ -2590,7 +2599,7 @@ public class ReceiveCommits {
new ArrayList<Change>(),
new HashMap<Change.Id, CodeReviewCommit>(),
currentUser.getAccount()).update();
} catch (InsertException e) {
} catch (RestApiException e) {
log.error("Can't insert patchset", e);
} catch (IOException | OrmException e) {
log.error("Can't scan for changes to close", e);

View File

@@ -21,6 +21,7 @@ public class NoSuchProjectException extends Exception {
private static final long serialVersionUID = 1L;
private static final String MESSAGE = "Project not found: ";
private final Project.NameKey project;
public NoSuchProjectException(final Project.NameKey key) {
this(key, null);
@@ -28,5 +29,10 @@ public class NoSuchProjectException extends Exception {
public NoSuchProjectException(final Project.NameKey key, final Throwable why) {
super(MESSAGE + key.toString(), why);
project = key;
}
public Project.NameKey project() {
return project;
}
}

View File

@@ -16,6 +16,7 @@ package com.google.gerrit.server.query.change;
import static com.google.gerrit.server.query.Predicate.and;
import static com.google.gerrit.server.query.Predicate.or;
import static com.google.gerrit.server.query.Predicate.not;
import static com.google.gerrit.server.query.change.ChangeStatusPredicate.open;
import com.google.gerrit.common.Nullable;
@@ -126,6 +127,19 @@ public class InternalChangeQuery {
open()));
}
public List<ChangeData> byCommitsOnBranchNotMerged(Branch.NameKey branch,
List<String> hashes) throws OrmException {
List<Predicate<ChangeData>> commits = new ArrayList<>();
for (String s : hashes) {
commits.add(commit(AbbreviatedObjectId.fromString(s)));
}
return query(and(
ref(branch),
project(branch.getParentKey()),
not(status(Change.Status.MERGED)),
or(commits)));
}
public List<ChangeData> byProjectOpen(Project.NameKey project)
throws OrmException {
return query(and(project(project), open()));