Convert replace patch set in ReceiveCommits to BatchUpdate

Change-Id: I94005689341edce5c6551e8c7c4a79988728cdc6
Signed-off-by: Edwin Kempin <ekempin@google.com>
This commit is contained in:
Edwin Kempin
2016-02-16 18:07:25 +01:00
parent a8872c67f0
commit a1bfee927e
3 changed files with 493 additions and 278 deletions

View File

@@ -85,6 +85,7 @@ import com.google.gerrit.server.git.GitModule;
import com.google.gerrit.server.git.MergeUtil;
import com.google.gerrit.server.git.NotesBranchUtil;
import com.google.gerrit.server.git.ReceivePackInitializer;
import com.google.gerrit.server.git.ReplaceOp;
import com.google.gerrit.server.git.TagCache;
import com.google.gerrit.server.git.TransferConfig;
import com.google.gerrit.server.git.strategy.SubmitStrategy;
@@ -316,6 +317,7 @@ public class GerritGlobalModule extends FactoryModule {
factory(ProjectConfigValidator.Factory.class);
factory(NotesBranchUtil.Factory.class);
factory(SubmoduleSectionParser.Factory.class);
factory(ReplaceOp.Factory.class);
bind(AccountManager.class);
factory(ChangeUserName.Factory.class);

View File

@@ -20,7 +20,6 @@ import static com.google.gerrit.reviewdb.client.RefNames.REFS_CHANGES;
import static com.google.gerrit.server.change.HashtagsUtil.cleanupHashtag;
import static com.google.gerrit.server.git.MultiProgressMonitor.UNKNOWN;
import static com.google.gerrit.server.mail.MailUtil.getRecipientsFromFooters;
import static com.google.gerrit.server.mail.MailUtil.getRecipientsFromReviewers;
import static org.eclipse.jgit.lib.Constants.R_HEADS;
import static org.eclipse.jgit.lib.RefDatabase.ALL;
import static org.eclipse.jgit.transport.ReceiveCommand.Result.NOT_ATTEMPTED;
@@ -81,7 +80,6 @@ import com.google.gerrit.reviewdb.client.Project;
import com.google.gerrit.reviewdb.client.RefNames;
import com.google.gerrit.reviewdb.client.SubmoduleSubscription;
import com.google.gerrit.reviewdb.server.ReviewDb;
import com.google.gerrit.server.ApprovalCopier;
import com.google.gerrit.server.ApprovalsUtil;
import com.google.gerrit.server.ChangeMessagesUtil;
import com.google.gerrit.server.ChangeUtil;
@@ -91,8 +89,6 @@ import com.google.gerrit.server.Sequences;
import com.google.gerrit.server.account.AccountCache;
import com.google.gerrit.server.account.AccountResolver;
import com.google.gerrit.server.change.ChangeInserter;
import com.google.gerrit.server.change.ChangeKind;
import com.google.gerrit.server.change.ChangeKindCache;
import com.google.gerrit.server.change.ChangesCollection;
import com.google.gerrit.server.change.RevisionResource;
import com.google.gerrit.server.change.SetHashtagsOp;
@@ -110,10 +106,8 @@ import com.google.gerrit.server.git.MultiProgressMonitor.Task;
import com.google.gerrit.server.git.validators.CommitValidationException;
import com.google.gerrit.server.git.validators.CommitValidationMessage;
import com.google.gerrit.server.git.validators.CommitValidators;
import com.google.gerrit.server.index.ChangeIndexer;
import com.google.gerrit.server.mail.MailUtil.MailRecipients;
import com.google.gerrit.server.mail.MergedSender;
import com.google.gerrit.server.mail.ReplacePatchSetSender;
import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.notedb.ChangeUpdate;
import com.google.gerrit.server.notedb.NotesMigration;
@@ -130,7 +124,6 @@ import com.google.gerrit.server.util.LabelVote;
import com.google.gerrit.server.util.MagicBranch;
import com.google.gerrit.server.util.RequestScopePropagator;
import com.google.gerrit.util.cli.CmdLineParser;
import com.google.gwtorm.server.AtomicUpdate;
import com.google.gwtorm.server.OrmException;
import com.google.gwtorm.server.ResultSet;
import com.google.gwtorm.server.SchemaFactory;
@@ -172,7 +165,6 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.StringWriter;
import java.sql.Timestamp;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -292,19 +284,14 @@ public class ReceiveCommits {
private final ReviewDb db;
private final Sequences seq;
private final Provider<InternalChangeQuery> queryProvider;
private final ChangeData.Factory changeDataFactory;
private final ChangeNotes.Factory notesFactory;
private final ChangeUpdate.Factory updateFactory;
private final SchemaFactory<ReviewDb> schemaFactory;
private final AccountResolver accountResolver;
private final CmdLineParser.Factory optionParserFactory;
private final MergedSender.Factory mergedSenderFactory;
private final ReplacePatchSetSender.Factory replacePatchSetFactory;
private final GitReferenceUpdated gitRefUpdated;
private final PatchSetInfoFactory patchSetInfoFactory;
private final ChangeHooks hooks;
private final ApprovalsUtil approvalsUtil;
private final ApprovalCopier approvalCopier;
private final ChangeMessagesUtil cmUtil;
private final PatchSetUtil psUtil;
private final GitRepositoryManager repoManager;
@@ -318,14 +305,13 @@ public class ReceiveCommits {
private final ExecutorService sendEmailExecutor;
private final ListeningExecutorService changeUpdateExector;
private final RequestScopePropagator requestScopePropagator;
private final ChangeIndexer indexer;
private final SshInfo sshInfo;
private final AllProjectsName allProjectsName;
private final ReceiveConfig receiveConfig;
private final DynamicSet<ReceivePackInitializer> initializers;
private final ChangeKindCache changeKindCache;
private final BatchUpdate.Factory batchUpdateFactory;
private final SetHashtagsOp.Factory hashtagsFactory;
private final ReplaceOp.Factory replaceOpFactory;
private final ProjectControl projectControl;
private final Project project;
@@ -367,18 +353,13 @@ public class ReceiveCommits {
final Sequences seq,
final Provider<InternalChangeQuery> queryProvider,
final SchemaFactory<ReviewDb> schemaFactory,
final ChangeData.Factory changeDataFactory,
final ChangeNotes.Factory notesFactory,
final ChangeUpdate.Factory updateFactory,
final AccountResolver accountResolver,
final CmdLineParser.Factory optionParserFactory,
final MergedSender.Factory mergedSenderFactory,
final ReplacePatchSetSender.Factory replacePatchSetFactory,
final GitReferenceUpdated gitRefUpdated,
final PatchSetInfoFactory patchSetInfoFactory,
final ChangeHooks hooks,
final ApprovalsUtil approvalsUtil,
final ApprovalCopier approvalCopier,
final ChangeMessagesUtil cmUtil,
final PatchSetUtil psUtil,
final ProjectCache projectCache,
@@ -393,7 +374,6 @@ public class ReceiveCommits {
@SendEmailExecutor final ExecutorService sendEmailExecutor,
@ChangeUpdateExecutor ListeningExecutorService changeUpdateExector,
final RequestScopePropagator requestScopePropagator,
final ChangeIndexer indexer,
final SshInfo sshInfo,
final AllProjectsName allProjectsName,
ReceiveConfig receiveConfig,
@@ -405,29 +385,24 @@ public class ReceiveCommits {
final Provider<SubmoduleOp> subOpProvider,
final Provider<Submit> submitProvider,
final Provider<MergeOp> mergeOpProvider,
final ChangeKindCache changeKindCache,
final DynamicMap<ProjectConfigEntry> pluginConfigEntries,
final NotesMigration notesMigration,
final ChangeEditUtil editUtil,
final BatchUpdate.Factory batchUpdateFactory,
final SetHashtagsOp.Factory hashtagsFactory) throws IOException {
final SetHashtagsOp.Factory hashtagsFactory,
final ReplaceOp.Factory replaceOpFactory) throws IOException {
this.user = projectControl.getUser().asIdentifiedUser();
this.db = db;
this.seq = seq;
this.queryProvider = queryProvider;
this.changeDataFactory = changeDataFactory;
this.notesFactory = notesFactory;
this.updateFactory = updateFactory;
this.schemaFactory = schemaFactory;
this.accountResolver = accountResolver;
this.optionParserFactory = optionParserFactory;
this.mergedSenderFactory = mergedSenderFactory;
this.replacePatchSetFactory = replacePatchSetFactory;
this.gitRefUpdated = gitRefUpdated;
this.patchSetInfoFactory = patchSetInfoFactory;
this.hooks = hooks;
this.approvalsUtil = approvalsUtil;
this.approvalCopier = approvalCopier;
this.cmUtil = cmUtil;
this.psUtil = psUtil;
this.projectCache = projectCache;
@@ -441,14 +416,13 @@ public class ReceiveCommits {
this.sendEmailExecutor = sendEmailExecutor;
this.changeUpdateExector = changeUpdateExector;
this.requestScopePropagator = requestScopePropagator;
this.indexer = indexer;
this.sshInfo = sshInfo;
this.allProjectsName = allProjectsName;
this.receiveConfig = receiveConfig;
this.initializers = initializers;
this.changeKindCache = changeKindCache;
this.batchUpdateFactory = batchUpdateFactory;
this.hashtagsFactory = hashtagsFactory;
this.replaceOpFactory = replaceOpFactory;
this.projectControl = projectControl;
this.labelTypes = projectControl.getLabelTypes();
@@ -1157,7 +1131,7 @@ public class ReceiveCommits {
}
}
private static class MagicBranchInput {
static class MagicBranchInput {
private static final Splitter COMMAS = Splitter.on(',').omitEmptyStrings();
final ReceiveCommand cmd;
@@ -1956,7 +1930,6 @@ public class ReceiveCommits {
final ObjectId newCommitId;
final ReceiveCommand inputCommand;
final boolean checkMergedInto;
final Timestamp createdOn;
Change change;
ChangeControl changeCtl;
BiMap<RevCommit, PatchSet.Id> revisions;
@@ -1964,8 +1937,6 @@ public class ReceiveCommits {
ReceiveCommand prev;
ReceiveCommand cmd;
PatchSetInfo info;
ChangeMessage msg;
String mergedIntoRef;
boolean skip;
private PatchSet.Id priorPatchSet;
List<String> groups = ImmutableList.of();
@@ -1976,7 +1947,6 @@ public class ReceiveCommits {
this.newCommitId = newCommit.copy();
this.inputCommand = cmd;
this.checkMergedInto = checkMergedInto;
createdOn = TimeUtil.nowTs();
revisions = HashBiMap.create();
for (Ref ref : refs(toChange)) {
@@ -2177,54 +2147,6 @@ public class ReceiveCommits {
return Futures.makeChecked(future, INSERT_EXCEPTION);
}
private ChangeMessage newChangeMessage(ReviewDb db, ChangeKind changeKind,
Map<String, Short> approvals)
throws OrmException {
msg =
new ChangeMessage(new ChangeMessage.Key(change.getId(), ChangeUtil
.messageUUID(db)), user.getAccountId(), createdOn, psId);
msg.setMessage(renderMessageWithApprovals(psId.get(),
changeKindMessage(changeKind), approvals, scanLabels(db, approvals)));
return msg;
}
private String changeKindMessage(ChangeKind changeKind) {
switch (changeKind) {
case MERGE_FIRST_PARENT_UPDATE:
case TRIVIAL_REBASE:
case NO_CHANGE:
return ": Patch Set " + priorPatchSet.get() + " was rebased";
case NO_CODE_CHANGE:
return ": Commit message was updated";
case REWORK:
default:
return null;
}
}
private Map<String, PatchSetApproval> scanLabels(ReviewDb db,
Map<String, Short> approvals)
throws OrmException {
Map<String, PatchSetApproval> current = new HashMap<>();
// We optimize here and only retrieve current when approvals provided
if (!approvals.isEmpty()) {
for (PatchSetApproval a : approvalsUtil.byPatchSetUser(
db, changeCtl, priorPatchSet, user.getAccountId())) {
if (a.isSubmit()) {
continue;
}
LabelType lt = labelTypes.byLabel(a.getLabelId());
if (lt != null) {
current.put(lt.getName(), a);
}
}
}
return current;
}
PatchSet.Id upsertEdit() {
if (cmd.getResult() == NOT_ATTEMPTED) {
cmd.execute(rp);
@@ -2234,187 +2156,41 @@ public class ReceiveCommits {
PatchSet.Id insertPatchSet(RequestState state)
throws OrmException, IOException, RestApiException, UpdateException {
ReviewDb db = state.db;
Repository repo = state.repo;
final RevCommit newCommit = state.rw.parseCommit(newCommitId);
RevCommit newCommit = state.rw.parseCommit(newCommitId);
state.rw.parseBody(newCommit);
final Account.Id me = user.getAccountId();
final List<FooterLine> footerLines = newCommit.getFooterLines();
final MailRecipients recipients = new MailRecipients();
final PatchSet newPatchSet;
Map<String, Short> approvals = new HashMap<>();
ChangeUpdate update = updateFactory.create(changeCtl, createdOn);
update.setSubjectForCommit("Create patch set " + psId.get());
update.setPatchSetId(psId);
RevCommit priorCommit = revisions.inverse().get(priorPatchSet);
if (magicBranch != null) {
recipients.add(magicBranch.getMailRecipients());
approvals = magicBranch.labels;
Set<String> hashtags = magicBranch.hashtags;
ChangeNotes notes = changeCtl.getNotes().load();
if (!hashtags.isEmpty()) {
hashtags.addAll(notes.getHashtags());
update.setHashtags(hashtags);
}
if (magicBranch.topic != null
&& !magicBranch.topic.equals(notes.getChange().getTopic())) {
update.setTopic(magicBranch.topic);
}
ReplaceOp replaceOp = replaceOpFactory.create(requestScopePropagator,
projectControl, checkMergedInto, priorPatchSet, priorCommit, psId,
newCommit, info, groups, magicBranch, rp.getPushCertificate());
try (BatchUpdate bu = batchUpdateFactory.create(state.db, project.getNameKey(),
user, TimeUtil.nowTs())) {
bu.setRepository(state.repo, state.rw, state.ins);
bu.addOp(change.getId(), replaceOp);
bu.execute();
}
db.changes().beginTransaction(change.getId());
ChangeKind changeKind = ChangeKind.REWORK;
try {
change = db.changes().get(change.getId());
if (change == null || change.getStatus().isClosed()) {
reject(inputCommand, "change is closed");
return null;
}
List<String> newGroups = groups;
if (newGroups.isEmpty()) {
PatchSet prevPs = psUtil.current(db, update.getChangeNotes());
newGroups = prevPs != null
? prevPs.getGroups()
: ImmutableList.<String> of();
}
boolean draft = magicBranch != null && magicBranch.draft;
newPatchSet = psUtil.insert(
db, state.rw, update, psId, newCommit, draft, newGroups,
rp.getPushCertificate() != null
? rp.getPushCertificate().toTextWithSignature()
: null);
if (checkMergedInto) {
final Ref mergedInto = findMergedInto(change.getDest().get(), newCommit);
mergedIntoRef = mergedInto != null ? mergedInto.getName() : null;
}
recipients.add(getRecipientsFromFooters(
accountResolver, draft, footerLines));
recipients.remove(me);
ChangeData cd = changeDataFactory.create(db, changeCtl);
MailRecipients oldRecipients =
getRecipientsFromReviewers(cd.reviewers());
approvalCopier.copy(db, changeCtl, newPatchSet);
approvalsUtil.addReviewers(db, update, labelTypes, change, newPatchSet,
info, recipients.getReviewers(), oldRecipients.getAll());
approvalsUtil.addApprovals(db, update, labelTypes, newPatchSet,
changeCtl, approvals);
recipients.add(oldRecipients);
RevCommit priorCommit = revisions.inverse().get(priorPatchSet);
changeKind = changeKindCache.getChangeKind(
projectControl.getProjectState(), repo, priorCommit, newCommit);
cmUtil.addChangeMessage(db, update, newChangeMessage(db, changeKind,
approvals));
if (mergedIntoRef == null) {
// Change should be new, so it can go through review again.
//
change =
db.changes().atomicUpdate(change.getId(), new AtomicUpdate<Change>() {
@Override
public Change update(Change change) {
if (change.getStatus().isClosed()) {
return null;
}
if (!change.currentPatchSetId().equals(priorPatchSet)) {
return change;
}
if (magicBranch != null && magicBranch.topic != null) {
change.setTopic(magicBranch.topic);
}
if (change.getStatus() == Change.Status.DRAFT && newPatchSet.isDraft()) {
// Leave in draft status.
} else {
change.setStatus(Change.Status.NEW);
}
change.setCurrentPatchSet(info);
final List<String> idList = newCommit.getFooterLines(CHANGE_ID);
if (idList.isEmpty()) {
change.setKey(new Change.Key("I" + newCommit.name()));
} else {
change.setKey(new Change.Key(idList.get(idList.size() - 1).trim()));
}
ChangeUtil.updated(change);
return change;
}
});
if (change == null) {
db.patchSets().delete(Collections.singleton(newPatchSet));
db.changeMessages().delete(Collections.singleton(msg));
reject(inputCommand, "change is closed");
return null;
}
}
db.commit();
} finally {
db.rollback();
if (replaceOp.getRejectMessage() != null) {
reject(inputCommand, replaceOp.getRejectMessage());
return null;
}
update.commit();
groups = replaceOp.getGroups();
if (mergedIntoRef != null) {
//TODO(ekempin): mark changes as merged inside of ReplaceOp
if (replaceOp.getMergedIntoRef() != null) {
// Change was already submitted to a branch, close it.
//
markChangeMergedByPush(db, info, mergedIntoRef);
markChangeMergedByPush(db, info, replaceOp.getMergedIntoRef());
}
if (cmd.getResult() == NOT_ATTEMPTED) {
cmd.execute(rp);
}
indexer.index(db, change);
if (changeKind != ChangeKind.TRIVIAL_REBASE) {
sendEmailExecutor.submit(requestScopePropagator.wrap(new Runnable() {
@Override
public void run() {
try {
ReplacePatchSetSender cm = replacePatchSetFactory
.create(project.getNameKey(), change.getId());
cm.setFrom(me);
cm.setPatchSet(newPatchSet, info);
cm.setChangeMessage(msg);
if (magicBranch != null) {
cm.setNotify(magicBranch.notify);
}
cm.addReviewers(recipients.getReviewers());
cm.addExtraCC(recipients.getCcOnly());
cm.send();
} catch (Exception e) {
log.error("Cannot send email for new patch set " + newPatchSet.getId(), e);
}
if (mergedIntoRef != null) {
sendMergedEmail(newPatchSet, info);
}
}
@Override
public String toString() {
return "send-email newpatchset";
}
}));
}
PatchSet newPatchSet = replaceOp.getPatchSet();
gitRefUpdated.fire(project.getNameKey(), newPatchSet.getRefName(),
ObjectId.zeroId(), newCommit);
hooks.doPatchsetCreatedHook(change, newPatchSet, db);
if (mergedIntoRef != null) {
hooks.doChangeMergedHook(
change, user.getAccount(), newPatchSet, db, newCommit.getName());
}
if (!approvals.isEmpty()) {
hooks.doCommentAddedHook(change, user.getAccount(), newPatchSet,
null, approvals, db);
}
if (magicBranch != null && magicBranch.submit) {
submit(changeCtl, newPatchSet);
@@ -2549,33 +2325,6 @@ public class ReceiveCommits {
}
}
private Ref findMergedInto(final String first, final RevCommit commit) {
try {
final Map<String, Ref> all = repo.getRefDatabase().getRefs(ALL);
Ref firstRef = all.get(first);
if (firstRef != null && isMergedInto(commit, firstRef)) {
return firstRef;
}
for (Ref ref : all.values()) {
if (isHead(ref)) {
if (isMergedInto(commit, ref)) {
return ref;
}
}
}
return null;
} catch (IOException e) {
log.warn("Can't check for already submitted change", e);
return null;
}
}
private boolean isMergedInto(final RevCommit commit, final Ref ref)
throws IOException {
final RevWalk rw = rp.getRevWalk();
return rw.isMergedInto(commit, rw.parseCommit(ref.getObjectId()));
}
private void validateNewCommits(RefControl ctl, ReceiveCommand cmd) {
if (ctl.canForgeAuthor()
&& ctl.canForgeCommitter()
@@ -2845,10 +2594,6 @@ public class ReceiveCommits {
commandProgress.update(1);
}
private static boolean isHead(final Ref ref) {
return ref.getName().startsWith(Constants.R_HEADS);
}
private static boolean isHead(final ReceiveCommand cmd) {
return cmd.getRefName().startsWith(Constants.R_HEADS);
}

View File

@@ -0,0 +1,468 @@
// Copyright (C) 2016 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 static com.google.gerrit.common.FooterConstants.CHANGE_ID;
import static com.google.gerrit.server.mail.MailUtil.getRecipientsFromFooters;
import static com.google.gerrit.server.mail.MailUtil.getRecipientsFromReviewers;
import static org.eclipse.jgit.lib.RefDatabase.ALL;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
import com.google.gerrit.common.ChangeHooks;
import com.google.gerrit.common.Nullable;
import com.google.gerrit.common.data.LabelType;
import com.google.gerrit.reviewdb.client.Account;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.client.ChangeMessage;
import com.google.gerrit.reviewdb.client.PatchSet;
import com.google.gerrit.reviewdb.client.PatchSetApproval;
import com.google.gerrit.reviewdb.client.PatchSetInfo;
import com.google.gerrit.server.ApprovalCopier;
import com.google.gerrit.server.ApprovalsUtil;
import com.google.gerrit.server.ChangeMessagesUtil;
import com.google.gerrit.server.ChangeUtil;
import com.google.gerrit.server.PatchSetUtil;
import com.google.gerrit.server.account.AccountResolver;
import com.google.gerrit.server.change.ChangeKind;
import com.google.gerrit.server.change.ChangeKindCache;
import com.google.gerrit.server.git.BatchUpdate.ChangeContext;
import com.google.gerrit.server.git.BatchUpdate.Context;
import com.google.gerrit.server.git.BatchUpdate.RepoContext;
import com.google.gerrit.server.git.ReceiveCommits.MagicBranchInput;
import com.google.gerrit.server.mail.MailUtil.MailRecipients;
import com.google.gerrit.server.mail.MergedSender;
import com.google.gerrit.server.mail.ReplacePatchSetSender;
import com.google.gerrit.server.notedb.ChangeUpdate;
import com.google.gerrit.server.project.ProjectControl;
import com.google.gerrit.server.query.change.ChangeData;
import com.google.gerrit.server.util.LabelVote;
import com.google.gerrit.server.util.RequestScopePropagator;
import com.google.gwtorm.server.OrmException;
import com.google.inject.assistedinject.Assisted;
import com.google.inject.assistedinject.AssistedInject;
import org.eclipse.jgit.lib.Constants;
import org.eclipse.jgit.lib.Ref;
import org.eclipse.jgit.revwalk.RevCommit;
import org.eclipse.jgit.revwalk.RevWalk;
import org.eclipse.jgit.transport.PushCertificate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutorService;
public class ReplaceOp extends BatchUpdate.Op {
public interface Factory {
ReplaceOp create(
RequestScopePropagator requestScopePropagator,
ProjectControl projectControl,
boolean checkMergedInto,
@Assisted("priorPatchSetId") PatchSet.Id priorPatchSetId,
@Assisted("priorCommit") RevCommit priorCommit,
@Assisted("patchSetId") PatchSet.Id patchSetId,
@Assisted("commit") RevCommit commit,
PatchSetInfo info,
List<String> groups,
@Nullable MagicBranchInput magicBranch,
@Nullable PushCertificate pushCertificate);
}
private static final Logger log =
LoggerFactory.getLogger(ReplaceOp.class);
private static final String CHANGE_IS_CLOSED = "change is closed";
private final PatchSetUtil psUtil;
private final ChangeData.Factory changeDataFactory;
private final ChangeKindCache changeKindCache;
private final ChangeMessagesUtil cmUtil;
private final ChangeHooks hooks;
private final ApprovalsUtil approvalsUtil;
private final ApprovalCopier approvalCopier;
private final AccountResolver accountResolver;
private final ExecutorService sendEmailExecutor;
private final ReplacePatchSetSender.Factory replacePatchSetFactory;
private final MergedSender.Factory mergedSenderFactory;
private final RequestScopePropagator requestScopePropagator;
private final ProjectControl projectControl;
private final boolean checkMergedInto;
private final PatchSet.Id priorPatchSetId;
private final RevCommit priorCommit;
private final PatchSet.Id patchSetId;
private final RevCommit commit;
private final PatchSetInfo info;
private final MagicBranchInput magicBranch;
private final PushCertificate pushCertificate;
private List<String> groups = ImmutableList.of();
private final Map<String, Short> approvals = new HashMap<>();
private final MailRecipients recipients = new MailRecipients();
private Change change;
private PatchSet newPatchSet;
private ChangeKind changeKind;
private ChangeMessage msg;
private String rejectMessage;
private String mergedIntoRef;
@AssistedInject
ReplaceOp(PatchSetUtil psUtil,
ChangeData.Factory changeDataFactory,
ChangeKindCache changeKindCache,
ChangeMessagesUtil cmUtil,
ChangeHooks hooks,
ApprovalsUtil approvalsUtil,
ApprovalCopier approvalCopier,
AccountResolver accountResolver,
@SendEmailExecutor ExecutorService sendEmailExecutor,
ReplacePatchSetSender.Factory replacePatchSetFactory,
MergedSender.Factory mergedSenderFactory,
@Assisted RequestScopePropagator requestScopePropagator,
@Assisted ProjectControl projectControl,
@Assisted boolean checkMergedInto,
@Assisted("priorPatchSetId") PatchSet.Id priorPatchSetId,
@Assisted("priorCommit") RevCommit priorCommit,
@Assisted("patchSetId") PatchSet.Id patchSetId,
@Assisted("commit") RevCommit commit,
@Assisted PatchSetInfo info,
@Assisted List<String> groups,
@Assisted @Nullable MagicBranchInput magicBranch,
@Assisted @Nullable PushCertificate pushCertificate) {
this.psUtil = psUtil;
this.changeDataFactory = changeDataFactory;
this.changeKindCache = changeKindCache;
this.cmUtil = cmUtil;
this.hooks = hooks;
this.approvalsUtil = approvalsUtil;
this.approvalCopier = approvalCopier;
this.accountResolver = accountResolver;
this.sendEmailExecutor = sendEmailExecutor;
this.replacePatchSetFactory = replacePatchSetFactory;
this.mergedSenderFactory = mergedSenderFactory;
this.requestScopePropagator = requestScopePropagator;
this.projectControl = projectControl;
this.checkMergedInto = checkMergedInto;
this.priorPatchSetId = priorPatchSetId;
this.priorCommit = priorCommit;
this.patchSetId = patchSetId;
this.commit = commit;
this.info = info;
this.groups = groups;
this.magicBranch = magicBranch;
this.pushCertificate = pushCertificate;
}
@Override
public void updateRepo(RepoContext ctx) throws Exception {
changeKind = changeKindCache.getChangeKind(projectControl.getProjectState(),
ctx.getRepository(), priorCommit, commit);
}
@Override
public boolean updateChange(BatchUpdate.ChangeContext ctx)
throws OrmException, IOException {
change = ctx.getChange();
if (change == null || change.getStatus().isClosed()) {
rejectMessage = CHANGE_IS_CLOSED;
return false;
}
if (groups.isEmpty()) {
PatchSet prevPs = psUtil.current(ctx.getDb(), ctx.getNotes());
groups = prevPs != null
? prevPs.getGroups()
: ImmutableList.<String> of();
}
ChangeUpdate update = ctx.getUpdate(patchSetId);
update.setSubjectForCommit("Create patch set " + patchSetId.get());
if (magicBranch != null) {
recipients.add(magicBranch.getMailRecipients());
approvals.putAll(magicBranch.labels);
Set<String> hashtags = magicBranch.hashtags;
if (hashtags != null && !hashtags.isEmpty()) {
hashtags.addAll(ctx.getNotes().getHashtags());
update.setHashtags(hashtags);
}
if (magicBranch.topic != null
&& !magicBranch.topic.equals(ctx.getChange().getTopic())) {
update.setTopic(magicBranch.topic);
}
}
boolean draft = magicBranch != null && magicBranch.draft;
newPatchSet = psUtil.insert(
ctx.getDb(), ctx.getRevWalk(), update, patchSetId, commit, draft, groups,
pushCertificate != null
? pushCertificate.toTextWithSignature()
: null);
if (checkMergedInto) {
Ref mergedInto = findMergedInto(ctx, change.getDest().get(), commit);
mergedIntoRef = mergedInto != null ? mergedInto.getName() : null;
}
recipients.add(getRecipientsFromFooters(
accountResolver, draft, commit.getFooterLines()));
recipients.remove(ctx.getUser().getAccountId());
ChangeData cd = changeDataFactory.create(ctx.getDb(), ctx.getControl());
MailRecipients oldRecipients =
getRecipientsFromReviewers(cd.reviewers());
approvalCopier.copy(ctx.getDb(), ctx.getControl(), newPatchSet);
approvalsUtil.addReviewers(ctx.getDb(), update,
projectControl.getLabelTypes(), change, newPatchSet, info,
recipients.getReviewers(), oldRecipients.getAll());
approvalsUtil.addApprovals(ctx.getDb(), update,
projectControl.getLabelTypes(), newPatchSet, ctx.getControl(),
approvals);
recipients.add(oldRecipients);
msg = new ChangeMessage(
new ChangeMessage.Key(change.getId(),
ChangeUtil.messageUUID(ctx.getDb())),
ctx.getUser().getAccountId(), ctx.getWhen(), patchSetId);
msg.setMessage(renderMessageWithApprovals(patchSetId.get(),
changeKindMessage(changeKind), approvals, scanLabels(ctx, approvals)));
cmUtil.addChangeMessage(ctx.getDb(), update, msg);
if (mergedIntoRef == null) {
resetChange(ctx, msg);
}
ctx.saveChange();
return true;
}
private String changeKindMessage(ChangeKind changeKind) {
switch (changeKind) {
case MERGE_FIRST_PARENT_UPDATE:
case TRIVIAL_REBASE:
case NO_CHANGE:
return ": Patch Set " + priorPatchSetId.get() + " was rebased";
case NO_CODE_CHANGE:
return ": Commit message was updated";
case REWORK:
default:
return null;
}
}
private static String renderMessageWithApprovals(int patchSetId,
String suffix, Map<String, Short> n, Map<String, PatchSetApproval> c) {
StringBuilder msgs = new StringBuilder("Uploaded patch set " + patchSetId);
if (!n.isEmpty()) {
boolean first = true;
for (Map.Entry<String, Short> e : n.entrySet()) {
if (c.containsKey(e.getKey())
&& c.get(e.getKey()).getValue() == e.getValue()) {
continue;
}
if (first) {
msgs.append(":");
first = false;
}
msgs.append(" ")
.append(LabelVote.create(e.getKey(), e.getValue()).format());
}
}
if (!Strings.isNullOrEmpty(suffix)) {
msgs.append(suffix);
}
return msgs.append('.').toString();
}
private Map<String, PatchSetApproval> scanLabels(ChangeContext ctx,
Map<String, Short> approvals) throws OrmException {
Map<String, PatchSetApproval> current = new HashMap<>();
// We optimize here and only retrieve current when approvals provided
if (!approvals.isEmpty()) {
for (PatchSetApproval a : approvalsUtil.byPatchSetUser(ctx.getDb(),
ctx.getControl(), priorPatchSetId,
ctx.getUser().getAccountId())) {
if (a.isSubmit()) {
continue;
}
LabelType lt = projectControl.getLabelTypes().byLabel(a.getLabelId());
if (lt != null) {
current.put(lt.getName(), a);
}
}
}
return current;
}
private void resetChange(ChangeContext ctx, ChangeMessage msg)
throws OrmException {
Change change = ctx.getChange();
if (change.getStatus().isClosed()) {
ctx.getDb().patchSets().delete(Collections.singleton(newPatchSet));
ctx.getDb().changeMessages().delete(Collections.singleton(msg));
rejectMessage = CHANGE_IS_CLOSED;
return;
}
if (!change.currentPatchSetId().equals(priorPatchSetId)) {
return;
}
if (magicBranch != null && magicBranch.topic != null) {
change.setTopic(magicBranch.topic);
}
if (change.getStatus() == Change.Status.DRAFT && newPatchSet.isDraft()) {
// Leave in draft status.
} else {
change.setStatus(Change.Status.NEW);
}
change.setCurrentPatchSet(info);
List<String> idList = commit.getFooterLines(CHANGE_ID);
if (idList.isEmpty()) {
change.setKey(new Change.Key("I" + commit.name()));
} else {
change.setKey(new Change.Key(idList.get(idList.size() - 1).trim()));
}
}
@Override
public void postUpdate(final Context ctx) throws Exception {
if (changeKind != ChangeKind.TRIVIAL_REBASE) {
Runnable sender = new Runnable() {
@Override
public void run() {
try {
ReplacePatchSetSender cm = replacePatchSetFactory.create(
projectControl.getProject().getNameKey(), change.getId());
cm.setFrom(ctx.getUser().getAccountId());
cm.setPatchSet(newPatchSet, info);
cm.setChangeMessage(msg);
if (magicBranch != null && magicBranch.notify != null) {
cm.setNotify(magicBranch.notify);
}
cm.addReviewers(recipients.getReviewers());
cm.addExtraCC(recipients.getCcOnly());
cm.send();
} catch (Exception e) {
log.error("Cannot send email for new patch set " + newPatchSet.getId(), e);
}
if (mergedIntoRef != null) {
sendMergedEmail(ctx);
}
}
@Override
public String toString() {
return "send-email newpatchset";
}
};
if (requestScopePropagator != null) {
sendEmailExecutor.submit(requestScopePropagator.wrap(sender));
} else {
sender.run();
}
}
Account account = ctx.getUser().asIdentifiedUser().getAccount();
hooks.doPatchsetCreatedHook(change, newPatchSet, ctx.getDb());
if (mergedIntoRef != null) {
hooks.doChangeMergedHook(change, account, newPatchSet, ctx.getDb(),
commit.getName());
}
if (!approvals.isEmpty()) {
hooks.doCommentAddedHook(change, account, newPatchSet, null, approvals,
ctx.getDb());
}
}
private void sendMergedEmail(final Context ctx) {
sendEmailExecutor.submit(requestScopePropagator.wrap(new Runnable() {
@Override
public void run() {
try {
MergedSender cm = mergedSenderFactory
.create(projectControl.getProject().getNameKey(), change.getId());
cm.setFrom(ctx.getUser().getAccountId());
cm.setPatchSet(newPatchSet, info);
cm.send();
} catch (Exception e) {
log.error("Cannot send email for submitted patch set "
+ patchSetId, e);
}
}
@Override
public String toString() {
return "send-email merged";
}
}));
}
public PatchSet getPatchSet() {
return newPatchSet;
}
public List<String> getGroups() {
return groups;
}
public String getMergedIntoRef() {
return mergedIntoRef;
}
public String getRejectMessage() {
return rejectMessage;
}
private Ref findMergedInto(ChangeContext ctx, String first, RevCommit commit) {
try {
Map<String, Ref> all = ctx.getRepository().getRefDatabase().getRefs(ALL);
Ref firstRef = all.get(first);
if (firstRef != null && isMergedInto(ctx.getRevWalk(), commit, firstRef)) {
return firstRef;
}
for (Ref ref : all.values()) {
if (isBranch(ref)) {
if (isMergedInto(ctx.getRevWalk(), commit, ref)) {
return ref;
}
}
}
return null;
} catch (IOException e) {
log.warn("Can't check for already submitted change", e);
return null;
}
}
private static boolean isMergedInto(RevWalk rw, RevCommit commit, Ref ref)
throws IOException {
return rw.isMergedInto(commit, rw.parseCommit(ref.getObjectId()));
}
private static boolean isBranch(Ref ref) {
return ref.getName().startsWith(Constants.R_HEADS);
}
}