Add an action to replay commit logs to SQL (#887)

* Add an action to replay commit logs to SQL

- We import from the commit-log GCS files so that we are sure that we
have a consistent snapshot.
- We use the object weighting (moved to ObjectWeights) to verify that we
are replaying objects in the correct order.
- We add a config setting (default false) of whether or not to replay
the logs.
- The action is triggered after the export if the aforementioned config
setting is on.

* Responses to CR

- Remove triggering of replay from the export action and remove the test
changes
- Add a method to load commit log diffs by transaction
- Replay one Datastore transaction at a time, per SQL transaction
- Minor logging / comment changes
- Change ObjectWeights to EntityWritePriorities and flesh out javadoc

* More CR responses

- Use one transaction per GCS diff file
- Fix up comments minutiae

* Add a class-level javadoc

* Add a log message and some periods

* bit of formatting

* Merge remote-tracking branch 'origin/master' into replayAction

* Handle toSqlEntity rather than toSqlEntities
This commit is contained in:
gbrodman 2020-12-03 16:50:41 -05:00 committed by GitHub
parent 4f0189c162
commit a8effe8a1e
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
13 changed files with 897 additions and 112 deletions

View file

@ -0,0 +1,458 @@
// Copyright 2020 The Nomulus Authors. All Rights Reserved.
//
// 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 google.registry.backup;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.truth.Truth.assertThat;
import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService;
import static google.registry.backup.RestoreCommitLogsActionTest.GCS_BUCKET;
import static google.registry.backup.RestoreCommitLogsActionTest.createCheckpoint;
import static google.registry.backup.RestoreCommitLogsActionTest.saveDiffFile;
import static google.registry.backup.RestoreCommitLogsActionTest.saveDiffFileNotToRestore;
import static google.registry.model.common.EntityGroupRoot.getCrossTldKey;
import static google.registry.model.ofy.CommitLogBucket.getBucketKey;
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import static google.registry.testing.DatabaseHelper.createTld;
import static google.registry.testing.DatabaseHelper.newDomainBase;
import static google.registry.testing.DatabaseHelper.persistActiveContact;
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
import static javax.servlet.http.HttpServletResponse.SC_OK;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import com.google.appengine.tools.cloudstorage.GcsService;
import com.google.appengine.tools.cloudstorage.GcsServiceFactory;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.truth.Truth8;
import com.googlecode.objectify.Key;
import google.registry.config.RegistryConfig;
import google.registry.model.common.Cursor;
import google.registry.model.common.Cursor.CursorType;
import google.registry.model.contact.ContactResource;
import google.registry.model.domain.DomainBase;
import google.registry.model.domain.GracePeriod;
import google.registry.model.domain.secdns.DelegationSignerData;
import google.registry.model.ofy.CommitLogBucket;
import google.registry.model.ofy.CommitLogManifest;
import google.registry.model.ofy.CommitLogMutation;
import google.registry.model.registrar.RegistrarContact;
import google.registry.model.registry.label.ReservedList;
import google.registry.model.server.Lock;
import google.registry.model.tmch.ClaimsListShard;
import google.registry.model.translators.VKeyTranslatorFactory;
import google.registry.persistence.VKey;
import google.registry.persistence.transaction.JpaTransactionManager;
import google.registry.persistence.transaction.TransactionManagerFactory;
import google.registry.schema.replay.SqlReplayCheckpoint;
import google.registry.testing.AppEngineExtension;
import google.registry.testing.FakeClock;
import google.registry.testing.FakeResponse;
import google.registry.testing.TestObject;
import google.registry.util.RequestStatusChecker;
import java.io.IOException;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.api.extension.RegisterExtension;
import org.mockito.ArgumentCaptor;
import org.mockito.InOrder;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
/** Tests for {@link ReplayCommitLogsToSqlAction}. */
@ExtendWith(MockitoExtension.class)
public class ReplayCommitLogsToSqlActionTest {
private final FakeClock fakeClock = new FakeClock(DateTime.parse("2000-01-01TZ"));
@RegisterExtension
public final AppEngineExtension appEngine =
AppEngineExtension.builder()
.withDatastoreAndCloudSql()
.withClock(fakeClock)
.withOfyTestEntities(TestObject.class)
.withJpaUnitTestEntities(
RegistrarContact.class,
TestObject.class,
SqlReplayCheckpoint.class,
ContactResource.class,
DomainBase.class,
GracePeriod.class,
DelegationSignerData.class)
.build();
/** Local GCS service. */
private final GcsService gcsService = GcsServiceFactory.createGcsService();
private final ReplayCommitLogsToSqlAction action = new ReplayCommitLogsToSqlAction();
private final FakeResponse response = new FakeResponse();
@Mock private RequestStatusChecker requestStatusChecker;
@BeforeAll
static void beforeAll() {
VKeyTranslatorFactory.addTestEntityClass(TestObject.class);
}
@BeforeEach
void beforeEach() {
action.gcsService = gcsService;
action.response = response;
action.requestStatusChecker = requestStatusChecker;
action.diffLister = new GcsDiffFileLister();
action.diffLister.gcsService = gcsService;
action.diffLister.gcsBucket = GCS_BUCKET;
action.diffLister.executor = newDirectExecutorService();
RegistryConfig.overrideCloudSqlReplayCommitLogs(true);
}
@Test
void testReplay_multipleDiffFiles() throws Exception {
jpaTm()
.transact(
() -> {
jpaTm().insertWithoutBackup(TestObject.create("previous to keep"));
jpaTm().insertWithoutBackup(TestObject.create("previous to delete"));
});
DateTime now = fakeClock.nowUtc();
// Create 3 transactions, across two diff files.
// Before: {"previous to keep", "previous to delete"}
// 1a: Add {"a", "b"}, Delete {"previous to delete"}
// 1b: Add {"c", "d"}, Delete {"a"}
// 2: Add {"e", "f"}, Delete {"c"}
// After: {"previous to keep", "b", "d", "e", "f"}
Key<CommitLogManifest> manifest1aKey =
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(3));
Key<CommitLogManifest> manifest1bKey =
CommitLogManifest.createKey(getBucketKey(2), now.minusMinutes(2));
Key<CommitLogManifest> manifest2Key =
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
saveDiffFileNotToRestore(gcsService, now.minusMinutes(2));
saveDiffFile(
gcsService,
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(
getBucketKey(1),
now.minusMinutes(3),
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))),
CommitLogMutation.create(manifest1aKey, TestObject.create("a")),
CommitLogMutation.create(manifest1aKey, TestObject.create("b")),
CommitLogManifest.create(
getBucketKey(2),
now.minusMinutes(2),
ImmutableSet.of(Key.create(TestObject.create("a")))),
CommitLogMutation.create(manifest1bKey, TestObject.create("c")),
CommitLogMutation.create(manifest1bKey, TestObject.create("d")));
saveDiffFile(
gcsService,
createCheckpoint(now),
CommitLogManifest.create(
getBucketKey(1),
now.minusMinutes(1),
ImmutableSet.of(Key.create(TestObject.create("c")))),
CommitLogMutation.create(manifest2Key, TestObject.create("e")),
CommitLogMutation.create(manifest2Key, TestObject.create("f")));
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
fakeClock.advanceOneMilli();
runAndAssertSuccess(now);
assertExpectedIds("previous to keep", "b", "d", "e", "f");
}
@Test
void testReplay_noManifests() throws Exception {
DateTime now = fakeClock.nowUtc();
jpaTm().transact(() -> jpaTm().insertWithoutBackup(TestObject.create("previous to keep")));
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1));
saveDiffFile(gcsService, createCheckpoint(now.minusMillis(2)));
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMillis(1)));
runAndAssertSuccess(now.minusMillis(1));
assertExpectedIds("previous to keep");
}
@Test
void testReplay_manifestWithNoDeletions() throws Exception {
DateTime now = fakeClock.nowUtc();
jpaTm().transact(() -> jpaTm().insertWithoutBackup(TestObject.create("previous to keep")));
Key<CommitLogBucket> bucketKey = getBucketKey(1);
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(bucketKey, now);
saveDiffFileNotToRestore(gcsService, now.minusMinutes(2));
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
saveDiffFile(
gcsService,
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(bucketKey, now, null),
CommitLogMutation.create(manifestKey, TestObject.create("a")),
CommitLogMutation.create(manifestKey, TestObject.create("b")));
runAndAssertSuccess(now.minusMinutes(1));
assertExpectedIds("previous to keep", "a", "b");
}
@Test
void testReplay_manifestWithNoMutations() throws Exception {
DateTime now = fakeClock.nowUtc();
jpaTm()
.transact(
() -> {
jpaTm().insertWithoutBackup(TestObject.create("previous to keep"));
jpaTm().insertWithoutBackup(TestObject.create("previous to delete"));
});
saveDiffFileNotToRestore(gcsService, now.minusMinutes(2));
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
saveDiffFile(
gcsService,
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(
getBucketKey(1),
now,
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))));
runAndAssertSuccess(now.minusMinutes(1));
assertExpectedIds("previous to keep");
}
@Test
void wtestReplay_mutateExistingEntity() throws Exception {
DateTime now = fakeClock.nowUtc();
jpaTm().transact(() -> jpaTm().put(TestObject.create("existing", "a")));
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(getBucketKey(1), now);
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1).minusMillis(1));
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1)));
saveDiffFile(
gcsService,
createCheckpoint(now.minusMillis(1)),
CommitLogManifest.create(getBucketKey(1), now, null),
CommitLogMutation.create(manifestKey, TestObject.create("existing", "b")));
action.run();
TestObject fromDatabase =
jpaTm().transact(() -> jpaTm().load(VKey.createSql(TestObject.class, "existing")));
assertThat(fromDatabase.getField()).isEqualTo("b");
}
// This should be harmless
@Test
void testReplay_deleteMissingEntity() throws Exception {
DateTime now = fakeClock.nowUtc();
jpaTm().transact(() -> jpaTm().put(TestObject.create("previous to keep", "a")));
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1).minusMillis(1));
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1)));
saveDiffFile(
gcsService,
createCheckpoint(now.minusMillis(1)),
CommitLogManifest.create(
getBucketKey(1),
now,
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))));
action.run();
assertExpectedIds("previous to keep");
}
@Test
@SuppressWarnings({"unchecked", "rawtypes"})
void testReplay_properlyWeighted() throws Exception {
DateTime now = fakeClock.nowUtc();
Key<CommitLogManifest> manifestKey =
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
// Create (but don't save to SQL) a domain + contact
createTld("tld");
DomainBase domain = newDomainBase("example.tld");
CommitLogMutation domainMutation =
tm().transact(() -> CommitLogMutation.create(manifestKey, domain));
ContactResource contact = tm().transact(() -> tm().load(domain.getRegistrant()));
CommitLogMutation contactMutation =
tm().transact(() -> CommitLogMutation.create(manifestKey, contact));
// Create and save to SQL a registrar contact that we will delete
RegistrarContact toDelete = AppEngineExtension.makeRegistrarContact1();
jpaTm().transact(() -> jpaTm().put(toDelete));
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
// spy the txn manager so we can see what order things were inserted/removed
JpaTransactionManager spy = spy(jpaTm());
TransactionManagerFactory.setJpaTm(() -> spy);
// Save in the commit logs the domain and contact (in that order) and the token deletion
saveDiffFile(
gcsService,
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(
getBucketKey(1), now.minusMinutes(1), ImmutableSet.of(Key.create(toDelete))),
domainMutation,
contactMutation);
runAndAssertSuccess(now.minusMinutes(1));
// Verify two things:
// 1. that the contact insert occurred before the domain insert (necessary for FK ordering)
// even though the domain came first in the file
// 2. that the allocation token delete occurred after the insertions
InOrder inOrder = Mockito.inOrder(spy);
inOrder.verify(spy).put(any(ContactResource.class));
inOrder.verify(spy).put(any(DomainBase.class));
inOrder.verify(spy).delete(toDelete.createVKey());
inOrder.verify(spy).put(any(SqlReplayCheckpoint.class));
}
@Test
@SuppressWarnings({"unchecked", "rawtypes"})
void testReplay_properlyWeighted_doesNotApplyCrossTransactions() throws Exception {
DateTime now = fakeClock.nowUtc();
Key<CommitLogManifest> manifestKey =
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
// Create and save the standard contact
ContactResource contact = persistActiveContact("contact1234");
jpaTm().transact(() -> jpaTm().put(contact));
// Simulate a Datastore transaction with a new version of the contact
ContactResource contactWithEdit =
contact.asBuilder().setEmailAddress("replay@example.tld").build();
CommitLogMutation contactMutation =
tm().transact(() -> CommitLogMutation.create(manifestKey, contactWithEdit));
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
// spy the txn manager so we can see what order things were inserted
JpaTransactionManager spy = spy(jpaTm());
TransactionManagerFactory.setJpaTm(() -> spy);
// Save two commits -- the deletion, then the new version of the contact
saveDiffFile(
gcsService,
createCheckpoint(now.minusMinutes(1).plusMillis(1)),
CommitLogManifest.create(
getBucketKey(1), now.minusMinutes(1), ImmutableSet.of(Key.create(contact))),
CommitLogManifest.create(
getBucketKey(1), now.minusMinutes(1).plusMillis(1), ImmutableSet.of()),
contactMutation);
runAndAssertSuccess(now.minusMinutes(1).plusMillis(1));
// Verify that the delete occurred first (because it was in the first transaction) even though
// deletes have higher weight
ArgumentCaptor<Object> putCaptor = ArgumentCaptor.forClass(Object.class);
InOrder inOrder = Mockito.inOrder(spy);
inOrder.verify(spy).delete(contact.createVKey());
inOrder.verify(spy).put(putCaptor.capture());
assertThat(putCaptor.getValue().getClass()).isEqualTo(ContactResource.class);
assertThat(jpaTm().transact(() -> jpaTm().load(contact.createVKey()).getEmailAddress()))
.isEqualTo("replay@example.tld");
}
@Test
void testSuccess_nonReplicatedEntity_isNotReplayed() {
DateTime now = fakeClock.nowUtc();
// spy the txn manager so we can verify it's never called
JpaTransactionManager spy = spy(jpaTm());
TransactionManagerFactory.setJpaTm(() -> spy);
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
Key<CommitLogManifest> manifestKey =
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
// Have a commit log with a couple objects that shouldn't be replayed
ReservedList reservedList =
new ReservedList.Builder().setReservedListMap(ImmutableMap.of()).setName("name").build();
Cursor cursor = Cursor.createGlobal(CursorType.RECURRING_BILLING, now.minusHours(1));
tm().transact(
() -> {
try {
saveDiffFile(
gcsService,
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(
getBucketKey(1), now.minusMinutes(1), ImmutableSet.of()),
// Reserved list is dually-written non-replicated
CommitLogMutation.create(manifestKey, reservedList),
// Cursors aren't replayed to SQL at all
CommitLogMutation.create(manifestKey, cursor));
} catch (IOException e) {
throw new RuntimeException(e);
}
});
runAndAssertSuccess(now.minusMinutes(1));
// jpaTm()::put should only have been called with the checkpoint
verify(spy, times(2)).put(any(SqlReplayCheckpoint.class));
verify(spy, times(2)).put(any());
}
@Test
void testSuccess_nonReplicatedEntity_isNotDeleted() throws Exception {
DateTime now = fakeClock.nowUtc();
// spy the txn manager so we can verify it's never called
JpaTransactionManager spy = spy(jpaTm());
TransactionManagerFactory.setJpaTm(() -> spy);
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
// Save a couple deletes that aren't propagated to SQL (the objects deleted are irrelevant)
Key<ClaimsListShard> claimsListKey = Key.create(ClaimsListShard.class, 1L);
saveDiffFile(
gcsService,
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(
getBucketKey(1),
now.minusMinutes(1),
// one object only exists in Datastore, one is dually-written (so isn't replicated)
ImmutableSet.of(getCrossTldKey(), claimsListKey)));
runAndAssertSuccess(now.minusMinutes(1));
verify(spy, times(0)).delete(any(VKey.class));
}
@Test
void testFailure_notEnabled() {
RegistryConfig.overrideCloudSqlReplayCommitLogs(false);
action.run();
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
assertThat(response.getPayload())
.isEqualTo("ReplayCommitLogsToSqlAction was called but disabled in the config.");
}
@Test
void testFailure_cannotAcquireLock() {
Truth8.assertThat(
Lock.acquire(
ReplayCommitLogsToSqlAction.class.getSimpleName(),
null,
Duration.standardHours(1),
requestStatusChecker,
false))
.isPresent();
fakeClock.advanceOneMilli();
action.run();
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
assertThat(response.getPayload())
.isEqualTo("Can't acquire SQL commit log replay lock, aborting.");
}
private void runAndAssertSuccess(DateTime expectedCheckpointTime) {
action.run();
assertThat(response.getStatus()).isEqualTo(SC_OK);
assertThat(jpaTm().transact(SqlReplayCheckpoint::get)).isEqualTo(expectedCheckpointTime);
}
private void assertExpectedIds(String... expectedIds) {
ImmutableList<String> actualIds =
jpaTm()
.transact(
() ->
jpaTm().loadAll(TestObject.class).stream()
.map(TestObject::getId)
.collect(toImmutableList()));
assertThat(actualIds).containsExactlyElementsIn(expectedIds);
}
}

