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,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;
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;
}
}

View file

@ -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();
}