mirror of
https://github.com/google/nomulus.git
synced 2025-07-08 20:23:24 +02:00
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:
parent
6a96b1a9cd
commit
ed64dd3548
9 changed files with 801 additions and 15 deletions
|
@ -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)");
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -123,7 +123,9 @@ public final class AppEngineRule extends ExternalResource
|
|||
|
||||
JpaUnitTestRule jpaUnitTestRule;
|
||||
|
||||
private boolean withDatastoreAndCloudSql;
|
||||
private boolean withDatastore;
|
||||
private boolean withoutCannedData;
|
||||
private boolean withCloudSql;
|
||||
private boolean enableJpaEntityCoverageCheck;
|
||||
private boolean withJpaUnitTest;
|
||||
private boolean withLocalModules;
|
||||
|
@ -148,9 +150,22 @@ public final class AppEngineRule extends ExternalResource
|
|||
|
||||
/** Turn on the Datastore service and the Cloud SQL service. */
|
||||
public Builder withDatastoreAndCloudSql() {
|
||||
rule.withDatastoreAndCloudSql = true;
|
||||
rule.withDatastore = rule.withCloudSql = true;
|
||||
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
|
||||
* members of SqlIntegrationTestSuite.
|
||||
|
@ -221,10 +236,10 @@ public final class AppEngineRule extends ExternalResource
|
|||
|
||||
public AppEngineRule build() {
|
||||
checkState(
|
||||
!rule.enableJpaEntityCoverageCheck || rule.withDatastoreAndCloudSql,
|
||||
!rule.enableJpaEntityCoverageCheck || rule.withCloudSql,
|
||||
"withJpaEntityCoverageCheck enabled without Cloud SQL");
|
||||
checkState(
|
||||
!rule.withJpaUnitTest || rule.withDatastoreAndCloudSql,
|
||||
!rule.withJpaUnitTest || rule.withCloudSql,
|
||||
"withJpaUnitTestEntities enabled without Cloud SQL");
|
||||
checkState(
|
||||
!rule.withJpaUnitTest || !rule.enableJpaEntityCoverageCheck,
|
||||
|
@ -341,7 +356,7 @@ public final class AppEngineRule extends ExternalResource
|
|||
@Override
|
||||
public void beforeEach(ExtensionContext context) throws Exception {
|
||||
before();
|
||||
if (withDatastoreAndCloudSql) {
|
||||
if (withCloudSql) {
|
||||
JpaTestRules.Builder builder = new JpaTestRules.Builder();
|
||||
if (clock != null) {
|
||||
builder.withClock(clock);
|
||||
|
@ -360,13 +375,15 @@ public final class AppEngineRule extends ExternalResource
|
|||
jpaIntegrationTestRule.before();
|
||||
}
|
||||
}
|
||||
injectTmForDualDatabaseTest(context);
|
||||
if (isWithDatastoreAndCloudSql()) {
|
||||
injectTmForDualDatabaseTest(context);
|
||||
}
|
||||
}
|
||||
|
||||
/** Called after each test method. JUnit 5 only. */
|
||||
@Override
|
||||
public void afterEach(ExtensionContext context) throws Exception {
|
||||
if (withDatastoreAndCloudSql) {
|
||||
if (withCloudSql) {
|
||||
if (enableJpaEntityCoverageCheck) {
|
||||
jpaIntegrationWithCoverageExtension.afterEach(context);
|
||||
} else if (withJpaUnitTest) {
|
||||
|
@ -376,7 +393,9 @@ public final class AppEngineRule extends ExternalResource
|
|||
}
|
||||
}
|
||||
after();
|
||||
restoreTmAfterDualDatabaseTest(context);
|
||||
if (isWithDatastoreAndCloudSql()) {
|
||||
restoreTmAfterDualDatabaseTest(context);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -389,7 +408,7 @@ public final class AppEngineRule extends ExternalResource
|
|||
@Override
|
||||
public Statement apply(Statement base, Description description) {
|
||||
Statement statement = base;
|
||||
if (withDatastoreAndCloudSql) {
|
||||
if (withCloudSql) {
|
||||
JpaTestRules.Builder builder = new JpaTestRules.Builder();
|
||||
if (clock != null) {
|
||||
builder.withClock(clock);
|
||||
|
@ -410,7 +429,7 @@ public final class AppEngineRule extends ExternalResource
|
|||
if (withUrlFetch) {
|
||||
configs.add(new LocalURLFetchServiceTestConfig());
|
||||
}
|
||||
if (withDatastoreAndCloudSql) {
|
||||
if (withDatastore) {
|
||||
configs.add(new LocalDatastoreServiceTestConfig()
|
||||
// We need to set this to allow cross entity group transactions.
|
||||
.setApplyAllHighRepJobPolicy()
|
||||
|
@ -462,11 +481,13 @@ public final class AppEngineRule extends ExternalResource
|
|||
|
||||
helper.setUp();
|
||||
|
||||
if (withDatastoreAndCloudSql) {
|
||||
if (withDatastore) {
|
||||
ObjectifyService.initOfy();
|
||||
// Reset id allocation in ObjectifyService so that ids are deterministic in tests.
|
||||
ObjectifyService.resetNextTestId();
|
||||
loadInitialData();
|
||||
if (!withoutCannedData) {
|
||||
loadInitialData();
|
||||
}
|
||||
this.ofyTestEntities.forEach(AppEngineRule::register);
|
||||
}
|
||||
}
|
||||
|
@ -593,6 +614,6 @@ public final class AppEngineRule extends ExternalResource
|
|||
}
|
||||
|
||||
boolean isWithDatastoreAndCloudSql() {
|
||||
return withDatastoreAndCloudSql;
|
||||
return withDatastore && withCloudSql;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,7 +39,7 @@ public final class LevelDbFileBuilder {
|
|||
}
|
||||
|
||||
/** 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);
|
||||
byte[] protoBytes = proto.toByteArray();
|
||||
if (protoBytes.length > BLOCK_SIZE - (currentPos + HEADER_SIZE)) {
|
||||
|
@ -53,7 +53,7 @@ public final class LevelDbFileBuilder {
|
|||
}
|
||||
|
||||
/** Writes all remaining data and closes the block. */
|
||||
void build() throws IOException {
|
||||
public void build() throws IOException {
|
||||
out.write(currentBlock);
|
||||
out.close();
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue