Load raw records from Datastore export (#605)

* Load raw records from Datastore export

Created a tool that can export from a test instance of Datastore.

Defined Beam pipeline transforms for loading raw records back from
the export.

This is the first part of the effort to create a consistent snapshot
of Datastore. The next step is to load entity records from CommitLog
files.
This commit is contained in:
Weimin Yu 2020-06-02 18:55:03 -04:00 committed by GitHub
parent 6a96b1a9cd
commit ed64dd3548
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
9 changed files with 801 additions and 15 deletions

View file

@ -0,0 +1,86 @@
// 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 com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Strings.isNullOrEmpty;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* Helpers for determining the fully qualified paths to Nomulus backup files. A backup consists of a
* Datastore export and Nomulus CommitLogs that overlap with the export.
*/
public final class BackupPaths {
private BackupPaths() {}
private static final String WILDCARD_CHAR = "*";
private static final String EXPORT_PATTERN_TEMPLATE = "%s/all_namespaces/kind_%s/input-%s";
/**
* Regex pattern that captures the kind string in a file name. Datastore places no restrictions on
* what characters may be used in a kind string.
*/
private static final String FILENAME_TO_KIND_REGEX = ".+/all_namespaces/kind_(.+)/input-.+";
private static final Pattern FILENAME_TO_KIND_PATTERN = Pattern.compile(FILENAME_TO_KIND_REGEX);
/**
* Returns a regex pattern that matches all Datastore export files of a given {@code kind}.
*
* @param exportDir path to the top directory of a Datastore export
* @param kind the 'kind' of the Datastore entity
*/
public static String getExportFileNamePattern(String exportDir, String kind) {
checkArgument(!isNullOrEmpty(exportDir), "Null or empty exportDir.");
checkArgument(!isNullOrEmpty(kind), "Null or empty kind.");
return String.format(EXPORT_PATTERN_TEMPLATE, exportDir, kind, WILDCARD_CHAR);
}
/**
* Returns the fully qualified path of a Datastore export file with the given {@code kind} and
* {@code shard}.
*
* @param exportDir path to the top directory of a Datastore export
* @param kind the 'kind' of the Datastore entity
* @param shard an integer suffix of the file name
*/
public static String getExportFileNameByShard(String exportDir, String kind, int shard) {
checkArgument(!isNullOrEmpty(exportDir), "Null or empty exportDir.");
checkArgument(!isNullOrEmpty(kind), "Null or empty kind.");
checkArgument(shard >= 0, "Negative shard %s not allowed.", shard);
return String.format(EXPORT_PATTERN_TEMPLATE, exportDir, kind, Integer.toString(shard));
}
/**
* Returns the 'kind' of entity stored in a file based on the file name.
*
* <p>This method poses low risk and greatly simplifies the implementation of some transforms in
* {@link ExportLoadingTransforms}.
*
* @see ExportLoadingTransforms
*/
public static String getKindFromFileName(String fileName) {
checkArgument(!isNullOrEmpty(fileName), "Null or empty fileName.");
Matcher matcher = FILENAME_TO_KIND_PATTERN.matcher(fileName);
checkArgument(
matcher.matches(),
"Illegal file name %s, should match %s.",
fileName,
FILENAME_TO_KIND_REGEX);
return matcher.group(1);
}
}

View file

@ -0,0 +1,117 @@
// 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.beam.initsql.BackupPaths.getExportFileNamePattern;
import static google.registry.beam.initsql.BackupPaths.getKindFromFileName;
import static org.apache.beam.sdk.values.TypeDescriptors.kvs;
import static org.apache.beam.sdk.values.TypeDescriptors.strings;
import com.google.common.collect.ImmutableList;
import google.registry.tools.LevelDbLogReader;
import java.io.IOException;
import java.util.Collection;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.io.Compression;
import org.apache.beam.sdk.io.FileIO;
import org.apache.beam.sdk.io.FileIO.ReadableFile;
import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TypeDescriptor;
/**
* {@link PTransform Pipeline transforms} for loading from Datastore export files. They are all part
* of a transformation that loads raw records from a Datastore export, and are broken apart for
* testing.
*
* <p>We drop the 'kind' information in {@link #getDatastoreExportFilePatterns} and recover it later
* using the file paths. Although we could have kept it by passing around {@link KV key-value
* pairs}, the code would be more complicated, especially in {@link #loadDataFromFiles()}.
*/
public class ExportLoadingTransforms {
/**
* Returns a {@link PTransform transform} that can generate a collection of patterns that match
* all Datastore export files of the given {@code kinds}.
*/
public static PTransform<PBegin, PCollection<String>> getDatastoreExportFilePatterns(
String exportDir, Collection<String> kinds) {
return Create.of(
kinds.stream()
.map(kind -> getExportFileNamePattern(exportDir, kind))
.collect(ImmutableList.toImmutableList()))
.withCoder(StringUtf8Coder.of());
}
/**
* Returns a {@link PTransform} from file name patterns to file {@link Metadata Metadata records}.
*/
public static PTransform<PCollection<String>, PCollection<Metadata>> getFilesByPatterns() {
return new PTransform<PCollection<String>, PCollection<Metadata>>() {
@Override
public PCollection<Metadata> expand(PCollection<String> input) {
return input.apply(FileIO.matchAll().withEmptyMatchTreatment(EmptyMatchTreatment.DISALLOW));
}
};
}
/**
* Returns a {@link PTransform} from file {@link Metadata} to {@link KV Key-Value pairs} with
* entity 'kind' as key and raw record as value.
*/
public static PTransform<PCollection<Metadata>, PCollection<KV<String, byte[]>>>
loadDataFromFiles() {
return new PTransform<PCollection<Metadata>, PCollection<KV<String, byte[]>>>() {
@Override
public PCollection<KV<String, byte[]>> expand(PCollection<Metadata> input) {
return input
.apply(FileIO.readMatches().withCompression(Compression.UNCOMPRESSED))
.apply(
MapElements.into(kvs(strings(), TypeDescriptor.of(ReadableFile.class)))
.via(file -> KV.of(getKindFromFileName(file.getMetadata().toString()), file)))
.apply("Load One LevelDb File", ParDo.of(new LoadOneFile()));
}
};
}
/**
* Reads a LevelDb file and converts each raw record into a {@link KV pair} of kind and bytes.
*
* <p>LevelDb files are not seekable because a large object may span multiple blocks. If a
* sequential read fails, the file needs to be retried from the beginning.
*/
private static class LoadOneFile extends DoFn<KV<String, ReadableFile>, KV<String, byte[]>> {
@ProcessElement
public void processElement(
@Element KV<String, ReadableFile> kv, OutputReceiver<KV<String, byte[]>> output) {
try {
LevelDbLogReader.from(kv.getValue().open())
.forEachRemaining(record -> output.output(KV.of(kv.getKey(), record)));
} catch (IOException e) {
// Let the pipeline retry the whole file.
throw new RuntimeException(e);
}
}
}
}

View file

@ -0,0 +1,3 @@
## Summary
This package contains a BEAM pipeline that populates a Cloud SQL database from a Datastore backup.

View file

@ -0,0 +1,50 @@
// 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 com.google.common.truth.Truth.assertThat;
import static google.registry.beam.initsql.BackupPaths.getKindFromFileName;
import static org.junit.jupiter.api.Assertions.assertThrows;
import org.junit.jupiter.api.Test;
/** Unit tests for {@link google.registry.beam.initsql.BackupPaths}. */
public class BackupPathsTest {
@Test
void getKindFromFileName_empty() {
assertThrows(IllegalArgumentException.class, () -> getKindFromFileName(""));
}
@Test
void getKindFromFileName_notMatch() {
assertThrows(
IllegalArgumentException.class,
() -> getKindFromFileName("/tmp/all_namespaces/kind_/input-0"));
}
@Test
void getKindFromFileName_success() {
assertThat(getKindFromFileName("scheme:/somepath/all_namespaces/kind_mykind/input-something"))
.isEqualTo("mykind");
}
@Test
void getKindFromFileName_specialChar_success() {
assertThat(
getKindFromFileName("scheme:/somepath/all_namespaces/kind_.*+? /(a)/input-something"))
.isEqualTo(".*+? /(a)");
}
}

View file

@ -0,0 +1,137 @@
// 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 com.google.common.base.Preconditions.checkState;
import static google.registry.model.ofy.ObjectifyService.ofy;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import com.google.appengine.api.datastore.Entity;
import com.googlecode.objectify.Key;
import google.registry.testing.AppEngineRule;
import google.registry.testing.FakeClock;
import google.registry.tools.LevelDbFileBuilder;
import java.io.File;
import java.io.IOException;
import java.util.Set;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
/**
* Wrapper of a Datastore test instance that can generate backups.
*
* <p>A Datastore backup consists of an unsynchronized data export and a sequence of incremental
* Commit Logs that overlap with the export process. Together they can be used to recreate a
* consistent snapshot of the Datastore.
*/
class BackupTestStore implements AutoCloseable {
private static final DateTimeFormatter EXPORT_TIMESTAMP_FORMAT =
DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss_SSS");
private final FakeClock fakeClock;
private AppEngineRule appEngine;
BackupTestStore(FakeClock fakeClock) throws Exception {
this.fakeClock = fakeClock;
this.appEngine =
new AppEngineRule.Builder()
.withDatastore()
.withoutCannedData()
.withClock(fakeClock)
.build();
this.appEngine.beforeEach(null);
}
/** Inserts or updates {@code entities} in the Datastore. */
@SafeVarargs
final void insertOrUpdate(Object... entities) {
tm().transact(() -> ofy().save().entities(entities).now());
}
/** Deletes {@code entities} from the Datastore. */
@SafeVarargs
final void delete(Object... entities) {
tm().transact(() -> ofy().delete().entities(entities).now());
}
/**
* Exports entities of the caller provided types and returns the directory where data is exported.
*
* @param exportRootPath path to the root directory of all exports. A subdirectory will be created
* for this export
* @param pojoTypes java class of all entities to be exported
* @param excludes {@link Set} of {@link Key keys} of the entities not to export.This can be used
* to simulate an inconsistent export
* @return directory where data is exported
*/
File export(String exportRootPath, Iterable<Class<?>> pojoTypes, Set<Key<?>> excludes)
throws IOException {
File exportDirectory = getExportDirectory(exportRootPath);
for (Class<?> pojoType : pojoTypes) {
File perKindFile =
new File(
BackupPaths.getExportFileNameByShard(
exportDirectory.getAbsolutePath(), Key.getKind(pojoType), 0));
checkState(
perKindFile.getParentFile().mkdirs(),
"Failed to create per-kind export directory for %s.",
perKindFile.getParentFile().getAbsolutePath());
exportOneKind(perKindFile, pojoType, excludes);
}
return exportDirectory;
}
private void exportOneKind(File perKindFile, Class<?> pojoType, Set<Key<?>> excludes)
throws IOException {
LevelDbFileBuilder builder = new LevelDbFileBuilder(perKindFile);
for (Object pojo : ofy().load().type(pojoType).iterable()) {
if (!excludes.contains(Key.create(pojo))) {
try {
builder.addEntity(toEntity(pojo));
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
builder.build();
}
void saveCommitLog() {
throw new UnsupportedOperationException("Not implemented yet");
}
@Override
public void close() throws Exception {
if (appEngine != null) {
appEngine.afterEach(null);
appEngine = null;
}
}
private Entity toEntity(Object pojo) {
return tm().transactNew(() -> ofy().save().toEntity(pojo));
}
private File getExportDirectory(String exportRootPath) {
File exportDirectory =
new File(exportRootPath, fakeClock.nowUtc().toString(EXPORT_TIMESTAMP_FORMAT));
checkState(
exportDirectory.mkdirs(),
"Failed to create export directory %s.",
exportDirectory.getAbsolutePath());
return exportDirectory;
}
}

View file

@ -0,0 +1,171 @@
// 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 com.google.common.truth.Truth.assertThat;
import static com.google.common.truth.Truth.assertWithMessage;
import static com.google.common.truth.Truth8.assertThat;
import static google.registry.model.common.EntityGroupRoot.getCrossTldKey;
import static google.registry.model.ofy.ObjectifyService.ofy;
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.appengine.api.datastore.EntityTranslator;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Streams;
import com.google.storage.onestore.v3.OnestoreEntity.EntityProto;
import com.googlecode.objectify.Key;
import google.registry.model.contact.ContactResource;
import google.registry.model.domain.DomainBase;
import google.registry.model.registry.Registry;
import google.registry.testing.FakeClock;
import google.registry.tools.LevelDbLogReader;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Collections;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Stream;
import org.joda.time.DateTime;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
/** Unit tests for {@link BackupTestStore}. */
public class BackupTestStoreTest {
private static final DateTime START_TIME = DateTime.parse("2000-01-01T00:00:00.0Z");
private FakeClock fakeClock;
private BackupTestStore store;
@TempDir File tempDir;
@BeforeEach
void beforeEach() throws Exception {
fakeClock = new FakeClock(START_TIME);
store = new BackupTestStore(fakeClock);
store.insertOrUpdate(newRegistry("tld1", "TLD1"));
ContactResource contact1 = newContactResource("contact_1");
DomainBase domain1 = newDomainBase("domain1.tld1", contact1);
store.insertOrUpdate(contact1, domain1);
}
@AfterEach
void afterEach() throws Exception {
store.close();
}
@Test
void export_filesCreated() throws IOException {
String exportRootPath = tempDir.getAbsolutePath();
File exportFolder = new File(exportRootPath, "2000-01-01T00:00:00_000");
assertWithMessage("Directory %s should not exist.", exportFolder.getAbsoluteFile())
.that(exportFolder.exists())
.isFalse();
File actualExportFolder = export(exportRootPath, Collections.EMPTY_SET);
assertThat(actualExportFolder).isEquivalentAccordingToCompareTo(exportFolder);
try (Stream<String> files =
Files.walk(exportFolder.toPath())
.filter(Files::isRegularFile)
.map(Path::toString)
.map(string -> string.substring(exportFolder.getAbsolutePath().length()))) {
assertThat(files)
.containsExactly(
"/all_namespaces/kind_Registry/input-0",
"/all_namespaces/kind_DomainBase/input-0",
"/all_namespaces/kind_ContactResource/input-0");
}
}
@Test
void export_folderNameChangesWithTime() throws IOException {
String exportRootPath = tempDir.getAbsolutePath();
fakeClock.advanceOneMilli();
File exportFolder = new File(exportRootPath, "2000-01-01T00:00:00_001");
assertWithMessage("Directory %s should not exist.", exportFolder.getAbsoluteFile())
.that(exportFolder.exists())
.isFalse();
assertThat(export(exportRootPath, Collections.EMPTY_SET))
.isEquivalentAccordingToCompareTo(exportFolder);
}
@Test
void export_dataReadBack() throws IOException {
String exportRootPath = tempDir.getAbsolutePath();
File exportFolder = export(exportRootPath, Collections.EMPTY_SET);
ImmutableList<String> tldStrings =
loadPropertyFromExportedEntities(
new File(exportFolder, "/all_namespaces/kind_Registry/input-0"),
Registry.class,
Registry::getTldStr);
assertThat(tldStrings).containsExactly("tld1");
ImmutableList<String> domainStrings =
loadPropertyFromExportedEntities(
new File(exportFolder, "/all_namespaces/kind_DomainBase/input-0"),
DomainBase.class,
DomainBase::getFullyQualifiedDomainName);
assertThat(domainStrings).containsExactly("domain1.tld1");
ImmutableList<String> contactIds =
loadPropertyFromExportedEntities(
new File(exportFolder, "/all_namespaces/kind_ContactResource/input-0"),
ContactResource.class,
ContactResource::getContactId);
assertThat(contactIds).containsExactly("contact_1");
}
@Test
void export_excludeSomeEntity() throws IOException {
store.insertOrUpdate(newRegistry("tld2", "TLD2"));
String exportRootPath = tempDir.getAbsolutePath();
File exportFolder =
export(
exportRootPath, ImmutableSet.of(Key.create(getCrossTldKey(), Registry.class, "tld1")));
ImmutableList<String> tlds =
loadPropertyFromExportedEntities(
new File(exportFolder, "/all_namespaces/kind_Registry/input-0"),
Registry.class,
Registry::getTldStr);
assertThat(tlds).containsExactly("tld2");
}
private File export(String exportRootPath, Set<Key<?>> excludes) throws IOException {
return store.export(
exportRootPath,
ImmutableList.of(ContactResource.class, DomainBase.class, Registry.class),
excludes);
}
private static <T> ImmutableList<String> loadPropertyFromExportedEntities(
File dataFile, Class<T> ofyEntityType, Function<T, String> getter) throws IOException {
return Streams.stream(LevelDbLogReader.from(dataFile.toPath()))
.map(bytes -> toOfyEntity(bytes, ofyEntityType))
.map(getter)
.collect(ImmutableList.toImmutableList());
}
private static <T> T toOfyEntity(byte[] rawRecord, Class<T> ofyEntityType) {
EntityProto proto = new EntityProto();
proto.parseFrom(rawRecord);
Entity entity = EntityTranslator.createFromPb(proto);
return ofyEntityType.cast(ofy().load().fromEntity(entity));
}
}

View file

@ -0,0 +1,201 @@
// 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.model.ofy.ObjectifyService.ofy;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
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.appengine.api.datastore.EntityTranslator;
import com.google.common.collect.ImmutableList;
import com.google.storage.onestore.v3.OnestoreEntity.EntityProto;
import com.googlecode.objectify.Key;
import google.registry.model.contact.ContactResource;
import google.registry.model.domain.DomainBase;
import google.registry.model.registry.Registry;
import google.registry.testing.FakeClock;
import java.io.File;
import java.io.Serializable;
import java.util.Collections;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
import org.apache.beam.sdk.testing.NeedsRunner;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
/**
* Unit tests for {@link ExportLoadingTransforms}.
*
* <p>This class implements {@link Serializable} so that test {@link DoFn} classes may be inlined.
*/
// TODO(weiminyu): Upgrade to JUnit5 when TestPipeline is upgraded. It is also easy to adapt with
// a wrapper.
@RunWith(JUnit4.class)
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());
@Rule public final transient TemporaryFolder exportRootDir = new TemporaryFolder();
@Rule
public final transient TestPipeline pipeline =
TestPipeline.create().enableAbandonedNodeEnforcement(true);
private FakeClock fakeClock;
private transient BackupTestStore store;
private File exportDir;
// Canned data that are persisted to Datastore, used by assertions in tests.
// TODO(weiminyu): use Ofy entity pojos directly.
private transient ImmutableList<Entity> persistedEntities;
@Before
public void beforeEach() throws Exception {
fakeClock = new FakeClock();
store = new BackupTestStore(fakeClock);
Registry registry = newRegistry("tld1", "TLD1");
store.insertOrUpdate(registry);
ContactResource contact1 = newContactResource("contact_1");
DomainBase domain1 = newDomainBase("domain1.tld1", contact1);
store.insertOrUpdate(contact1, domain1);
persistedEntities =
ImmutableList.of(registry, contact1, domain1).stream()
.map(ofyEntity -> tm().transact(() -> ofy().save().toEntity(ofyEntity)))
.collect(ImmutableList.toImmutableList());
exportDir =
store.export(exportRootDir.getRoot().getAbsolutePath(), ALL_KINDS, Collections.EMPTY_SET);
}
@After
public void afterEach() throws Exception {
if (store != null) {
store.close();
store = null;
}
}
@Test
@Category(NeedsRunner.class)
public void getBackupDataFilePatterns() {
PCollection<String> patterns =
pipeline.apply(
"Get Datastore file patterns",
ExportLoadingTransforms.getDatastoreExportFilePatterns(
exportDir.getAbsolutePath(), ALL_KIND_STRS));
ImmutableList<String> expectedPatterns =
ImmutableList.of(
exportDir.getAbsolutePath() + "/all_namespaces/kind_Registry/input-*",
exportDir.getAbsolutePath() + "/all_namespaces/kind_DomainBase/input-*",
exportDir.getAbsolutePath() + "/all_namespaces/kind_ContactResource/input-*");
PAssert.that(patterns).containsInAnyOrder(expectedPatterns);
pipeline.run();
}
@Test
@Category(NeedsRunner.class)
public void getFilesByPatterns() {
PCollection<Metadata> fileMetas =
pipeline
.apply(
"File patterns to metadata",
Create.of(
exportDir.getAbsolutePath() + "/all_namespaces/kind_Registry/input-*",
exportDir.getAbsolutePath() + "/all_namespaces/kind_DomainBase/input-*",
exportDir.getAbsolutePath()
+ "/all_namespaces/kind_ContactResource/input-*")
.withCoder(StringUtf8Coder.of()))
.apply(ExportLoadingTransforms.getFilesByPatterns());
// Transform fileMetas to file names for assertions.
PCollection<String> fileNames =
fileMetas.apply(
"File metadata to path string",
ParDo.of(
new DoFn<Metadata, String>() {
@ProcessElement
public void processElement(
@Element Metadata metadata, OutputReceiver<String> out) {
out.output(metadata.resourceId().toString());
}
}));
ImmutableList<String> expectedFilenames =
ImmutableList.of(
exportDir.getAbsolutePath() + "/all_namespaces/kind_Registry/input-0",
exportDir.getAbsolutePath() + "/all_namespaces/kind_DomainBase/input-0",
exportDir.getAbsolutePath() + "/all_namespaces/kind_ContactResource/input-0");
PAssert.that(fileNames).containsInAnyOrder(expectedFilenames);
pipeline.run();
}
@Test
public void loadDataFromFiles() {
PCollection<KV<String, byte[]>> taggedRecords =
pipeline
.apply(
"Get Datastore file patterns",
ExportLoadingTransforms.getDatastoreExportFilePatterns(
exportDir.getAbsolutePath(), ALL_KIND_STRS))
.apply("Find Datastore files", ExportLoadingTransforms.getFilesByPatterns())
.apply("Load from Datastore files", ExportLoadingTransforms.loadDataFromFiles());
// Transform bytes to pojo for analysis
PCollection<Entity> entities =
taggedRecords.apply(
"Raw records to Entity",
ParDo.of(
new DoFn<KV<String, byte[]>, Entity>() {
@ProcessElement
public void processElement(
@Element KV<String, byte[]> kv, OutputReceiver<Entity> out) {
out.output(parseBytes(kv.getValue()));
}
}));
PAssert.that(entities).containsInAnyOrder(persistedEntities);
pipeline.run();
}
private static Entity parseBytes(byte[] record) {
EntityProto proto = new EntityProto();
proto.parseFrom(record);
return EntityTranslator.createFromPb(proto);
}
}

View file

@ -123,7 +123,9 @@ public final class AppEngineRule extends ExternalResource
JpaUnitTestRule jpaUnitTestRule; JpaUnitTestRule jpaUnitTestRule;
private boolean withDatastoreAndCloudSql; private boolean withDatastore;
private boolean withoutCannedData;
private boolean withCloudSql;
private boolean enableJpaEntityCoverageCheck; private boolean enableJpaEntityCoverageCheck;
private boolean withJpaUnitTest; private boolean withJpaUnitTest;
private boolean withLocalModules; private boolean withLocalModules;
@ -148,9 +150,22 @@ public final class AppEngineRule extends ExternalResource
/** Turn on the Datastore service and the Cloud SQL service. */ /** Turn on the Datastore service and the Cloud SQL service. */
public Builder withDatastoreAndCloudSql() { public Builder withDatastoreAndCloudSql() {
rule.withDatastoreAndCloudSql = true; rule.withDatastore = rule.withCloudSql = true;
return this; return this;
} }
/** Turns on Datastore only, for use by test data generators. */
public Builder withDatastore() {
rule.withDatastore = true;
return this;
}
/** Disables insertion of canned data. */
public Builder withoutCannedData() {
rule.withoutCannedData = true;
return this;
}
/** /**
* Enables JPA entity coverage check if {@code enabled} is true. This should only be enabled for * Enables JPA entity coverage check if {@code enabled} is true. This should only be enabled for
* members of SqlIntegrationTestSuite. * members of SqlIntegrationTestSuite.
@ -221,10 +236,10 @@ public final class AppEngineRule extends ExternalResource
public AppEngineRule build() { public AppEngineRule build() {
checkState( checkState(
!rule.enableJpaEntityCoverageCheck || rule.withDatastoreAndCloudSql, !rule.enableJpaEntityCoverageCheck || rule.withCloudSql,
"withJpaEntityCoverageCheck enabled without Cloud SQL"); "withJpaEntityCoverageCheck enabled without Cloud SQL");
checkState( checkState(
!rule.withJpaUnitTest || rule.withDatastoreAndCloudSql, !rule.withJpaUnitTest || rule.withCloudSql,
"withJpaUnitTestEntities enabled without Cloud SQL"); "withJpaUnitTestEntities enabled without Cloud SQL");
checkState( checkState(
!rule.withJpaUnitTest || !rule.enableJpaEntityCoverageCheck, !rule.withJpaUnitTest || !rule.enableJpaEntityCoverageCheck,
@ -341,7 +356,7 @@ public final class AppEngineRule extends ExternalResource
@Override @Override
public void beforeEach(ExtensionContext context) throws Exception { public void beforeEach(ExtensionContext context) throws Exception {
before(); before();
if (withDatastoreAndCloudSql) { if (withCloudSql) {
JpaTestRules.Builder builder = new JpaTestRules.Builder(); JpaTestRules.Builder builder = new JpaTestRules.Builder();
if (clock != null) { if (clock != null) {
builder.withClock(clock); builder.withClock(clock);
@ -360,13 +375,15 @@ public final class AppEngineRule extends ExternalResource
jpaIntegrationTestRule.before(); jpaIntegrationTestRule.before();
} }
} }
injectTmForDualDatabaseTest(context); if (isWithDatastoreAndCloudSql()) {
injectTmForDualDatabaseTest(context);
}
} }
/** Called after each test method. JUnit 5 only. */ /** Called after each test method. JUnit 5 only. */
@Override @Override
public void afterEach(ExtensionContext context) throws Exception { public void afterEach(ExtensionContext context) throws Exception {
if (withDatastoreAndCloudSql) { if (withCloudSql) {
if (enableJpaEntityCoverageCheck) { if (enableJpaEntityCoverageCheck) {
jpaIntegrationWithCoverageExtension.afterEach(context); jpaIntegrationWithCoverageExtension.afterEach(context);
} else if (withJpaUnitTest) { } else if (withJpaUnitTest) {
@ -376,7 +393,9 @@ public final class AppEngineRule extends ExternalResource
} }
} }
after(); after();
restoreTmAfterDualDatabaseTest(context); if (isWithDatastoreAndCloudSql()) {
restoreTmAfterDualDatabaseTest(context);
}
} }
/** /**
@ -389,7 +408,7 @@ public final class AppEngineRule extends ExternalResource
@Override @Override
public Statement apply(Statement base, Description description) { public Statement apply(Statement base, Description description) {
Statement statement = base; Statement statement = base;
if (withDatastoreAndCloudSql) { if (withCloudSql) {
JpaTestRules.Builder builder = new JpaTestRules.Builder(); JpaTestRules.Builder builder = new JpaTestRules.Builder();
if (clock != null) { if (clock != null) {
builder.withClock(clock); builder.withClock(clock);
@ -410,7 +429,7 @@ public final class AppEngineRule extends ExternalResource
if (withUrlFetch) { if (withUrlFetch) {
configs.add(new LocalURLFetchServiceTestConfig()); configs.add(new LocalURLFetchServiceTestConfig());
} }
if (withDatastoreAndCloudSql) { if (withDatastore) {
configs.add(new LocalDatastoreServiceTestConfig() configs.add(new LocalDatastoreServiceTestConfig()
// We need to set this to allow cross entity group transactions. // We need to set this to allow cross entity group transactions.
.setApplyAllHighRepJobPolicy() .setApplyAllHighRepJobPolicy()
@ -462,11 +481,13 @@ public final class AppEngineRule extends ExternalResource
helper.setUp(); helper.setUp();
if (withDatastoreAndCloudSql) { if (withDatastore) {
ObjectifyService.initOfy(); ObjectifyService.initOfy();
// Reset id allocation in ObjectifyService so that ids are deterministic in tests. // Reset id allocation in ObjectifyService so that ids are deterministic in tests.
ObjectifyService.resetNextTestId(); ObjectifyService.resetNextTestId();
loadInitialData(); if (!withoutCannedData) {
loadInitialData();
}
this.ofyTestEntities.forEach(AppEngineRule::register); this.ofyTestEntities.forEach(AppEngineRule::register);
} }
} }
@ -593,6 +614,6 @@ public final class AppEngineRule extends ExternalResource
} }
boolean isWithDatastoreAndCloudSql() { boolean isWithDatastoreAndCloudSql() {
return withDatastoreAndCloudSql; return withDatastore && withCloudSql;
} }
} }

View file

@ -39,7 +39,7 @@ public final class LevelDbFileBuilder {
} }
/** Adds an {@link Entity Datastore Entity object} to the leveldb log file. */ /** Adds an {@link Entity Datastore Entity object} to the leveldb log file. */
LevelDbFileBuilder addEntity(Entity entity) throws IOException { public LevelDbFileBuilder addEntity(Entity entity) throws IOException {
EntityProto proto = EntityTranslator.convertToPb(entity); EntityProto proto = EntityTranslator.convertToPb(entity);
byte[] protoBytes = proto.toByteArray(); byte[] protoBytes = proto.toByteArray();
if (protoBytes.length > BLOCK_SIZE - (currentPos + HEADER_SIZE)) { if (protoBytes.length > BLOCK_SIZE - (currentPos + HEADER_SIZE)) {
@ -53,7 +53,7 @@ public final class LevelDbFileBuilder {
} }
/** Writes all remaining data and closes the block. */ /** Writes all remaining data and closes the block. */
void build() throws IOException { public void build() throws IOException {
out.write(currentBlock); out.write(currentBlock);
out.close(); out.close();
} }