Load Datastore snapshot from backup files (#660)

* Load Datastore snapshot from backup files

Defined a composite transform that loads from a Datastore export and
concurrent CommitLog files, identify entities that still exist at the
end of the time window, and resolve their latest states in the window.
This commit is contained in:
Weimin Yu 2020-07-01 09:58:42 -04:00 committed by GitHub
parent d065ff63fc
commit 1961a5759d
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
7 changed files with 385 additions and 40 deletions

View file

@ -15,8 +15,6 @@
package google.registry.beam.initsql;
import static com.google.common.truth.Truth.assertThat;
import static org.hamcrest.Matchers.notNullValue;
import static org.junit.Assume.assumeThat;
import google.registry.persistence.NomulusPostgreSql;
import google.registry.persistence.transaction.JpaTransactionManager;
@ -25,28 +23,28 @@ import java.io.IOException;
import java.io.PrintStream;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.condition.EnabledIfSystemProperty;
import org.junit.jupiter.api.io.TempDir;
import org.testcontainers.containers.PostgreSQLContainer;
import org.testcontainers.junit.jupiter.Container;
import org.testcontainers.junit.jupiter.Testcontainers;
/** Unit tests for {@link BeamJpaModule}. */
@RunWith(JUnit4.class) // TODO(weiminyu): upgrade to JUnit 5.
@Testcontainers
public class BeamJpaModuleTest {
@Rule
@Container
public PostgreSQLContainer database = new PostgreSQLContainer(NomulusPostgreSql.getDockerTag());
@Rule public TemporaryFolder temporaryFolder = new TemporaryFolder();
@TempDir File tempFolder;
private File credentialFile;
@Before
@BeforeEach
public void beforeEach() throws IOException {
credentialFile = temporaryFolder.newFile();
credentialFile = new File(tempFolder, "credential");
new PrintStream(credentialFile)
.printf("%s %s %s", database.getJdbcUrl(), database.getUsername(), database.getPassword())
.close();
@ -76,10 +74,9 @@ public class BeamJpaModuleTest {
* information.
*/
@Test
@EnabledIfSystemProperty(named = "test.gcp_integration.env", matches = "\\S+")
public void getJpaTransactionManager_cloudSql_authRequired() {
String environmentName = System.getProperty("test.gcp_integration.env");
assumeThat(environmentName, notNullValue());
FileSystems.setDefaultPipelineOptions(PipelineOptionsFactory.create());
JpaTransactionManager jpa =
DaggerBeamJpaModule_JpaTransactionManagerComponent.builder()

View file

@ -19,6 +19,7 @@ import static google.registry.testing.DatastoreHelper.newDomainBase;
import static google.registry.testing.DatastoreHelper.newRegistry;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import google.registry.backup.VersionedEntity;
import google.registry.model.contact.ContactResource;
import google.registry.model.domain.DomainBase;
@ -27,6 +28,7 @@ import google.registry.model.registry.Registry;
import google.registry.testing.FakeClock;
import google.registry.testing.InjectRule;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
@ -152,28 +154,45 @@ public class CommitLogTransformsTest implements Serializable {
@Test
@Category(NeedsRunner.class)
public void filterCommitLogsByTime() {
public void filterCommitLogsByTime() throws IOException {
ImmutableList<String> commitLogFilenames =
ImmutableList.of(
"/commit_diff_until_2000-01-01T00:00:00.000Z",
"/commit_diff_until_2000-01-01T00:00:00.001Z",
"/commit_diff_until_2000-01-01T00:00:00.002Z",
"/commit_diff_until_2000-01-01T00:00:00.003Z",
"/commit_diff_until_2000-01-01T00:00:00.004Z");
"commit_diff_until_2000-01-01T00:00:00.000Z",
"commit_diff_until_2000-01-01T00:00:00.001Z",
"commit_diff_until_2000-01-01T00:00:00.002Z",
"commit_diff_until_2000-01-01T00:00:00.003Z",
"commit_diff_until_2000-01-01T00:00:00.004Z");
File commitLogDir = temporaryFolder.newFolder();
for (String name : commitLogFilenames) {
new File(commitLogDir, name).createNewFile();
}
PCollection<String> filteredFilenames =
pipeline
.apply(
"Generate All Filenames",
Create.of(commitLogFilenames).withCoder(StringUtf8Coder.of()))
"Get commitlog file patterns",
Transforms.getCommitLogFilePatterns(commitLogDir.getAbsolutePath()))
.apply("Find commitlog files", Transforms.getFilesByPatterns())
.apply(
"Filtered by Time",
Transforms.filterCommitLogsByTime(
DateTime.parse("2000-01-01T00:00:00.001Z"),
DateTime.parse("2000-01-01T00:00:00.003Z")));
DateTime.parse("2000-01-01T00:00:00.003Z")))
.apply(
"Extract path strings",
ParDo.of(
new DoFn<Metadata, String>() {
@ProcessElement
public void processElement(
@Element Metadata fileMeta, OutputReceiver<String> out) {
out.output(fileMeta.resourceId().getFilename());
}
}));
PAssert.that(filteredFilenames)
.containsInAnyOrder(
"/commit_diff_until_2000-01-01T00:00:00.001Z",
"/commit_diff_until_2000-01-01T00:00:00.002Z");
"commit_diff_until_2000-01-01T00:00:00.001Z",
"commit_diff_until_2000-01-01T00:00:00.002Z");
pipeline.run();
}
@ -187,7 +206,9 @@ public class CommitLogTransformsTest implements Serializable {
"Get CommitLog file patterns",
Transforms.getCommitLogFilePatterns(commitLogsDir.getAbsolutePath()))
.apply("Find CommitLogs", Transforms.getFilesByPatterns())
.apply(Transforms.loadCommitLogsFromFiles());
.apply(
Transforms.loadCommitLogsFromFiles(
ImmutableSet.of("Registry", "ContactResource", "DomainBase")));
InitSqlTestUtils.assertContainsExactlyElementsIn(
entities,
@ -197,4 +218,24 @@ public class CommitLogTransformsTest implements Serializable {
pipeline.run();
}
@Test
@Category(NeedsRunner.class)
public void loadOneCommitLogFile_filterByKind() {
PCollection<VersionedEntity> entities =
pipeline
.apply(
"Get CommitLog file patterns",
Transforms.getCommitLogFilePatterns(commitLogsDir.getAbsolutePath()))
.apply("Find CommitLogs", Transforms.getFilesByPatterns())
.apply(
Transforms.loadCommitLogsFromFiles(ImmutableSet.of("Registry", "ContactResource")));
InitSqlTestUtils.assertContainsExactlyElementsIn(
entities,
KV.of(fakeClock.nowUtc().getMillis() - 2, store.loadAsDatastoreEntity(registry)),
KV.of(fakeClock.nowUtc().getMillis() - 1, store.loadAsDatastoreEntity(contact)));
pipeline.run();
}
}

View file

@ -19,6 +19,7 @@ import static google.registry.testing.DatastoreHelper.newDomainBase;
import static google.registry.testing.DatastoreHelper.newRegistry;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.googlecode.objectify.Key;
import google.registry.backup.VersionedEntity;
import google.registry.model.contact.ContactResource;
@ -63,8 +64,8 @@ public class ExportloadingTransformsTest implements Serializable {
private static final ImmutableList<Class<?>> ALL_KINDS =
ImmutableList.of(Registry.class, ContactResource.class, DomainBase.class);
private static final ImmutableList<String> ALL_KIND_STRS =
ALL_KINDS.stream().map(Key::getKind).collect(ImmutableList.toImmutableList());
private static final ImmutableSet<String> ALL_KIND_STRS =
ALL_KINDS.stream().map(Key::getKind).collect(ImmutableSet.toImmutableSet());
@Rule public final transient TemporaryFolder exportRootDir = new TemporaryFolder();

View file

@ -30,6 +30,7 @@ import google.registry.backup.AppEngineEnvironment;
import google.registry.backup.VersionedEntity;
import java.io.Serializable;
import java.util.Collection;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Stream;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.transforms.DoFn;
@ -43,6 +44,9 @@ import org.apache.beam.sdk.values.TypeDescriptor;
/** Test helpers for populating SQL with Datastore backups. */
public final class InitSqlTestUtils {
// Generates unique ids to distinguish reused transforms.
private static final AtomicInteger TRANSFORM_ID_GEN = new AtomicInteger(0);
/** Converts a Datastore {@link Entity} to an Objectify entity. */
public static Object datastoreToOfyEntity(Entity entity) {
return ofy().load().fromEntity(entity);
@ -114,11 +118,12 @@ public final class InitSqlTestUtils {
PCollection<String> errMsgs =
actual
.apply(
"MapElements_" + TRANSFORM_ID_GEN.getAndIncrement(),
MapElements.into(kvs(strings(), TypeDescriptor.of(VersionedEntity.class)))
.via(rawEntity -> KV.of("The One Key", rawEntity)))
.apply(GroupByKey.create())
.apply("GroupByKey_" + TRANSFORM_ID_GEN.getAndIncrement(), GroupByKey.create())
.apply(
"assertContainsExactlyElementsIn",
"assertContainsExactlyElementsIn_" + TRANSFORM_ID_GEN.getAndIncrement(),
ParDo.of(
new DoFn<KV<String, Iterable<VersionedEntity>>, String>() {
@ProcessElement

View file

@ -0,0 +1,175 @@
// 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.beam.initsql;
import static google.registry.testing.DatastoreHelper.newContactResource;
import static google.registry.testing.DatastoreHelper.newDomainBase;
import static google.registry.testing.DatastoreHelper.newRegistry;
import com.google.appengine.api.datastore.Entity;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.googlecode.objectify.Key;
import google.registry.model.contact.ContactResource;
import google.registry.model.domain.DomainAuthInfo;
import google.registry.model.domain.DomainBase;
import google.registry.model.eppcommon.AuthInfo.PasswordAuth;
import google.registry.model.ofy.Ofy;
import google.registry.model.registry.Registry;
import google.registry.testing.FakeClock;
import google.registry.testing.InjectRule;
import java.io.File;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.joda.time.DateTime;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
/**
* Unit test for {@link Transforms#loadDatastoreSnapshot}.
*
* <p>The test setup involves three entities, one Registry, one Domain, and two Contacts. Events
* happen in the following order:
*
* <ol>
* <li>Registry and a filler Contact are inserted to Datastore.
* <li>A CommitLog is persisted.
* <li>Registry is updated.
* <li>Another Contact and Domain are inserted into Datastore.
* <li>Datastore is exported, but misses the newly inserted Contact.
* <li>Filler Contact is deleted.
* <li>A second CommitLog is persisted.
* <li>Domain is updated in the Datastore.
* <li>The third and last CommitLog is persisted.
* </ol>
*
* The final snapshot includes Registry, Domain, and Contact. This scenario verifies that:
*
* <ul>
* <li>Incremental changes committed before an export does not override the exported valie.
* <li>Entity missed by an export can be recovered from later CommitLogs.
* <li>Multiple changes to an entity is applied in order.
* <li>Deletes are properly handled.
* </ul>
*/
@RunWith(JUnit4.class)
public class LoadDatastoreSnapshotTest {
private static final DateTime START_TIME = DateTime.parse("2000-01-01T00:00:00.0Z");
private static final ImmutableList<Class<?>> ALL_KINDS =
ImmutableList.of(Registry.class, ContactResource.class, DomainBase.class);
private static final ImmutableSet<String> ALL_KIND_STRS =
ALL_KINDS.stream().map(Key::getKind).collect(ImmutableSet.toImmutableSet());
@Rule public final transient TemporaryFolder temporaryFolder = new TemporaryFolder();
@Rule public final transient InjectRule injectRule = new InjectRule();
@Rule
public final transient TestPipeline pipeline =
TestPipeline.create().enableAbandonedNodeEnforcement(true);
private FakeClock fakeClock;
private File exportRootDir;
private File exportDir;
private File commitLogsDir;
// Canned data:
private transient Entity dsRegistry;
private transient Entity dsContact;
private transient Entity dsDomain;
private transient DateTime registryLastUpdateTime;
private transient DateTime contactLastUpdateTime;
private transient DateTime domainLastUpdateTime;
@Before
public void beforeEach() throws Exception {
fakeClock = new FakeClock(START_TIME);
try (BackupTestStore store = new BackupTestStore(fakeClock)) {
injectRule.setStaticField(Ofy.class, "clock", fakeClock);
exportRootDir = temporaryFolder.newFolder();
commitLogsDir = temporaryFolder.newFolder();
Registry registry = newRegistry("tld1", "TLD1");
ContactResource fillerContact = newContactResource("contact_filler");
store.insertOrUpdate(registry, fillerContact);
store.saveCommitLogs(commitLogsDir.getAbsolutePath());
registry =
registry
.asBuilder()
.setCreateBillingCost(registry.getStandardCreateCost().plus(1.0d))
.build();
registryLastUpdateTime = fakeClock.nowUtc();
store.insertOrUpdate(registry);
ContactResource contact = newContactResource("contact");
DomainBase domain = newDomainBase("domain1.tld1", contact);
contactLastUpdateTime = fakeClock.nowUtc();
store.insertOrUpdate(contact, domain);
exportDir =
store.export(
exportRootDir.getAbsolutePath(), ALL_KINDS, ImmutableSet.of(Key.create(contact)));
store.delete(fillerContact);
store.saveCommitLogs(commitLogsDir.getAbsolutePath());
domain =
domain
.asBuilder()
.setAuthInfo(DomainAuthInfo.create(PasswordAuth.create("NewPass")))
.build();
domainLastUpdateTime = fakeClock.nowUtc();
store.insertOrUpdate(domain);
store.saveCommitLogs(commitLogsDir.getAbsolutePath());
fakeClock.advanceOneMilli();
// Save persisted data for assertions.
dsRegistry = store.loadAsDatastoreEntity(registry);
dsContact = store.loadAsDatastoreEntity(contact);
dsDomain = store.loadAsDatastoreEntity(domain);
}
}
@Test
public void loadDatastoreSnapshot() {
PCollectionTuple snapshot =
pipeline.apply(
Transforms.loadDatastoreSnapshot(
exportDir.getAbsolutePath(),
commitLogsDir.getAbsolutePath(),
START_TIME,
fakeClock.nowUtc(),
ALL_KIND_STRS));
InitSqlTestUtils.assertContainsExactlyElementsIn(
snapshot.get(Transforms.createTagForKind("DomainBase")),
KV.of(domainLastUpdateTime.getMillis(), dsDomain));
InitSqlTestUtils.assertContainsExactlyElementsIn(
snapshot.get(Transforms.createTagForKind("Registry")),
KV.of(registryLastUpdateTime.getMillis(), dsRegistry));
InitSqlTestUtils.assertContainsExactlyElementsIn(
snapshot.get(Transforms.createTagForKind("ContactResource")),
KV.of(contactLastUpdateTime.getMillis(), dsContact));
pipeline.run();
}
}