mirror of
https://github.com/google/nomulus.git
synced 2025-07-09 12:43: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,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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,3 @@
|
||||||
|
## Summary
|
||||||
|
|
||||||
|
This package contains a BEAM pipeline that populates a Cloud SQL database from a Datastore backup.
|
|
@ -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;
|
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();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if (isWithDatastoreAndCloudSql()) {
|
||||||
injectTmForDualDatabaseTest(context);
|
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,8 +393,10 @@ public final class AppEngineRule extends ExternalResource
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
after();
|
after();
|
||||||
|
if (isWithDatastoreAndCloudSql()) {
|
||||||
restoreTmAfterDualDatabaseTest(context);
|
restoreTmAfterDualDatabaseTest(context);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Hack to make sure AppEngineRule is always wrapped in a {@link JpaIntegrationWithCoverageRule}.
|
* Hack to make sure AppEngineRule is always wrapped in a {@link JpaIntegrationWithCoverageRule}.
|
||||||
|
@ -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();
|
||||||
|
if (!withoutCannedData) {
|
||||||
loadInitialData();
|
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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue