Support temporarily read-only changes via NoteDbChangeState

Add a new piece of data to NoteDbChangeState indicating that a change
is read-only until a specified future time.  We use a timestamp
instead of a separate state to support a migration step where the job
doing the migration might die, so the change automatically becomes
read-write again without manual intervention.

This read-only state only needs to be checked while updating
ReviewDb, which includes an atomic update to the noteDbState field.
There are exactly two locations that do this atomic update:
ChangeRebuilderImpl#execute and BatchUpdate.ChangeTask#call.

We don't bother with attempting to double-check this field when doing
the corresponding NoteDb write, for example from NoteDbUpdateManager.
It's not possible when we're about to write to NoteDb to do a
race-free atomic read of both the read-only state from ReviewDb and
the ref in NoteDb that we're about to update. So, we just let the
NoteDb write attempt proceed. This turns out to not affect the
correctness of the migration process; the race condition is explained
in more detail in the commit that implements the migration process.

Change-Id: If21353582f678b8788285bc2e9f7b50a5c14f6d4
This commit is contained in:
Dave Borowitz
2016-10-24 14:51:22 -04:00
parent f125df6aa6
commit a96742c035
9 changed files with 385 additions and 37 deletions

View File

@@ -15,11 +15,15 @@
package com.google.gerrit.acceptance.server.notedb; package com.google.gerrit.acceptance.server.notedb;
import static com.google.common.truth.Truth.assertThat; 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.common.truth.TruthJUnit.assume;
import static com.google.gerrit.reviewdb.client.RefNames.changeMetaRef; import static com.google.gerrit.reviewdb.client.RefNames.changeMetaRef;
import static com.google.gerrit.reviewdb.client.RefNames.refsDraftComments; import static com.google.gerrit.reviewdb.client.RefNames.refsDraftComments;
import static com.google.gerrit.server.group.SystemGroupBackend.REGISTERED_USERS; import static com.google.gerrit.server.group.SystemGroupBackend.REGISTERED_USERS;
import static java.nio.charset.StandardCharsets.UTF_8; 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 java.util.stream.Collectors.toList;
import static org.eclipse.jgit.lib.Constants.OBJ_BLOB; import static org.eclipse.jgit.lib.Constants.OBJ_BLOB;
import static org.junit.Assert.fail; 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.TestChanges;
import com.google.gerrit.testutil.TestTimeUtil; import com.google.gerrit.testutil.TestTimeUtil;
import com.google.gwtorm.server.OrmException; import com.google.gwtorm.server.OrmException;
import com.google.gwtorm.server.OrmRuntimeException;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.google.inject.Provider; import com.google.inject.Provider;
@@ -158,7 +163,7 @@ public class ChangeRebuilderIT extends AbstractDaemonTest {
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
assume().that(NoteDbMode.readWrite()).isFalse(); assume().that(NoteDbMode.readWrite()).isFalse();
TestTimeUtil.resetWithClockStep(1, TimeUnit.SECONDS); TestTimeUtil.resetWithClockStep(1, SECONDS);
setNotesMigration(false, false); setNotesMigration(false, false);
} }
@@ -630,7 +635,8 @@ public class ChangeRebuilderIT extends AbstractDaemonTest {
Optional.of( Optional.of(
NoteDbChangeState.RefState.create( NoteDbChangeState.RefState.create(
NoteDbChangeState.parse(c).getChangeMetaId(), NoteDbChangeState.parse(c).getChangeMetaId(),
ImmutableMap.of(user.getId(), badSha)))); ImmutableMap.of(user.getId(), badSha))),
Optional.empty());
c.setNoteDbState(bogusState.toString()); c.setNoteDbState(bogusState.toString());
db.changes().update(Collections.singleton(c)); 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 { private void assertChangesReadOnly(RestApiException e) throws Exception {
Throwable cause = e.getCause(); Throwable cause = e.getCause();
assertThat(cause).isInstanceOf(UpdateException.class); assertThat(cause).isInstanceOf(UpdateException.class);

View File

@@ -15,19 +15,23 @@
package com.google.gerrit.acceptance.server.notedb; package com.google.gerrit.acceptance.server.notedb;
import static com.google.common.truth.Truth.assertThat; 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.common.truth.TruthJUnit.assume;
import static com.google.gerrit.server.notedb.NoteDbChangeState.PrimaryStorage.REVIEW_DB; import static com.google.gerrit.server.notedb.NoteDbChangeState.PrimaryStorage.REVIEW_DB;
import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toList;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.gerrit.acceptance.AbstractDaemonTest; import com.google.gerrit.acceptance.AbstractDaemonTest;
import com.google.gerrit.acceptance.PushOneCommit; 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.DraftInput;
import com.google.gerrit.extensions.api.changes.ReviewInput; import com.google.gerrit.extensions.api.changes.ReviewInput;
import com.google.gerrit.extensions.client.ChangeStatus; import com.google.gerrit.extensions.client.ChangeStatus;
import com.google.gerrit.extensions.common.ApprovalInfo; import com.google.gerrit.extensions.common.ApprovalInfo;
import com.google.gerrit.extensions.common.ChangeInfo; import com.google.gerrit.extensions.common.ChangeInfo;
import com.google.gerrit.extensions.common.CommentInfo; 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.Account;
import com.google.gerrit.reviewdb.client.Change; import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.server.ReviewDbUtil; 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.ChangeNotes;
import com.google.gerrit.server.notedb.NoteDbChangeState; import com.google.gerrit.server.notedb.NoteDbChangeState;
import com.google.gerrit.testutil.NoteDbMode; import com.google.gerrit.testutil.NoteDbMode;
import com.google.gerrit.testutil.TestTimeUtil;
import com.google.gwtorm.server.OrmRuntimeException;
import com.google.inject.Inject; import com.google.inject.Inject;
import org.eclipse.jgit.lib.Repository; import org.eclipse.jgit.lib.Repository;
import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.sql.Timestamp;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
public class NoteDbPrimaryIT extends AbstractDaemonTest { public class NoteDbPrimaryIT extends AbstractDaemonTest {
@Inject @Inject
@@ -54,6 +64,12 @@ public class NoteDbPrimaryIT extends AbstractDaemonTest {
public void setUp() throws Exception { public void setUp() throws Exception {
assume().that(NoteDbMode.get()).isEqualTo(NoteDbMode.READ_WRITE); assume().that(NoteDbMode.get()).isEqualTo(NoteDbMode.READ_WRITE);
db = ReviewDbUtil.unwrapDb(db); db = ReviewDbUtil.unwrapDb(db);
TestTimeUtil.resetWithClockStep(1, TimeUnit.SECONDS);
}
@After
public void tearDown() {
TestTimeUtil.useSystemTime();
} }
@Test @Test
@@ -162,6 +178,56 @@ public class NoteDbPrimaryIT extends AbstractDaemonTest {
assertThat(getReviewers(id)).isEmpty(); 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 { private void setNoteDbPrimary(Change.Id id) throws Exception {
Change c = db.changes().get(id); Change c = db.changes().get(id);
assertThat(c).named("change " + id).isNotNull(); 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.GerritPersonIdent;
import com.google.gerrit.server.IdentifiedUser; import com.google.gerrit.server.IdentifiedUser;
import com.google.gerrit.server.config.AllUsersName; 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.extensions.events.GitReferenceUpdated;
import com.google.gerrit.server.index.change.ChangeIndexer; import com.google.gerrit.server.index.change.ChangeIndexer;
import com.google.gerrit.server.notedb.ChangeNotes; import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.notedb.ChangeUpdate; 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.NoteDbChangeState.PrimaryStorage;
import com.google.gerrit.server.notedb.NoteDbUpdateManager; import com.google.gerrit.server.notedb.NoteDbUpdateManager;
import com.google.gerrit.server.notedb.NoteDbUpdateManager.MismatchedStateException; 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 com.google.inject.assistedinject.AssistedInject;
import org.eclipse.jgit.lib.BatchRefUpdate; import org.eclipse.jgit.lib.BatchRefUpdate;
import org.eclipse.jgit.lib.Config;
import org.eclipse.jgit.lib.NullProgressMonitor; import org.eclipse.jgit.lib.NullProgressMonitor;
import org.eclipse.jgit.lib.ObjectInserter; import org.eclipse.jgit.lib.ObjectInserter;
import org.eclipse.jgit.lib.ObjectReader; import org.eclipse.jgit.lib.ObjectReader;
@@ -508,6 +511,7 @@ public class BatchUpdate implements AutoCloseable {
private final NotesMigration notesMigration; private final NotesMigration notesMigration;
private final ReviewDb db; private final ReviewDb db;
private final SchemaFactory<ReviewDb> schemaFactory; private final SchemaFactory<ReviewDb> schemaFactory;
private final long skewMs;
private final Project.NameKey project; private final Project.NameKey project;
private final CurrentUser user; private final CurrentUser user;
@@ -533,6 +537,7 @@ public class BatchUpdate implements AutoCloseable {
@AssistedInject @AssistedInject
BatchUpdate( BatchUpdate(
@GerritServerConfig Config cfg,
AllUsersName allUsers, AllUsersName allUsers,
ChangeControl.GenericFactory changeControlFactory, ChangeControl.GenericFactory changeControlFactory,
ChangeIndexer indexer, ChangeIndexer indexer,
@@ -568,6 +573,7 @@ public class BatchUpdate implements AutoCloseable {
this.when = when; this.when = when;
tz = serverIdent.getTimeZone(); tz = serverIdent.getTimeZone();
order = Order.REPO_BEFORE_DB; order = Order.REPO_BEFORE_DB;
skewMs = NoteDbChangeState.getReadOnlySkew(cfg);
} }
@Override @Override
@@ -942,7 +948,10 @@ public class BatchUpdate implements AutoCloseable {
db.changes().beginTransaction(id); db.changes().beginTransaction(id);
try { try {
ChangeContext ctx = newChangeContext(db, repo, rw, id); 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 if (storage == PrimaryStorage.NOTE_DB
&& !notesMigration.readChanges()) { && !notesMigration.readChanges()) {
throw new OrmException( throw new OrmException(
@@ -1037,6 +1046,7 @@ public class BatchUpdate implements AutoCloseable {
logDebug("Failed to get change {} from unwrapped db", id); logDebug("Failed to get change {} from unwrapped db", id);
throw new NoSuchChangeException(id); throw new NoSuchChangeException(id);
} }
NoteDbChangeState.checkNotReadOnly(c, skewMs);
} }
// Pass in preloaded change to controlFor, to avoid: // Pass in preloaded change to controlFor, to avoid:
// - reading from a db that does not belong to this update // - 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.base.Strings;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.primitives.Longs;
import com.google.gerrit.common.Nullable; 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.Account;
import com.google.gerrit.reviewdb.client.Change; import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.server.ReviewDbUtil; import com.google.gerrit.reviewdb.server.ReviewDbUtil;
import com.google.gerrit.server.git.RefCache; 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 org.eclipse.jgit.lib.ObjectId;
import java.io.IOException; import java.io.IOException;
import java.sql.Timestamp;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
import java.util.Optional; import java.util.Optional;
import java.util.concurrent.TimeUnit;
/** /**
* The state of all relevant NoteDb refs across all repos corresponding to a * 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: * Serialized in one of the forms:
* <ul> * <ul>
* <li>[meta-sha],[account1]=[drafts-sha],[account2]=[drafts-sha]... * <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
* <li>N=[read-only-until]
* </ul> * </ul>
* *
* in numeric account ID order, with hex SHA-1s for human readability. * in numeric account ID order, with hex SHA-1s for human readability.
@@ -163,11 +172,13 @@ public class NoteDbChangeState {
return null; return null;
} }
List<String> parts = Splitter.on(',').splitToList(str); 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". // Only valid NOTE_DB state is "N".
String first = parts.get(0);
if (parts.size() == 1 && first.charAt(0) == NOTE_DB.code) { 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 // Otherwise it must be REVIEW_DB, either "R,<RefState>" or just
@@ -179,12 +190,43 @@ public class NoteDbChangeState {
} else { } else {
refState = RefState.parse(id, parts); 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); "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) { public static NoteDbChangeState applyDelta(Change change, Delta delta) {
if (delta == null) { if (delta == null) {
return null; return null;
@@ -230,11 +272,21 @@ public class NoteDbChangeState {
oldState != null oldState != null
? oldState.getPrimaryStorage() ? oldState.getPrimaryStorage()
: REVIEW_DB, : 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()); change.setNoteDbState(state.toString());
return state; 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, public static boolean isChangeUpToDate(@Nullable NoteDbChangeState state,
RefCache changeRepoRefs, Change.Id changeId) throws IOException { RefCache changeRepoRefs, Change.Id changeId) throws IOException {
if (PrimaryStorage.of(state) == NOTE_DB) { if (PrimaryStorage.of(state) == NOTE_DB) {
@@ -259,17 +311,46 @@ public class NoteDbChangeState {
return state.areDraftsUpToDate(draftsRepoRefs, accountId); 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 Change.Id changeId;
private final PrimaryStorage primaryStorage; private final PrimaryStorage primaryStorage;
private final Optional<RefState> refState; private final Optional<RefState> refState;
private final Optional<Timestamp> readOnlyUntil;
public NoteDbChangeState( public NoteDbChangeState(
Change.Id changeId, Change.Id changeId,
PrimaryStorage primaryStorage, PrimaryStorage primaryStorage,
Optional<RefState> refState) { Optional<RefState> refState,
Optional<Timestamp> readOnlyUntil) {
this.changeId = checkNotNull(changeId); this.changeId = checkNotNull(changeId);
this.primaryStorage = checkNotNull(primaryStorage); this.primaryStorage = checkNotNull(primaryStorage);
this.refState = refState; this.refState = checkNotNull(refState);
this.readOnlyUntil = checkNotNull(readOnlyUntil);
switch (primaryStorage) { switch (primaryStorage) {
case REVIEW_DB: case REVIEW_DB:
@@ -334,23 +415,32 @@ public class NoteDbChangeState {
return true; return true;
} }
@VisibleForTesting public boolean isReadOnly(Timestamp now) {
Change.Id getChangeId() { 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; return changeId;
} }
@VisibleForTesting
public ObjectId getChangeMetaId() { public ObjectId getChangeMetaId() {
return refState().changeMetaId(); return refState().changeMetaId();
} }
@VisibleForTesting public ImmutableMap<Account.Id, ObjectId> getDraftIds() {
ImmutableMap<Account.Id, ObjectId> getDraftIds() {
return refState().draftIds(); return refState().draftIds();
} }
@VisibleForTesting public Optional<RefState> getRefState() {
Optional<RefState> getRefState() {
return refState; return refState;
} }
@@ -364,13 +454,37 @@ public class NoteDbChangeState {
public String toString() { public String toString() {
switch (primaryStorage) { switch (primaryStorage) {
case REVIEW_DB: case REVIEW_DB:
// Don't include enum field, just IDs (though parse would accept it). if (!readOnlyUntil.isPresent()) {
return refState().toString(); // 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: case NOTE_DB:
return NOTE_DB_PRIMARY_STATE; if (!readOnlyUntil.isPresent()) {
return NOTE_DB_PRIMARY_STATE;
}
return primaryStorage.code + "=" + readOnlyUntil.get().getTime();
default: default:
throw new IllegalArgumentException( throw new IllegalArgumentException(
"Unsupported PrimaryStorage: " + primaryStorage); "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.common.cache.CacheBuilder;
import com.google.gerrit.extensions.config.FactoryModule; import com.google.gerrit.extensions.config.FactoryModule;
import com.google.gerrit.reviewdb.client.Change; 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.reviewdb.server.ReviewDb;
import com.google.gerrit.server.notedb.NoteDbUpdateManager.Result; import com.google.gerrit.server.notedb.NoteDbUpdateManager.Result;
import com.google.gerrit.server.notedb.rebuild.ChangeRebuilder; import com.google.gerrit.server.notedb.rebuild.ChangeRebuilder;
@@ -68,6 +69,11 @@ public class NoteDbModule extends FactoryModule {
return null; return null;
} }
@Override
public Result rebuildEvenIfReadOnly(ReviewDb db, Id changeId) {
return null;
}
@Override @Override
public Result rebuild(NoteDbUpdateManager manager, public Result rebuild(NoteDbUpdateManager manager,
ChangeBundle bundle) { ChangeBundle bundle) {

View File

@@ -55,10 +55,23 @@ public class TestChangeRebuilderWrapper extends ChangeRebuilder {
@Override @Override
public Result rebuild(ReviewDb db, Change.Id changeId) public Result rebuild(ReviewDb db, Change.Id changeId)
throws IOException, OrmException { 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)) { if (failNextUpdate.getAndSet(false)) {
throw new IOException("Update failed"); 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)) { if (stealNextUpdate.getAndSet(false)) {
throw new IOException("Update stolen"); throw new IOException("Update stolen");
} }

View File

@@ -58,6 +58,9 @@ public abstract class ChangeRebuilder {
public abstract Result rebuild(ReviewDb db, Change.Id changeId) public abstract Result rebuild(ReviewDb db, Change.Id changeId)
throws IOException, OrmException; throws IOException, OrmException;
public abstract Result rebuildEvenIfReadOnly(ReviewDb db, Change.Id changeId)
throws IOException, OrmException;
public abstract Result rebuild(NoteDbUpdateManager manager, public abstract Result rebuild(NoteDbUpdateManager manager,
ChangeBundle bundle) throws IOException, OrmException; 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.GerritPersonIdent;
import com.google.gerrit.server.account.AccountCache; import com.google.gerrit.server.account.AccountCache;
import com.google.gerrit.server.config.AnonymousCowardName; 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.config.GerritServerId;
import com.google.gerrit.server.git.ChainedReceiveCommands; import com.google.gerrit.server.git.ChainedReceiveCommands;
import com.google.gerrit.server.notedb.ChangeBundle; import com.google.gerrit.server.notedb.ChangeBundle;
@@ -73,6 +74,7 @@ import com.google.gwtorm.server.SchemaFactory;
import com.google.inject.Inject; import com.google.inject.Inject;
import org.eclipse.jgit.errors.ConfigInvalidException; import org.eclipse.jgit.errors.ConfigInvalidException;
import org.eclipse.jgit.lib.Config;
import org.eclipse.jgit.lib.ObjectId; import org.eclipse.jgit.lib.ObjectId;
import org.eclipse.jgit.lib.PersonIdent; import org.eclipse.jgit.lib.PersonIdent;
import org.eclipse.jgit.lib.Ref; import org.eclipse.jgit.lib.Ref;
@@ -121,9 +123,11 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
private final ProjectCache projectCache; private final ProjectCache projectCache;
private final String anonymousCowardName; private final String anonymousCowardName;
private final String serverId; private final String serverId;
private final long skewMs;
@Inject @Inject
ChangeRebuilderImpl(SchemaFactory<ReviewDb> schemaFactory, ChangeRebuilderImpl(@GerritServerConfig Config cfg,
SchemaFactory<ReviewDb> schemaFactory,
AccountCache accountCache, AccountCache accountCache,
ChangeBundleReader bundleReader, ChangeBundleReader bundleReader,
ChangeDraftUpdate.Factory draftUpdateFactory, ChangeDraftUpdate.Factory draftUpdateFactory,
@@ -149,11 +153,23 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
this.projectCache = projectCache; this.projectCache = projectCache;
this.anonymousCowardName = anonymousCowardName; this.anonymousCowardName = anonymousCowardName;
this.serverId = serverId; this.serverId = serverId;
this.skewMs = NoteDbChangeState.getReadOnlySkew(cfg);
} }
@Override @Override
public Result rebuild(ReviewDb db, Change.Id changeId) public Result rebuild(ReviewDb db, Change.Id changeId)
throws IOException, OrmException { 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); db = ReviewDbUtil.unwrapDb(db);
// Read change just to get project; this instance is then discarded so we // Read change just to get project; this instance is then discarded so we
// can read a consistent ChangeBundle inside a transaction. // can read a consistent ChangeBundle inside a transaction.
@@ -164,7 +180,7 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
try (NoteDbUpdateManager manager = try (NoteDbUpdateManager manager =
updateManagerFactory.create(change.getProject())) { updateManagerFactory.create(change.getProject())) {
buildUpdates(manager, bundleReader.fromReviewDb(db, changeId)); 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 @Override
public Result execute(ReviewDb db, Change.Id changeId, public Result execute(ReviewDb db, Change.Id changeId,
NoteDbUpdateManager manager) throws OrmException, IOException { 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); db = ReviewDbUtil.unwrapDb(db);
Change change = Change change =
checkNoteDbState(ChangeNotes.readOneReviewDbChange(db, changeId)); checkNoteDbState(ChangeNotes.readOneReviewDbChange(db, changeId));
@@ -210,6 +232,9 @@ public class ChangeRebuilderImpl extends ChangeRebuilder {
db.changes().atomicUpdate(changeId, new AtomicUpdate<Change>() { db.changes().atomicUpdate(changeId, new AtomicUpdate<Change>() {
@Override @Override
public Change update(Change change) { public Change update(Change change) {
if (checkReadOnly) {
NoteDbChangeState.checkNotReadOnly(change, skewMs);
}
String currNoteDbState = change.getNoteDbState(); String currNoteDbState = change.getNoteDbState();
if (Objects.equals(currNoteDbState, newNoteDbState)) { if (Objects.equals(currNoteDbState, newNoteDbState)) {
// Another thread completed the same rebuild we were about to. // 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.base.Preconditions.checkArgument;
import static com.google.common.truth.Truth.assertThat; 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.applyDelta;
import static com.google.gerrit.server.notedb.NoteDbChangeState.parse; import static com.google.gerrit.server.notedb.NoteDbChangeState.parse;
import static com.google.gerrit.server.notedb.NoteDbChangeState.PrimaryStorage.NOTE_DB; 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.Change;
import com.google.gerrit.reviewdb.client.Project; import com.google.gerrit.reviewdb.client.Project;
import com.google.gerrit.server.notedb.NoteDbChangeState.Delta; 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.GerritBaseTests;
import com.google.gerrit.testutil.TestChanges; import com.google.gerrit.testutil.TestChanges;
import com.google.gerrit.testutil.TestTimeUtil;
import org.eclipse.jgit.lib.ObjectId; import org.eclipse.jgit.lib.ObjectId;
import org.junit.After;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.sql.Timestamp;
import java.util.Optional; import java.util.Optional;
import java.util.concurrent.TimeUnit;
/** Unit tests for {@link NoteDbChangeState}. */ /** Unit tests for {@link NoteDbChangeState}. */
public class NoteDbChangeStateTest extends GerritBaseTests { public class NoteDbChangeStateTest extends GerritBaseTests {
@@ -44,6 +51,16 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
ObjectId SHA3 = ObjectId SHA3 =
ObjectId.fromString("badc0feebadc0feebadc0feebadc0feebadc0fee"); ObjectId.fromString("badc0feebadc0feebadc0feebadc0feebadc0fee");
@Before
public void setUp() {
TestTimeUtil.resetWithClockStep(1, TimeUnit.SECONDS);
}
@After
public void tearDown() {
TestTimeUtil.useSystemTime();
}
@Test @Test
public void parseReviewDbWithoutDrafts() { public void parseReviewDbWithoutDrafts() {
NoteDbChangeState state = parse(new Change.Id(1), SHA1.name()); 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.getChangeId()).isEqualTo(new Change.Id(1));
assertThat(state.getChangeMetaId()).isEqualTo(SHA1); assertThat(state.getChangeMetaId()).isEqualTo(SHA1);
assertThat(state.getDraftIds()).isEmpty(); assertThat(state.getDraftIds()).isEmpty();
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
assertThat(state.toString()).isEqualTo(SHA1.name()); assertThat(state.toString()).isEqualTo(SHA1.name());
state = parse(new Change.Id(1), "R," + 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.getChangeId()).isEqualTo(new Change.Id(1));
assertThat(state.getChangeMetaId()).isEqualTo(SHA1); assertThat(state.getChangeMetaId()).isEqualTo(SHA1);
assertThat(state.getDraftIds()).isEmpty(); assertThat(state.getDraftIds()).isEmpty();
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
assertThat(state.toString()).isEqualTo(SHA1.name()); assertThat(state.toString()).isEqualTo(SHA1.name());
} }
@@ -73,6 +92,7 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
assertThat(state.getDraftIds()).containsExactly( assertThat(state.getDraftIds()).containsExactly(
new Account.Id(1001), SHA3, new Account.Id(1001), SHA3,
new Account.Id(2003), SHA2); new Account.Id(2003), SHA2);
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
assertThat(state.toString()).isEqualTo(expected); assertThat(state.toString()).isEqualTo(expected);
state = parse(new Change.Id(1), "R," + str); state = parse(new Change.Id(1), "R," + str);
@@ -82,23 +102,47 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
assertThat(state.getDraftIds()).containsExactly( assertThat(state.getDraftIds()).containsExactly(
new Account.Id(1001), SHA3, new Account.Id(1001), SHA3,
new Account.Id(2003), SHA2); new Account.Id(2003), SHA2);
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
assertThat(state.toString()).isEqualTo(expected); assertThat(state.toString()).isEqualTo(expected);
} }
@Test @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(); Change c = newChange();
assertThat(c.getNoteDbState()).isNull(); assertThat(c.getNoteDbState()).isNull();
applyDelta(c, Delta.create(c.getId(), noMetaId(), noDrafts())); applyDelta(c, Delta.create(c.getId(), noMetaId(), noDrafts()));
assertThat(c.getNoteDbState()).isNull(); assertThat(c.getNoteDbState()).isNull();
applyDelta(c, Delta.create(c.getId(), noMetaId(), applyDelta(
drafts(new Account.Id(1001), zeroId()))); c,
Delta.create(
c.getId(), noMetaId(),
drafts(new Account.Id(1001), zeroId())));
assertThat(c.getNoteDbState()).isNull(); assertThat(c.getNoteDbState()).isNull();
} }
@Test @Test
public void applyDeltaToMetaId() { public void applyDeltaToMetaId() throws Exception {
Change c = newChange(); Change c = newChange();
applyDelta(c, Delta.create(c.getId(), metaId(SHA1), noDrafts())); applyDelta(c, Delta.create(c.getId(), metaId(SHA1), noDrafts()));
assertThat(c.getNoteDbState()).isEqualTo(SHA1.name()); assertThat(c.getNoteDbState()).isEqualTo(SHA1.name());
@@ -116,33 +160,63 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
} }
@Test @Test
public void applyDeltaToDrafts() { public void applyDeltaToDrafts() throws Exception {
Change c = newChange(); Change c = newChange();
applyDelta(c, Delta.create(c.getId(), metaId(SHA1), applyDelta(
drafts(new Account.Id(1001), SHA2))); c,
Delta.create(
c.getId(), metaId(SHA1),
drafts(new Account.Id(1001), SHA2)));
assertThat(c.getNoteDbState()).isEqualTo( assertThat(c.getNoteDbState()).isEqualTo(
SHA1.name() + ",1001=" + SHA2.name()); SHA1.name() + ",1001=" + SHA2.name());
applyDelta(c, Delta.create(c.getId(), noMetaId(), applyDelta(
drafts(new Account.Id(2003), SHA3))); c,
Delta.create(
c.getId(), noMetaId(),
drafts(new Account.Id(2003), SHA3)));
assertThat(c.getNoteDbState()).isEqualTo( assertThat(c.getNoteDbState()).isEqualTo(
SHA1.name() + ",1001=" + SHA2.name() + ",2003=" + SHA3.name()); SHA1.name() + ",1001=" + SHA2.name() + ",2003=" + SHA3.name());
applyDelta(c, Delta.create(c.getId(), noMetaId(), applyDelta(
drafts(new Account.Id(2003), zeroId()))); c,
Delta.create(
c.getId(), noMetaId(),
drafts(new Account.Id(2003), zeroId())));
assertThat(c.getNoteDbState()).isEqualTo( assertThat(c.getNoteDbState()).isEqualTo(
SHA1.name() + ",1001=" + SHA2.name()); 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( assertThat(c.getNoteDbState()).isEqualTo(
SHA3.name() + ",1001=" + SHA2.name()); 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 @Test
public void parseNoteDbPrimary() { public void parseNoteDbPrimary() {
NoteDbChangeState state = parse(new Change.Id(1), "N"); NoteDbChangeState state = parse(new Change.Id(1), "N");
assertThat(state.getPrimaryStorage()).isEqualTo(NOTE_DB); assertThat(state.getPrimaryStorage()).isEqualTo(NOTE_DB);
assertThat(state.getRefState().isPresent()).isFalse(); assertThat(state.getRefState().isPresent()).isFalse();
assertThat(state.getReadOnlyUntil().isPresent()).isFalse();
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
@@ -151,7 +225,7 @@ public class NoteDbChangeStateTest extends GerritBaseTests {
} }
@Test @Test
public void applyDeltaToNoteDbPrimaryIsNoOp() { public void applyDeltaToNoteDbPrimaryIsNoOp() throws Exception {
Change c = newChange(); Change c = newChange();
c.setNoteDbState("N"); c.setNoteDbState("N");
applyDelta(c, Delta.create(c.getId(), metaId(SHA1), applyDelta(c, Delta.create(c.getId(), metaId(SHA1),