Merge "Support temporarily read-only changes via NoteDbChangeState"

This commit is contained in:
Dave Borowitz
2017-01-13 15:09:04 +00:00
committed by Gerrit Code Review
9 changed files with 385 additions and 37 deletions

View File

@@ -15,11 +15,15 @@
package com.google.gerrit.acceptance.server.notedb;
import static com.google.common.truth.Truth.assertThat;
import static com.google.common.truth.Truth.assert_;
import static com.google.common.truth.TruthJUnit.assume;
import static com.google.gerrit.reviewdb.client.RefNames.changeMetaRef;
import static com.google.gerrit.reviewdb.client.RefNames.refsDraftComments;
import static com.google.gerrit.server.group.SystemGroupBackend.REGISTERED_USERS;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.concurrent.TimeUnit.DAYS;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.stream.Collectors.toList;
import static org.eclipse.jgit.lib.Constants.OBJ_BLOB;
import static org.junit.Assert.fail;
@@ -83,6 +87,7 @@ import com.google.gerrit.testutil.NoteDbMode;
import com.google.gerrit.testutil.TestChanges;
import com.google.gerrit.testutil.TestTimeUtil;
import com.google.gwtorm.server.OrmException;
import com.google.gwtorm.server.OrmRuntimeException;
import com.google.inject.Inject;
import com.google.inject.Provider;
@@ -158,7 +163,7 @@ public class ChangeRebuilderIT extends AbstractDaemonTest {
@Before
public void setUp() throws Exception {
assume().that(NoteDbMode.readWrite()).isFalse();
TestTimeUtil.resetWithClockStep(1, TimeUnit.SECONDS);
TestTimeUtil.resetWithClockStep(1, SECONDS);
setNotesMigration(false, false);
}
@@ -630,7 +635,8 @@ public class ChangeRebuilderIT extends AbstractDaemonTest {
Optional.of(
NoteDbChangeState.RefState.create(
NoteDbChangeState.parse(c).getChangeMetaId(),
ImmutableMap.of(user.getId(), badSha))));
ImmutableMap.of(user.getId(), badSha))),
Optional.empty());
c.setNoteDbState(bogusState.toString());
db.changes().update(Collections.singleton(c));
@@ -1180,6 +1186,37 @@ public class ChangeRebuilderIT extends AbstractDaemonTest {
}
}
@Test
public void rebuilderRespectsReadOnlyInNoteDbChangeState() throws Exception {
TestTimeUtil.resetWithClockStep(1, SECONDS);
PushOneCommit.Result r = createChange();
PatchSet.Id psId1 = r.getPatchSetId();
Change.Id id = psId1.getParentKey();
checker.rebuildAndCheckChanges(id);
setNotesMigration(true, true);
ReviewDb db = getUnwrappedDb();
Change c = db.changes().get(id);
NoteDbChangeState state = NoteDbChangeState.parse(c);
Timestamp until =
new Timestamp(TimeUtil.nowMs() + MILLISECONDS.convert(1, DAYS));
state = state.withReadOnlyUntil(until);
c.setNoteDbState(state.toString());
db.changes().update(Collections.singleton(c));
try {
rebuilderWrapper.rebuild(db, id);
assert_().fail("expected rebuild to fail");
} catch (OrmRuntimeException e) {
assertThat(e.getMessage()).contains("read-only until");
}
TestTimeUtil.setClock(
new Timestamp(until.getTime() + MILLISECONDS.convert(1, SECONDS)));
rebuilderWrapper.rebuild(db, id);
}
private void assertChangesReadOnly(RestApiException e) throws Exception {
Throwable cause = e.getCause();
assertThat(cause).isInstanceOf(UpdateException.class);

View File

@@ -15,19 +15,23 @@
package com.google.gerrit.acceptance.server.notedb;
import static com.google.common.truth.Truth.assertThat;
import static com.google.common.truth.Truth.assert_;
import static com.google.common.truth.TruthJUnit.assume;
import static com.google.gerrit.server.notedb.NoteDbChangeState.PrimaryStorage.REVIEW_DB;
import static java.util.stream.Collectors.toList;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.gerrit.acceptance.AbstractDaemonTest;
import com.google.gerrit.acceptance.PushOneCommit;
import com.google.gerrit.common.TimeUtil;
import com.google.gerrit.extensions.api.changes.DraftInput;
import com.google.gerrit.extensions.api.changes.ReviewInput;
import com.google.gerrit.extensions.client.ChangeStatus;
import com.google.gerrit.extensions.common.ApprovalInfo;
import com.google.gerrit.extensions.common.ChangeInfo;
import com.google.gerrit.extensions.common.CommentInfo;
import com.google.gerrit.extensions.restapi.RestApiException;
import com.google.gerrit.reviewdb.client.Account;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.server.ReviewDbUtil;
@@ -36,15 +40,21 @@ import com.google.gerrit.server.git.RepoRefCache;
import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.notedb.NoteDbChangeState;
import com.google.gerrit.testutil.NoteDbMode;
import com.google.gerrit.testutil.TestTimeUtil;
import com.google.gwtorm.server.OrmRuntimeException;
import com.google.inject.Inject;
import org.eclipse.jgit.lib.Repository;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.sql.Timestamp;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
public class NoteDbPrimaryIT extends AbstractDaemonTest {
@Inject
@@ -54,6 +64,12 @@ public class NoteDbPrimaryIT extends AbstractDaemonTest {
public void setUp() throws Exception {
assume().that(NoteDbMode.get()).isEqualTo(NoteDbMode.READ_WRITE);
db = ReviewDbUtil.unwrapDb(db);
TestTimeUtil.resetWithClockStep(1, TimeUnit.SECONDS);
}
@After
public void tearDown() {
TestTimeUtil.useSystemTime();
}
@Test
@@ -162,6 +178,56 @@ public class NoteDbPrimaryIT extends AbstractDaemonTest {
assertThat(getReviewers(id)).isEmpty();
}
@Test
public void readOnlyReviewDb() throws Exception {
PushOneCommit.Result r = createChange();
Change.Id id = r.getChange().getId();
testReadOnly(id);
}
@Test
public void readOnlyNoteDb() throws Exception {
PushOneCommit.Result r = createChange();
Change.Id id = r.getChange().getId();
setNoteDbPrimary(id);
testReadOnly(id);
}
private void testReadOnly(Change.Id id) throws Exception {
Timestamp before = TimeUtil.nowTs();
Timestamp until = new Timestamp(before.getTime() + 1000 * 3600);
// Set read-only.
Change c = db.changes().get(id);
assertThat(c).named("change " + id).isNotNull();
NoteDbChangeState state = NoteDbChangeState.parse(c);
state = state.withReadOnlyUntil(until);
c.setNoteDbState(state.toString());
db.changes().update(Collections.singleton(c));
assertThat(gApi.changes().id(id.get()).get().subject)
.isEqualTo(PushOneCommit.SUBJECT);
assertThat(gApi.changes().id(id.get()).get().topic).isNull();
try {
gApi.changes().id(id.get()).topic("a-topic");
assert_().fail("expected read-only exception");
} catch (RestApiException e) {
Optional<Throwable> oe = Throwables.getCausalChain(e).stream()
.filter(x -> x instanceof OrmRuntimeException).findFirst();
assertThat(oe.isPresent())
.named("OrmRuntimeException in causal chain of " + e)
.isTrue();
assertThat(oe.get().getMessage()).contains("read-only");
}
assertThat(gApi.changes().id(id.get()).get().topic).isNull();
TestTimeUtil.setClock(new Timestamp(until.getTime() + 1000));
assertThat(gApi.changes().id(id.get()).get().subject)
.isEqualTo(PushOneCommit.SUBJECT);
gApi.changes().id(id.get()).topic("a-topic");
assertThat(gApi.changes().id(id.get()).get().topic).isEqualTo("a-topic");
}
private void setNoteDbPrimary(Change.Id id) throws Exception {
Change c = db.changes().get(id);
assertThat(c).named("change " + id).isNotNull();

View File

@@ -49,10 +49,12 @@ import com.google.gerrit.server.CurrentUser;
import com.google.gerrit.server.GerritPersonIdent;
import com.google.gerrit.server.IdentifiedUser;
import com.google.gerrit.server.config.AllUsersName;
import com.google.gerrit.server.config.GerritServerConfig;
import com.google.gerrit.server.extensions.events.GitReferenceUpdated;
import com.google.gerrit.server.index.change.ChangeIndexer;
import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.notedb.ChangeUpdate;
import com.google.gerrit.server.notedb.NoteDbChangeState;
import com.google.gerrit.server.notedb.NoteDbChangeState.PrimaryStorage;
import com.google.gerrit.server.notedb.NoteDbUpdateManager;
import com.google.gerrit.server.notedb.NoteDbUpdateManager.MismatchedStateException;
@@ -71,6 +73,7 @@ import com.google.inject.assistedinject.Assisted;
import com.google.inject.assistedinject.AssistedInject;
import org.eclipse.jgit.lib.BatchRefUpdate;
import org.eclipse.jgit.lib.Config;
import org.eclipse.jgit.lib.NullProgressMonitor;
import org.eclipse.jgit.lib.ObjectInserter;
import org.eclipse.jgit.lib.ObjectReader;
@@ -508,6 +511,7 @@ public class BatchUpdate implements AutoCloseable {
private final NotesMigration notesMigration;
private final ReviewDb db;
private final SchemaFactory<ReviewDb> schemaFactory;
private final long skewMs;
private final Project.NameKey project;
private final CurrentUser user;
@@ -533,6 +537,7 @@ public class BatchUpdate implements AutoCloseable {
@AssistedInject
BatchUpdate(
@GerritServerConfig Config cfg,
AllUsersName allUsers,
ChangeControl.GenericFactory changeControlFactory,
ChangeIndexer indexer,
@@ -568,6 +573,7 @@ public class BatchUpdate implements AutoCloseable {
this.when = when;
tz = serverIdent.getTimeZone();
order = Order.REPO_BEFORE_DB;
skewMs = NoteDbChangeState.getReadOnlySkew(cfg);
}
@Override
@@ -942,7 +948,10 @@ public class BatchUpdate implements AutoCloseable {
db.changes().beginTransaction(id);
try {
ChangeContext ctx = newChangeContext(db, repo, rw, id);
storage = PrimaryStorage.of(ctx.getChange());
NoteDbChangeState oldState = NoteDbChangeState.parse(ctx.getChange());
NoteDbChangeState.checkNotReadOnly(oldState, skewMs);
storage = PrimaryStorage.of(oldState);
if (storage == PrimaryStorage.NOTE_DB
&& !notesMigration.readChanges()) {
throw new OrmException(
@@ -1037,6 +1046,7 @@ public class BatchUpdate implements AutoCloseable {
logDebug("Failed to get change {} from unwrapped db", id);
throw new NoSuchChangeException(id);
}
NoteDbChangeState.checkNotReadOnly(c, skewMs);
}
// Pass in preloaded change to controlFor, to avoid:
// - reading from a db that does not belong to this update

View File

@@ -28,19 +28,26 @@ import com.google.common.base.Splitter;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.primitives.Longs;
import com.google.gerrit.common.Nullable;
import com.google.gerrit.common.TimeUtil;
import com.google.gerrit.reviewdb.client.Account;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.server.ReviewDbUtil;
import com.google.gerrit.server.git.RefCache;
import com.google.gwtorm.server.OrmRuntimeException;
import org.eclipse.jgit.lib.Config;
import org.eclipse.jgit.lib.ObjectId;
import java.io.IOException;
import java.sql.Timestamp;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
/**
* The state of all relevant NoteDb refs across all repos corresponding to a
@@ -52,8 +59,10 @@ import java.util.Optional;
* Serialized in one of the forms:
* <ul>
* <li>[meta-sha],[account1]=[drafts-sha],[account2]=[drafts-sha]...
* <li>R[meta-sha],[account1]=[drafts-sha],[account2]=[drafts-sha]...
* <li>R,[meta-sha],[account1]=[drafts-sha],[account2]=[drafts-sha]...
* <li>R=[read-only-until],[meta-sha],[account1]=[drafts-sha],[account2]=[drafts-sha]...
* <li>N
* <li>N=[read-only-until]
* </ul>
*
* in numeric account ID order, with hex SHA-1s for human readability.
@@ -163,11 +172,13 @@ public class NoteDbChangeState {
return null;
}
List<String> parts = Splitter.on(',').splitToList(str);
String first = parts.get(0);
Optional<Timestamp> readOnlyUntil = parseReadOnlyUntil(id, str, first);
// Only valid NOTE_DB state is "N".
String first = parts.get(0);
if (parts.size() == 1 && first.charAt(0) == NOTE_DB.code) {
return new NoteDbChangeState(id, NOTE_DB, Optional.empty());
return new NoteDbChangeState(
id, NOTE_DB, Optional.empty(), readOnlyUntil);
}
// Otherwise it must be REVIEW_DB, either "R,<RefState>" or just
@@ -179,12 +190,43 @@ public class NoteDbChangeState {
} else {
refState = RefState.parse(id, parts);
}
return new NoteDbChangeState(id, REVIEW_DB, refState);
return new NoteDbChangeState(id, REVIEW_DB, refState, readOnlyUntil);
}
throw new IllegalArgumentException(
throw invalidState(id, str);
}
private static Optional<Timestamp> parseReadOnlyUntil(Change.Id id,
String fullStr, String first) {
if (first.length() > 2 && first.charAt(1) == '=') {
Long ts = Longs.tryParse(first.substring(2));
if (ts == null) {
throw invalidState(id, fullStr);
}
return Optional.of(new Timestamp(ts));
}
return Optional.empty();
}
private static IllegalArgumentException invalidState(Change.Id id,
String str) {
return new IllegalArgumentException(
"invalid state string for change " + id + ": " + str);
}
/**
* Apply a delta to the state stored in a change entity.
* <p>
* This method does not check whether the old state was read-only; it is up to
* the caller to not violate read-only semantics when storing the change back
* in ReviewDb.
*
* @param change change entity. The delta is applied against this entity's
* {@code noteDbState} and the new state is stored back in the entity as a
* side effect.
* @param delta delta to apply.
* @return new state, equivalent to what is stored in {@code change} as a side
* effect.
*/
public static NoteDbChangeState applyDelta(Change change, Delta delta) {
if (delta == null) {
return null;
@@ -230,11 +272,21 @@ public class NoteDbChangeState {
oldState != null
? oldState.getPrimaryStorage()
: REVIEW_DB,
Optional.of(RefState.create(changeMetaId, draftIds)));
Optional.of(RefState.create(changeMetaId, draftIds)),
// Copy old read-only deadline rather than advancing it; the caller is
// still responsible for finishing the rest of its work before the lease
// runs out.
oldState != null ? oldState.getReadOnlyUntil() : Optional.empty());
change.setNoteDbState(state.toString());
return state;
}
// TODO(dborowitz): Ugly. Refactor these static methods into a Checker class
// or something. They do not belong in NoteDbChangeState itself because:
// - need to inject Config but don't want a whole Factory
// - can't be methods on NoteDbChangeState because state is nullable (though
// we could also solve this by inventing an empty-but-non-null state)
// Also we should clean up duplicated code between static/non-static methods.
public static boolean isChangeUpToDate(@Nullable NoteDbChangeState state,
RefCache changeRepoRefs, Change.Id changeId) throws IOException {
if (PrimaryStorage.of(state) == NOTE_DB) {
@@ -259,17 +311,46 @@ public class NoteDbChangeState {
return state.areDraftsUpToDate(draftsRepoRefs, accountId);
}
public static long getReadOnlySkew(Config cfg) {
return cfg.getTimeUnit(
"notedb", null, "maxTimestampSkew", 1000, TimeUnit.MILLISECONDS);
}
private static Timestamp timeForReadOnlyCheck(long skewMs) {
// Subtract some slop in case the machine that set the change's read-only
// lease has a clock behind ours.
return new Timestamp(TimeUtil.nowMs() - skewMs);
}
public static void checkNotReadOnly(@Nullable Change change, long skewMs) {
checkNotReadOnly(parse(change), skewMs);
}
public static void checkNotReadOnly(@Nullable NoteDbChangeState state,
long skewMs) {
if (state == null) {
return; // No state means ReviewDb primary non-read-only.
} else if (state.isReadOnly(timeForReadOnlyCheck(skewMs))) {
throw new OrmRuntimeException(
"change " + state.getChangeId() + " is read-only until "
+ state.getReadOnlyUntil().get());
}
}
private final Change.Id changeId;
private final PrimaryStorage primaryStorage;
private final Optional<RefState> refState;
private final Optional<Timestamp> readOnlyUntil;
public NoteDbChangeState(
Change.Id changeId,
PrimaryStorage primaryStorage,
Optional<RefState> refState) {
Optional<RefState> refState,
Optional<Timestamp> readOnlyUntil) {
this.changeId = checkNotNull(changeId);
this.primaryStorage = checkNotNull(primaryStorage);
this.refState = refState;
this.refState = checkNotNull(refState);
this.readOnlyUntil = checkNotNull(readOnlyUntil);
switch (primaryStorage) {
case REVIEW_DB:
@@ -334,23 +415,32 @@ public class NoteDbChangeState {
return true;
}
@VisibleForTesting
Change.Id getChangeId() {
public boolean isReadOnly(Timestamp now) {
return readOnlyUntil.isPresent() && now.before(readOnlyUntil.get());
}
public Optional<Timestamp> getReadOnlyUntil() {
return readOnlyUntil;
}
public NoteDbChangeState withReadOnlyUntil(Timestamp ts) {
return new NoteDbChangeState(
changeId, primaryStorage, refState, Optional.of(ts));
}
public Change.Id getChangeId() {
return changeId;
}
@VisibleForTesting
public ObjectId getChangeMetaId() {
return refState().changeMetaId();
}
@VisibleForTesting
ImmutableMap<Account.Id, ObjectId> getDraftIds() {
public ImmutableMap<Account.Id, ObjectId> getDraftIds() {
return refState().draftIds();
}
@VisibleForTesting
Optional<RefState> getRefState() {
public Optional<RefState> getRefState() {
return refState;
}
@@ -364,13 +454,37 @@ public class NoteDbChangeState {
public String toString() {
switch (primaryStorage) {
case REVIEW_DB:
// Don't include enum field, just IDs (though parse would accept it).
return refState().toString();
if (!readOnlyUntil.isPresent()) {
// Don't include enum field, just IDs (though parse would accept it).
return refState().toString();
}
return primaryStorage.code + "=" + readOnlyUntil.get().getTime()
+ "," + refState.get();
case NOTE_DB:
return NOTE_DB_PRIMARY_STATE;
if (!readOnlyUntil.isPresent()) {
return NOTE_DB_PRIMARY_STATE;
}
return primaryStorage.code + "=" + readOnlyUntil.get().getTime();
default:
throw new IllegalArgumentException(
"Unsupported PrimaryStorage: " + primaryStorage);
}
}
@Override
public int hashCode() {
return Objects.hash(changeId, primaryStorage, refState, readOnlyUntil);
}
@Override
public boolean equals(Object o) {
if (!(o instanceof NoteDbChangeState)) {
return false;
}
NoteDbChangeState s = (NoteDbChangeState) o;
return changeId.equals(s.changeId)
&& primaryStorage.equals(s.primaryStorage)
&& refState.equals(s.refState)
&& readOnlyUntil.equals(s.readOnlyUntil);
}
}

View File

@@ -18,6 +18,7 @@ import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.gerrit.extensions.config.FactoryModule;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.client.Change.Id;
import com.google.gerrit.reviewdb.server.ReviewDb;
import com.google.gerrit.server.notedb.NoteDbUpdateManager.Result;
import com.google.gerrit.server.notedb.rebuild.ChangeRebuilder;
@@ -68,6 +69,11 @@ public class NoteDbModule extends FactoryModule {
return null;
}
@Override
public Result rebuildEvenIfReadOnly(ReviewDb db, Id changeId) {
return null;
}
@Override
public Result rebuild(NoteDbUpdateManager manager,
ChangeBundle bundle) {

View File

@@ -55,10 +55,23 @@ public class TestChangeRebuilderWrapper extends ChangeRebuilder {
@Override
public Result rebuild(ReviewDb db, Change.Id changeId)
throws IOException, OrmException {
return rebuild(db, changeId, true);
}
@Override
public Result rebuildEvenIfReadOnly(ReviewDb db, Change.Id changeId)
throws IOException, OrmException {
return rebuild(db, changeId, false);
}
private Result rebuild(ReviewDb db, Change.Id changeId,
boolean checkReadOnly) throws IOException, OrmException {
if (failNextUpdate.getAndSet(false)) {
throw new IOException("Update failed");
}
Result result = delegate.rebuild(db, changeId);
Result result = checkReadOnly
? delegate.rebuild(db, changeId)
: delegate.rebuildEvenIfReadOnly(db, changeId);
if (stealNextUpdate.getAndSet(false)) {
throw new IOException("Update stolen");
}

View File

@@ -58,6 +58,9 @@ public abstract class ChangeRebuilder {
public abstract Result rebuild(ReviewDb db, Change.Id changeId)
throws IOException, OrmException;
public abstract Result rebuildEvenIfReadOnly(ReviewDb db, Change.Id changeId)
throws IOException, OrmException;
public abstract Result rebuild(NoteDbUpdateManager manager,
ChangeBundle bundle) throws IOException, OrmException;

View File

@@ -49,6 +49,7 @@ import com.google.gerrit.server.CommentsUtil;
import com.google.gerrit.server.GerritPersonIdent;
import com.google.gerrit.server.account.AccountCache;
import com.google.gerrit.server.config.AnonymousCowardName;
import com.google.gerrit.server.config.GerritServerConfig;
import com.google.gerrit.server.config.GerritServerId;
import com.google.gerrit.server.git.ChainedReceiveCommands;
import com.google.gerrit.server.notedb.ChangeBundle;
@@ -73,6 +74,7 @@ import com.google.gwtorm.server.SchemaFactory;
import com.google.inject.Inject;
import org.eclipse.jgit.errors.ConfigInvalidException;
import org.eclipse.jgit.lib.Config;
import org.eclipse.jgit.lib.ObjectId;
import org.eclipse.jgit.lib.PersonIdent;
import org.eclipse.jgit.lib.Ref;
@@ -121,9 +123,11 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
private final ProjectCache projectCache;
private final String anonymousCowardName;
private final String serverId;
private final long skewMs;
@Inject
ChangeRebuilderImpl(SchemaFactory<ReviewDb> schemaFactory,
ChangeRebuilderImpl(@GerritServerConfig Config cfg,
SchemaFactory<ReviewDb> schemaFactory,
AccountCache accountCache,
ChangeBundleReader bundleReader,
ChangeDraftUpdate.Factory draftUpdateFactory,
@@ -149,11 +153,23 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
this.projectCache = projectCache;
this.anonymousCowardName = anonymousCowardName;
this.serverId = serverId;
this.skewMs = NoteDbChangeState.getReadOnlySkew(cfg);
}
@Override
public Result rebuild(ReviewDb db, Change.Id changeId)
throws IOException, OrmException {
return rebuild(db, changeId, true);
}
@Override
public Result rebuildEvenIfReadOnly(ReviewDb db, Change.Id changeId)
throws IOException, OrmException {
return rebuild(db, changeId, false);
}
private Result rebuild(ReviewDb db, Change.Id changeId, boolean checkReadOnly)
throws IOException, OrmException {
db = ReviewDbUtil.unwrapDb(db);
// Read change just to get project; this instance is then discarded so we
// can read a consistent ChangeBundle inside a transaction.
@@ -164,7 +180,7 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
try (NoteDbUpdateManager manager =
updateManagerFactory.create(change.getProject())) {
buildUpdates(manager, bundleReader.fromReviewDb(db, changeId));
return execute(db, changeId, manager);
return execute(db, changeId, manager, checkReadOnly);
}
}
@@ -196,6 +212,12 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
@Override
public Result execute(ReviewDb db, Change.Id changeId,
NoteDbUpdateManager manager) throws OrmException, IOException {
return execute(db, changeId, manager, true);
}
public Result execute(ReviewDb db, Change.Id changeId,
NoteDbUpdateManager manager, boolean checkReadOnly)
throws OrmException, IOException {
db = ReviewDbUtil.unwrapDb(db);
Change change =
checkNoteDbState(ChangeNotes.readOneReviewDbChange(db, changeId));
@@ -210,6 +232,9 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
db.changes().atomicUpdate(changeId, new AtomicUpdate<Change>() {
@Override
public Change update(Change change) {
if (checkReadOnly) {
NoteDbChangeState.checkNotReadOnly(change, skewMs);
}
String currNoteDbState = change.getNoteDbState();
if (Objects.equals(currNoteDbState, newNoteDbState)) {
// Another thread completed the same rebuild we were about to.

View File

@@ -16,6 +16,7 @@ package com.google.gerrit.server.notedb;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.truth.Truth.assertThat;
import static com.google.gerrit.common.TimeUtil.nowTs;
import static com.google.gerrit.server.notedb.NoteDbChangeState.applyDelta;
import static com.google.gerrit.server.notedb.NoteDbChangeState.parse;
import static com.google.gerrit.server.notedb.NoteDbChangeState.PrimaryStorage.NOTE_DB;
@@ -27,13 +28,19 @@ import com.google.gerrit.reviewdb.client.Account;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.client.Project;
import com.google.gerrit.server.notedb.NoteDbChangeState.Delta;
import com.google.gerrit.server.notedb.NoteDbChangeState.RefState;
import com.google.gerrit.testutil.GerritBaseTests;
import com.google.gerrit.testutil.TestChanges;
import com.google.gerrit.testutil.TestTimeUtil;
import org.eclipse.jgit.lib.ObjectId;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.sql.Timestamp;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
/** Unit tests for {@link NoteDbChangeState}. */
public class NoteDbChangeStateTest extends GerritBaseTests {
@@ -44,6 +51,16 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
ObjectId SHA3 =
ObjectId.fromString("badc0feebadc0feebadc0feebadc0feebadc0fee");
@Before
public void setUp() {
TestTimeUtil.resetWithClockStep(1, TimeUnit.SECONDS);
}
@After
public void tearDown() {
TestTimeUtil.useSystemTime();
}
@Test
public void parseReviewDbWithoutDrafts() {
NoteDbChangeState state = parse(new Change.Id(1), SHA1.name());
@@ -51,6 +68,7 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
assertThat(state.getChangeId()).isEqualTo(new Change.Id(1));
assertThat(state.getChangeMetaId()).isEqualTo(SHA1);
assertThat(state.getDraftIds()).isEmpty();
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
assertThat(state.toString()).isEqualTo(SHA1.name());
state = parse(new Change.Id(1), "R," + SHA1.name());
@@ -58,6 +76,7 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
assertThat(state.getChangeId()).isEqualTo(new Change.Id(1));
assertThat(state.getChangeMetaId()).isEqualTo(SHA1);
assertThat(state.getDraftIds()).isEmpty();
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
assertThat(state.toString()).isEqualTo(SHA1.name());
}
@@ -73,6 +92,7 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
assertThat(state.getDraftIds()).containsExactly(
new Account.Id(1001), SHA3,
new Account.Id(2003), SHA2);
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
assertThat(state.toString()).isEqualTo(expected);
state = parse(new Change.Id(1), "R," + str);
@@ -82,23 +102,47 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
assertThat(state.getDraftIds()).containsExactly(
new Account.Id(1001), SHA3,
new Account.Id(2003), SHA2);
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
assertThat(state.toString()).isEqualTo(expected);
}
@Test
public void applyDeltaToNullWithNoNewMetaId() {
public void parseReadOnlyUntil() {
Timestamp ts = new Timestamp(12345);
String str = "R=12345," + SHA1.name();
NoteDbChangeState state = parse(new Change.Id(1), str);
assertThat(state.getPrimaryStorage()).isEqualTo(REVIEW_DB);
assertThat(state.getChangeId()).isEqualTo(new Change.Id(1));
assertThat(state.getChangeMetaId()).isEqualTo(SHA1);
assertThat(state.getReadOnlyUntil().get()).isEqualTo(ts);
assertThat(state.toString()).isEqualTo(str);
str = "N=12345";
state = parse(new Change.Id(1), str);
assertThat(state.getPrimaryStorage()).isEqualTo(NOTE_DB);
assertThat(state.getChangeId()).isEqualTo(new Change.Id(1));
assertThat(state.getRefState().isPresent()).isFalse();
assertThat(state.getReadOnlyUntil().get()).isEqualTo(ts);
assertThat(state.toString()).isEqualTo(str);
}
@Test
public void applyDeltaToNullWithNoNewMetaId() throws Exception {
Change c = newChange();
assertThat(c.getNoteDbState()).isNull();
applyDelta(c, Delta.create(c.getId(), noMetaId(), noDrafts()));
assertThat(c.getNoteDbState()).isNull();
applyDelta(c, Delta.create(c.getId(), noMetaId(),
drafts(new Account.Id(1001), zeroId())));
applyDelta(
c,
Delta.create(
c.getId(), noMetaId(),
drafts(new Account.Id(1001), zeroId())));
assertThat(c.getNoteDbState()).isNull();
}
@Test
public void applyDeltaToMetaId() {
public void applyDeltaToMetaId() throws Exception {
Change c = newChange();
applyDelta(c, Delta.create(c.getId(), metaId(SHA1), noDrafts()));
assertThat(c.getNoteDbState()).isEqualTo(SHA1.name());
@@ -116,33 +160,63 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
}
@Test
public void applyDeltaToDrafts() {
public void applyDeltaToDrafts() throws Exception {
Change c = newChange();
applyDelta(c, Delta.create(c.getId(), metaId(SHA1),
drafts(new Account.Id(1001), SHA2)));
applyDelta(
c,
Delta.create(
c.getId(), metaId(SHA1),
drafts(new Account.Id(1001), SHA2)));
assertThat(c.getNoteDbState()).isEqualTo(
SHA1.name() + ",1001=" + SHA2.name());
applyDelta(c, Delta.create(c.getId(), noMetaId(),
drafts(new Account.Id(2003), SHA3)));
applyDelta(
c,
Delta.create(
c.getId(), noMetaId(),
drafts(new Account.Id(2003), SHA3)));
assertThat(c.getNoteDbState()).isEqualTo(
SHA1.name() + ",1001=" + SHA2.name() + ",2003=" + SHA3.name());
applyDelta(c, Delta.create(c.getId(), noMetaId(),
drafts(new Account.Id(2003), zeroId())));
applyDelta(
c,
Delta.create(
c.getId(), noMetaId(),
drafts(new Account.Id(2003), zeroId())));
assertThat(c.getNoteDbState()).isEqualTo(
SHA1.name() + ",1001=" + SHA2.name());
applyDelta(c, Delta.create(c.getId(), metaId(SHA3), noDrafts()));
applyDelta(
c, Delta.create(c.getId(), metaId(SHA3), noDrafts()));
assertThat(c.getNoteDbState()).isEqualTo(
SHA3.name() + ",1001=" + SHA2.name());
}
@Test
public void applyDeltaToReadOnly() throws Exception {
Timestamp ts = nowTs();
Change c = newChange();
NoteDbChangeState state = new NoteDbChangeState(c.getId(),
REVIEW_DB,
Optional.of(RefState.create(SHA1, ImmutableMap.of())),
Optional.of(new Timestamp(ts.getTime() + 10000)));
c.setNoteDbState(state.toString());
Delta delta = Delta.create(c.getId(), metaId(SHA2), noDrafts());
applyDelta(c, delta);
assertThat(NoteDbChangeState.parse(c)).isEqualTo(
new NoteDbChangeState(
state.getChangeId(),
state.getPrimaryStorage(),
Optional.of(RefState.create(SHA2, ImmutableMap.of())),
state.getReadOnlyUntil()));
}
@Test
public void parseNoteDbPrimary() {
NoteDbChangeState state = parse(new Change.Id(1), "N");
assertThat(state.getPrimaryStorage()).isEqualTo(NOTE_DB);
assertThat(state.getRefState().isPresent()).isFalse();
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
}
@Test(expected = IllegalArgumentException.class)
@@ -151,7 +225,7 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
}
@Test
public void applyDeltaToNoteDbPrimaryIsNoOp() {
public void applyDeltaToNoteDbPrimaryIsNoOp() throws Exception {
Change c = newChange();
c.setNoteDbState("N");
applyDelta(c, Delta.create(c.getId(), metaId(SHA1),