View file

@ -61,7 +61,7 @@ import org.junit.jupiter.api.extension.RegisterExtension;
/** Unit tests for {@link RestoreCommitLogsAction}. */
public class RestoreCommitLogsActionTest {
private static final String GCS_BUCKET = "gcs bucket";
static final String GCS_BUCKET = "gcs bucket";
private final DateTime now = DateTime.now(UTC);
private final RestoreCommitLogsAction action = new RestoreCommitLogsAction();
@ -89,9 +89,10 @@ public class RestoreCommitLogsActionTest {
@Test
void testRestore_multipleDiffFiles() throws Exception {
ofy().saveWithoutBackup().entities(
TestObject.create("previous to keep"),
TestObject.create("previous to delete")).now();
ofy()
.saveWithoutBackup()
.entities(TestObject.create("previous to keep"), TestObject.create("previous to delete"))
.now();
// Create 3 transactions, across two diff files.
// Before: {"previous to keep", "previous to delete"}
// 1a: Add {"a", "b"}, Delete {"previous to delete"}
@ -104,29 +105,33 @@ public class RestoreCommitLogsActionTest {
CommitLogManifest.createKey(getBucketKey(2), now.minusMinutes(2));
Key<CommitLogManifest> manifest2Key =
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
saveDiffFileNotToRestore(now.minusMinutes(2));
Iterable<ImmutableObject> file1CommitLogs = saveDiffFile(
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(
getBucketKey(1),
now.minusMinutes(3),
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))),
CommitLogMutation.create(manifest1aKey, TestObject.create("a")),
CommitLogMutation.create(manifest1aKey, TestObject.create("b")),
CommitLogManifest.create(
getBucketKey(2),
now.minusMinutes(2),
ImmutableSet.of(Key.create(TestObject.create("a")))),
CommitLogMutation.create(manifest1bKey, TestObject.create("c")),
CommitLogMutation.create(manifest1bKey, TestObject.create("d")));
Iterable<ImmutableObject> file2CommitLogs = saveDiffFile(
createCheckpoint(now),
CommitLogManifest.create(
getBucketKey(1),
now.minusMinutes(1),
ImmutableSet.of(Key.create(TestObject.create("c")))),
CommitLogMutation.create(manifest2Key, TestObject.create("e")),
CommitLogMutation.create(manifest2Key, TestObject.create("f")));
saveDiffFileNotToRestore(gcsService, now.minusMinutes(2));
Iterable<ImmutableObject> file1CommitLogs =
saveDiffFile(
gcsService,
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(
getBucketKey(1),
now.minusMinutes(3),
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))),
CommitLogMutation.create(manifest1aKey, TestObject.create("a")),
CommitLogMutation.create(manifest1aKey, TestObject.create("b")),
CommitLogManifest.create(
getBucketKey(2),
now.minusMinutes(2),
ImmutableSet.of(Key.create(TestObject.create("a")))),
CommitLogMutation.create(manifest1bKey, TestObject.create("c")),
CommitLogMutation.create(manifest1bKey, TestObject.create("d")));
Iterable<ImmutableObject> file2CommitLogs =
saveDiffFile(
gcsService,
createCheckpoint(now),
CommitLogManifest.create(
getBucketKey(1),
now.minusMinutes(1),
ImmutableSet.of(Key.create(TestObject.create("c")))),
CommitLogMutation.create(manifest2Key, TestObject.create("e")),
CommitLogMutation.create(manifest2Key, TestObject.create("f")));
action.fromTime = now.minusMinutes(1).minusMillis(1);
action.run();
ofy().clearSessionCache();
@ -139,10 +144,9 @@ public class RestoreCommitLogsActionTest {
@Test
void testRestore_noManifests() throws Exception {
ofy().saveWithoutBackup().entity(
TestObject.create("previous to keep")).now();
saveDiffFileNotToRestore(now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs = saveDiffFile(createCheckpoint(now));
ofy().saveWithoutBackup().entity(TestObject.create("previous to keep")).now();
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs = saveDiffFile(gcsService, createCheckpoint(now));
action.run();
ofy().clearSessionCache();
assertExpectedIds("previous to keep");
@ -156,32 +160,37 @@ public class RestoreCommitLogsActionTest {
ofy().saveWithoutBackup().entity(TestObject.create("previous to keep")).now();
Key<CommitLogBucket> bucketKey = getBucketKey(1);
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(bucketKey, now);
saveDiffFileNotToRestore(now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs = saveDiffFile(
createCheckpoint(now),
CommitLogManifest.create(bucketKey, now, null),
CommitLogMutation.create(manifestKey, TestObject.create("a")),
CommitLogMutation.create(manifestKey, TestObject.create("b")));
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs =
saveDiffFile(
gcsService,
createCheckpoint(now),
CommitLogManifest.create(bucketKey, now, null),
CommitLogMutation.create(manifestKey, TestObject.create("a")),
CommitLogMutation.create(manifestKey, TestObject.create("b")));
action.run();
ofy().clearSessionCache();
assertExpectedIds("previous to keep", "a", "b");
assertInDatastore(commitLogs);
assertInDatastore(CommitLogCheckpointRoot.create(now));
assertCommitLogBuckets(ImmutableMap.of(1, now));
}
}
@Test
void testRestore_manifestWithNoMutations() throws Exception {
ofy().saveWithoutBackup().entities(
TestObject.create("previous to keep"),
TestObject.create("previous to delete")).now();
saveDiffFileNotToRestore(now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs = saveDiffFile(
createCheckpoint(now),
CommitLogManifest.create(
getBucketKey(1),
now,
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))));
ofy()
.saveWithoutBackup()
.entities(TestObject.create("previous to keep"), TestObject.create("previous to delete"))
.now();
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs =
saveDiffFile(
gcsService,
createCheckpoint(now),
CommitLogManifest.create(
getBucketKey(1),
now,
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))));
action.run();
ofy().clearSessionCache();
assertExpectedIds("previous to keep");
@ -193,12 +202,13 @@ public class RestoreCommitLogsActionTest {
// This is a pathological case that shouldn't be possible, but we should be robust to it.
@Test
void testRestore_manifestWithNoMutationsOrDeletions() throws Exception {
ofy().saveWithoutBackup().entities(
TestObject.create("previous to keep")).now();
saveDiffFileNotToRestore(now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs = saveDiffFile(
createCheckpoint(now),
CommitLogManifest.create(getBucketKey(1), now, null));
ofy().saveWithoutBackup().entities(TestObject.create("previous to keep")).now();
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs =
saveDiffFile(
gcsService,
createCheckpoint(now),
CommitLogManifest.create(getBucketKey(1), now, null));
action.run();
ofy().clearSessionCache();
assertExpectedIds("previous to keep");
@ -211,11 +221,13 @@ public class RestoreCommitLogsActionTest {
void testRestore_mutateExistingEntity() throws Exception {
ofy().saveWithoutBackup().entity(TestObject.create("existing", "a")).now();
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(getBucketKey(1), now);
saveDiffFileNotToRestore(now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs = saveDiffFile(
createCheckpoint(now),
CommitLogManifest.create(getBucketKey(1), now, null),
CommitLogMutation.create(manifestKey, TestObject.create("existing", "b")));
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs =
saveDiffFile(
gcsService,
createCheckpoint(now),
CommitLogManifest.create(getBucketKey(1), now, null),
CommitLogMutation.create(manifestKey, TestObject.create("existing", "b")));
action.run();
ofy().clearSessionCache();
assertThat(ofy().load().entity(TestObject.create("existing")).now().getField()).isEqualTo("b");
@ -228,13 +240,15 @@ public class RestoreCommitLogsActionTest {
@Test
void testRestore_deleteMissingEntity() throws Exception {
ofy().saveWithoutBackup().entity(TestObject.create("previous to keep", "a")).now();
saveDiffFileNotToRestore(now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs = saveDiffFile(
createCheckpoint(now),
CommitLogManifest.create(
getBucketKey(1),
now,
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))));
saveDiffFileNotToRestore(gcsService, now.minusMinutes(1));
Iterable<ImmutableObject> commitLogs =
saveDiffFile(
gcsService,
createCheckpoint(now),
CommitLogManifest.create(
getBucketKey(1),
now,
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))));
action.run();
ofy().clearSessionCache();
assertExpectedIds("previous to keep");
@ -243,12 +257,13 @@ public class RestoreCommitLogsActionTest {
assertInDatastore(CommitLogCheckpointRoot.create(now));
}
private CommitLogCheckpoint createCheckpoint(DateTime now) {
static CommitLogCheckpoint createCheckpoint(DateTime now) {
return CommitLogCheckpoint.create(now, toMap(getBucketIds(), x -> now));
}
private Iterable<ImmutableObject> saveDiffFile(
CommitLogCheckpoint checkpoint, ImmutableObject... entities) throws IOException {
static Iterable<ImmutableObject> saveDiffFile(
GcsService gcsService, CommitLogCheckpoint checkpoint, ImmutableObject... entities)
throws IOException {
DateTime now = checkpoint.getCheckpointTime();
List<ImmutableObject> allEntities = Lists.asList(checkpoint, entities);
ByteArrayOutputStream output = new ByteArrayOutputStream();
@ -264,8 +279,9 @@ public class RestoreCommitLogsActionTest {
return allEntities;
}
private void saveDiffFileNotToRestore(DateTime now) throws Exception {
static void saveDiffFileNotToRestore(GcsService gcsService, DateTime now) throws Exception {
saveDiffFile(
gcsService,
createCheckpoint(now),
CommitLogManifest.create(getBucketKey(1), now, null),
CommitLogMutation.create(

View file

@ -18,19 +18,20 @@ import static com.google.common.truth.Truth.assertThat;
import com.google.common.collect.ImmutableMap;
import com.googlecode.objectify.Key;
import google.registry.model.ofy.TransactionInfo.Delete;
import google.registry.model.registrar.Registrar;
import google.registry.model.reporting.HistoryEntry;
import google.registry.testing.AppEngineExtension;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.RegisterExtension;
class TransactionInfoTest {
class EntityWritePrioritiesTest {
@RegisterExtension
AppEngineExtension appEngine = new AppEngineExtension.Builder().withDatastore().build();
@Test
void testGetWeight() {
void testGetPriority() {
// just verify that the lowest is what we expect for both save and delete and verify that the
// Registrar class is zero.
ImmutableMap<Key<?>, Object> actions =
@ -39,10 +40,12 @@ class TransactionInfoTest {
Key.create(HistoryEntry.class, 200), "fake history entry",
Key.create(Registrar.class, 300), "fake registrar");
ImmutableMap<Long, Integer> expectedValues =
ImmutableMap.of(100L, TransactionInfo.DELETE_RANGE + 1, 200L, -1, 300L, 0);
ImmutableMap.of(100L, EntityWritePriorities.DELETE_RANGE + 10, 200L, -10, 300L, 0);
for (ImmutableMap.Entry<Key<?>, Object> entry : actions.entrySet()) {
assertThat(TransactionInfo.getWeight(entry))
assertThat(
EntityWritePriorities.getEntityPriority(
entry.getKey().getKind(), Delete.SENTINEL.equals(entry.getValue())))
.isEqualTo(expectedValues.get(entry.getKey().getId()));
}
}

View file

@ -24,16 +24,19 @@ import google.registry.model.ImmutableObject;
import google.registry.model.annotations.VirtualEntity;
import google.registry.model.common.EntityGroupRoot;
import google.registry.persistence.VKey;
import google.registry.schema.replay.DatastoreAndSqlEntity;
import google.registry.schema.replay.EntityTest.EntityForTesting;
import javax.persistence.Transient;
/** A test model object that can be persisted in any entity group. */
@Entity
@javax.persistence.Entity
@EntityForTesting
public class TestObject extends ImmutableObject {
public class TestObject extends ImmutableObject implements DatastoreAndSqlEntity {
@Parent Key<EntityGroupRoot> parent;
@Parent @Transient Key<EntityGroupRoot> parent;
@Id String id;
@Id @javax.persistence.Id String id;
String field;