mirror of
https://github.com/google/nomulus.git
synced 2025-07-25 12:08:36 +02:00
parent
b2d8eec4f2
commit
2c9d60f890
8 changed files with 14 additions and 342 deletions
|
@ -109,13 +109,6 @@ PRESUBMITS = {
|
||||||
"System.(out|err).println is only allowed in tools/ packages. Please "
|
"System.(out|err).println is only allowed in tools/ packages. Please "
|
||||||
"use a logger instead.",
|
"use a logger instead.",
|
||||||
|
|
||||||
# PostgreSQLContainer instantiation must specify docker tag
|
|
||||||
# TODO(b/204572437): Fix the pattern to pass DatabaseSnapshotTest.java
|
|
||||||
PresubmitCheck(
|
|
||||||
r"[\s\S]*new\s+PostgreSQLContainer(<[\s\S]*>)?\(\s*\)[\s\S]*",
|
|
||||||
"java", {"DatabaseSnapshotTest.java"}):
|
|
||||||
"PostgreSQLContainer instantiation must specify docker tag.",
|
|
||||||
|
|
||||||
# Various Soy linting checks
|
# Various Soy linting checks
|
||||||
PresubmitCheck(
|
PresubmitCheck(
|
||||||
r".* (/\*)?\* {?@param ",
|
r".* (/\*)?\* {?@param ",
|
||||||
|
|
|
@ -1,88 +0,0 @@
|
||||||
// Copyright 2021 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.common;
|
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkState;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
|
||||||
|
|
||||||
import com.google.common.flogger.FluentLogger;
|
|
||||||
import java.util.List;
|
|
||||||
import javax.persistence.EntityManager;
|
|
||||||
import javax.persistence.EntityTransaction;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A database snapshot shareable by concurrent queries from multiple database clients. A snapshot is
|
|
||||||
* uniquely identified by its {@link #getSnapshotId snapshotId}, and must stay open until all
|
|
||||||
* concurrent queries to this snapshot have attached to it by calling {@link
|
|
||||||
* google.registry.persistence.transaction.JpaTransactionManager#setDatabaseSnapshot}. However, it
|
|
||||||
* can be closed before those queries complete.
|
|
||||||
*
|
|
||||||
* <p>This feature is <em>Postgresql-only</em>.
|
|
||||||
*
|
|
||||||
* <p>To support large queries, transaction isolation level is fixed at the REPEATABLE_READ to avoid
|
|
||||||
* exhausting predicate locks at the SERIALIZABLE level.
|
|
||||||
*/
|
|
||||||
// TODO(b/193662898): vendor-independent support for richer transaction semantics.
|
|
||||||
public class DatabaseSnapshot implements AutoCloseable {
|
|
||||||
|
|
||||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
|
||||||
|
|
||||||
private String snapshotId;
|
|
||||||
private EntityManager entityManager;
|
|
||||||
private EntityTransaction transaction;
|
|
||||||
|
|
||||||
private DatabaseSnapshot() {}
|
|
||||||
|
|
||||||
public String getSnapshotId() {
|
|
||||||
checkState(entityManager != null, "Snapshot not opened yet.");
|
|
||||||
checkState(entityManager.isOpen(), "Snapshot already closed.");
|
|
||||||
return snapshotId;
|
|
||||||
}
|
|
||||||
|
|
||||||
private DatabaseSnapshot open() {
|
|
||||||
entityManager = tm().getStandaloneEntityManager();
|
|
||||||
transaction = entityManager.getTransaction();
|
|
||||||
transaction.setRollbackOnly();
|
|
||||||
transaction.begin();
|
|
||||||
|
|
||||||
entityManager
|
|
||||||
.createNativeQuery("SET TRANSACTION ISOLATION LEVEL REPEATABLE READ")
|
|
||||||
.executeUpdate();
|
|
||||||
|
|
||||||
List<?> snapshotIds =
|
|
||||||
entityManager.createNativeQuery("SELECT pg_export_snapshot();").getResultList();
|
|
||||||
checkState(snapshotIds.size() == 1, "Unexpected number of snapshots: %s", snapshotIds.size());
|
|
||||||
snapshotId = (String) snapshotIds.get(0);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() {
|
|
||||||
if (transaction != null && transaction.isActive()) {
|
|
||||||
try {
|
|
||||||
transaction.rollback();
|
|
||||||
} catch (Exception e) {
|
|
||||||
logger.atWarning().withCause(e).log("Failed to close a Database Snapshot");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (entityManager != null && entityManager.isOpen()) {
|
|
||||||
entityManager.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static DatabaseSnapshot createSnapshot() {
|
|
||||||
return new DatabaseSnapshot().open();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -122,6 +122,7 @@ public final class RegistryJpaIO {
|
||||||
@AutoValue
|
@AutoValue
|
||||||
public abstract static class Read<R, T> extends PTransform<PBegin, PCollection<T>> {
|
public abstract static class Read<R, T> extends PTransform<PBegin, PCollection<T>> {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 6906842877429561700L;
|
||||||
public static final String DEFAULT_NAME = "RegistryJpaIO.Read";
|
public static final String DEFAULT_NAME = "RegistryJpaIO.Read";
|
||||||
|
|
||||||
abstract String name();
|
abstract String name();
|
||||||
|
@ -133,9 +134,6 @@ public final class RegistryJpaIO {
|
||||||
@Nullable
|
@Nullable
|
||||||
abstract Coder<T> coder();
|
abstract Coder<T> coder();
|
||||||
|
|
||||||
@Nullable
|
|
||||||
abstract String snapshotId();
|
|
||||||
|
|
||||||
abstract Builder<R, T> toBuilder();
|
abstract Builder<R, T> toBuilder();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -145,8 +143,7 @@ public final class RegistryJpaIO {
|
||||||
input
|
input
|
||||||
.apply("Starting " + name(), Create.of((Void) null))
|
.apply("Starting " + name(), Create.of((Void) null))
|
||||||
.apply(
|
.apply(
|
||||||
"Run query for " + name(),
|
"Run query for " + name(), ParDo.of(new QueryRunner<>(query(), resultMapper())));
|
||||||
ParDo.of(new QueryRunner<>(query(), resultMapper(), snapshotId())));
|
|
||||||
if (coder() != null) {
|
if (coder() != null) {
|
||||||
output = output.setCoder(coder());
|
output = output.setCoder(coder());
|
||||||
}
|
}
|
||||||
|
@ -165,18 +162,6 @@ public final class RegistryJpaIO {
|
||||||
return toBuilder().coder(coder).build();
|
return toBuilder().coder(coder).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Specifies the database snapshot to use for this query.
|
|
||||||
*
|
|
||||||
* <p>This feature is <em>Postgresql-only</em>. User is responsible for keeping the snapshot
|
|
||||||
* available until all JVM workers have started using it by calling {@link
|
|
||||||
* JpaTransactionManager#setDatabaseSnapshot}.
|
|
||||||
*/
|
|
||||||
// TODO(b/193662898): vendor-independent support for richer transaction semantics.
|
|
||||||
public Read<R, T> withSnapshot(@Nullable String snapshotId) {
|
|
||||||
return toBuilder().snapshotId(snapshotId).build();
|
|
||||||
}
|
|
||||||
|
|
||||||
static <R, T> Builder<R, T> builder() {
|
static <R, T> Builder<R, T> builder() {
|
||||||
return new AutoValue_RegistryJpaIO_Read.Builder<R, T>().name(DEFAULT_NAME);
|
return new AutoValue_RegistryJpaIO_Read.Builder<R, T>().name(DEFAULT_NAME);
|
||||||
}
|
}
|
||||||
|
@ -192,8 +177,6 @@ public final class RegistryJpaIO {
|
||||||
|
|
||||||
abstract Builder<R, T> coder(Coder<T> coder);
|
abstract Builder<R, T> coder(Coder<T> coder);
|
||||||
|
|
||||||
abstract Builder<R, T> snapshotId(@Nullable String sharedSnapshotId);
|
|
||||||
|
|
||||||
abstract Read<R, T> build();
|
abstract Read<R, T> build();
|
||||||
|
|
||||||
Builder<R, T> criteriaQuery(CriteriaQuerySupplier<R> criteriaQuery) {
|
Builder<R, T> criteriaQuery(CriteriaQuerySupplier<R> criteriaQuery) {
|
||||||
|
@ -214,27 +197,20 @@ public final class RegistryJpaIO {
|
||||||
}
|
}
|
||||||
|
|
||||||
static class QueryRunner<R, T> extends DoFn<Void, T> {
|
static class QueryRunner<R, T> extends DoFn<Void, T> {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 7293891513058653334L;
|
||||||
private final RegistryQuery<R> query;
|
private final RegistryQuery<R> query;
|
||||||
private final SerializableFunction<R, T> resultMapper;
|
private final SerializableFunction<R, T> resultMapper;
|
||||||
// java.util.Optional is not serializable. Use of Guava Optional is discouraged.
|
|
||||||
@Nullable private final String snapshotId;
|
|
||||||
|
|
||||||
QueryRunner(
|
QueryRunner(RegistryQuery<R> query, SerializableFunction<R, T> resultMapper) {
|
||||||
RegistryQuery<R> query,
|
|
||||||
SerializableFunction<R, T> resultMapper,
|
|
||||||
@Nullable String snapshotId) {
|
|
||||||
this.query = query;
|
this.query = query;
|
||||||
this.resultMapper = resultMapper;
|
this.resultMapper = resultMapper;
|
||||||
this.snapshotId = snapshotId;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@ProcessElement
|
@ProcessElement
|
||||||
public void processElement(OutputReceiver<T> outputReceiver) {
|
public void processElement(OutputReceiver<T> outputReceiver) {
|
||||||
tm().transactNoRetry(
|
tm().transactNoRetry(
|
||||||
() -> {
|
() -> {
|
||||||
if (snapshotId != null) {
|
|
||||||
tm().setDatabaseSnapshot(snapshotId);
|
|
||||||
}
|
|
||||||
query.stream().map(resultMapper::apply).forEach(outputReceiver::output);
|
query.stream().map(resultMapper::apply).forEach(outputReceiver::output);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -256,8 +232,8 @@ public final class RegistryJpaIO {
|
||||||
@AutoValue
|
@AutoValue
|
||||||
public abstract static class Write<T> extends PTransform<PCollection<T>, PCollection<Void>> {
|
public abstract static class Write<T> extends PTransform<PCollection<T>, PCollection<Void>> {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = -4023583243078410323L;
|
||||||
public static final String DEFAULT_NAME = "RegistryJpaIO.Write";
|
public static final String DEFAULT_NAME = "RegistryJpaIO.Write";
|
||||||
|
|
||||||
public static final int DEFAULT_BATCH_SIZE = 1;
|
public static final int DEFAULT_BATCH_SIZE = 1;
|
||||||
|
|
||||||
public abstract String name();
|
public abstract String name();
|
||||||
|
@ -321,6 +297,8 @@ public final class RegistryJpaIO {
|
||||||
|
|
||||||
/** Writes a batch of entities to a SQL database through a {@link JpaTransactionManager}. */
|
/** Writes a batch of entities to a SQL database through a {@link JpaTransactionManager}. */
|
||||||
private static class SqlBatchWriter<T> extends DoFn<KV<ShardedKey<Integer>, Iterable<T>>, Void> {
|
private static class SqlBatchWriter<T> extends DoFn<KV<ShardedKey<Integer>, Iterable<T>>, Void> {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = -7519944406319472690L;
|
||||||
private final Counter counter;
|
private final Counter counter;
|
||||||
private final SerializableFunction<T, Object> jpaConverter;
|
private final SerializableFunction<T, Object> jpaConverter;
|
||||||
|
|
||||||
|
@ -337,7 +315,7 @@ public final class RegistryJpaIO {
|
||||||
private void actuallyProcessElement(@Element KV<ShardedKey<Integer>, Iterable<T>> kv) {
|
private void actuallyProcessElement(@Element KV<ShardedKey<Integer>, Iterable<T>> kv) {
|
||||||
ImmutableList<Object> entities =
|
ImmutableList<Object> entities =
|
||||||
Streams.stream(kv.getValue())
|
Streams.stream(kv.getValue())
|
||||||
.map(this.jpaConverter::apply)
|
.map(jpaConverter::apply)
|
||||||
// TODO(b/177340730): post migration delete the line below.
|
// TODO(b/177340730): post migration delete the line below.
|
||||||
.filter(Objects::nonNull)
|
.filter(Objects::nonNull)
|
||||||
.collect(ImmutableList.toImmutableList());
|
.collect(ImmutableList.toImmutableList());
|
||||||
|
@ -373,7 +351,7 @@ public final class RegistryJpaIO {
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns this entity's primary key field(s) in a string. */
|
/** Returns this entity's primary key field(s) in a string. */
|
||||||
private String toEntityKeyString(Object entity) {
|
private static String toEntityKeyString(Object entity) {
|
||||||
try {
|
try {
|
||||||
return tm().transact(
|
return tm().transact(
|
||||||
() ->
|
() ->
|
||||||
|
|
|
@ -31,21 +31,6 @@ public interface JpaTransactionManager extends TransactionManager {
|
||||||
*/
|
*/
|
||||||
EntityManager getStandaloneEntityManager();
|
EntityManager getStandaloneEntityManager();
|
||||||
|
|
||||||
/**
|
|
||||||
* Specifies a database snapshot exported by another transaction to use in the current
|
|
||||||
* transaction.
|
|
||||||
*
|
|
||||||
* <p>This is a Postgresql-specific feature. This method must be called before any other SQL
|
|
||||||
* commands in a transaction.
|
|
||||||
*
|
|
||||||
* <p>To support large queries, transaction isolation level is fixed at the REPEATABLE_READ to
|
|
||||||
* avoid exhausting predicate locks at the SERIALIZABLE level.
|
|
||||||
*
|
|
||||||
* @see google.registry.beam.common.DatabaseSnapshot
|
|
||||||
*/
|
|
||||||
// TODO(b/193662898): vendor-independent support for richer transaction semantics.
|
|
||||||
JpaTransactionManager setDatabaseSnapshot(String snapshotId);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the {@link EntityManager} for the current request.
|
* Returns the {@link EntityManager} for the current request.
|
||||||
*
|
*
|
||||||
|
@ -56,8 +41,8 @@ public interface JpaTransactionManager extends TransactionManager {
|
||||||
/**
|
/**
|
||||||
* Creates a JPA SQL query for the given query string and result class.
|
* Creates a JPA SQL query for the given query string and result class.
|
||||||
*
|
*
|
||||||
* <p>This is a convenience method for the longer <code>
|
* <p>This is a convenience method for the longer {@code
|
||||||
* jpaTm().getEntityManager().createQuery(...)</code>.
|
* jpaTm().getEntityManager().createQuery(...)}.
|
||||||
*/
|
*/
|
||||||
<T> TypedQuery<T> query(String sqlString, Class<T> resultClass);
|
<T> TypedQuery<T> query(String sqlString, Class<T> resultClass);
|
||||||
|
|
||||||
|
@ -67,8 +52,8 @@ public interface JpaTransactionManager extends TransactionManager {
|
||||||
/**
|
/**
|
||||||
* Creates a JPA SQL query for the given query string.
|
* Creates a JPA SQL query for the given query string.
|
||||||
*
|
*
|
||||||
* <p>This is a convenience method for the longer <code>
|
* <p>This is a convenience method for the longer {@code
|
||||||
* jpaTm().getEntityManager().createQuery(...)</code>.
|
* jpaTm().getEntityManager().createQuery(...)}.
|
||||||
*
|
*
|
||||||
* <p>Note that while this method can legally be used for queries that return results, <u>it
|
* <p>Note that while this method can legally be used for queries that return results, <u>it
|
||||||
* should not be</u>, as it does not correctly detach entities as must be done for nomulus model
|
* should not be</u>, as it does not correctly detach entities as must be done for nomulus model
|
||||||
|
|
|
@ -109,22 +109,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
return entityManager;
|
return entityManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public JpaTransactionManager setDatabaseSnapshot(String snapshotId) {
|
|
||||||
// Postgresql-specific: 'set transaction' command must be called inside a transaction
|
|
||||||
assertInTransaction();
|
|
||||||
|
|
||||||
EntityManager entityManager = getEntityManager();
|
|
||||||
// Isolation is hardcoded to REPEATABLE READ, as specified by parent's Javadoc.
|
|
||||||
entityManager
|
|
||||||
.createNativeQuery("SET TRANSACTION ISOLATION LEVEL REPEATABLE READ")
|
|
||||||
.executeUpdate();
|
|
||||||
entityManager
|
|
||||||
.createNativeQuery(String.format("SET TRANSACTION SNAPSHOT '%s'", snapshotId))
|
|
||||||
.executeUpdate();
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <T> TypedQuery<T> query(String sqlString, Class<T> resultClass) {
|
public <T> TypedQuery<T> query(String sqlString, Class<T> resultClass) {
|
||||||
return new DetachingTypedQuery<>(getEntityManager().createQuery(sqlString, resultClass));
|
return new DetachingTypedQuery<>(getEntityManager().createQuery(sqlString, resultClass));
|
||||||
|
|
|
@ -1,174 +0,0 @@
|
||||||
// Copyright 2021 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.common;
|
|
||||||
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
|
||||||
import com.google.common.truth.Truth;
|
|
||||||
import google.registry.beam.TestPipelineExtension;
|
|
||||||
import google.registry.beam.common.RegistryJpaIO.Read;
|
|
||||||
import google.registry.model.tld.Tld;
|
|
||||||
import google.registry.persistence.NomulusPostgreSql;
|
|
||||||
import google.registry.persistence.PersistenceModule;
|
|
||||||
import google.registry.persistence.transaction.CriteriaQueryBuilder;
|
|
||||||
import google.registry.persistence.transaction.JpaTransactionManager;
|
|
||||||
import google.registry.persistence.transaction.JpaTransactionManagerImpl;
|
|
||||||
import google.registry.persistence.transaction.TransactionManagerFactory;
|
|
||||||
import google.registry.testing.DatabaseHelper;
|
|
||||||
import google.registry.testing.FakeClock;
|
|
||||||
import javax.persistence.Persistence;
|
|
||||||
import org.apache.beam.sdk.coders.SerializableCoder;
|
|
||||||
import org.apache.beam.sdk.testing.PAssert;
|
|
||||||
import org.apache.beam.sdk.values.PCollection;
|
|
||||||
import org.hibernate.cfg.Environment;
|
|
||||||
import org.junit.jupiter.api.AfterAll;
|
|
||||||
import org.junit.jupiter.api.BeforeAll;
|
|
||||||
import org.junit.jupiter.api.Test;
|
|
||||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
|
||||||
import org.testcontainers.containers.PostgreSQLContainer;
|
|
||||||
import org.testcontainers.junit.jupiter.Container;
|
|
||||||
import org.testcontainers.junit.jupiter.Testcontainers;
|
|
||||||
|
|
||||||
/** Unit tests for {@link DatabaseSnapshot}. */
|
|
||||||
@Testcontainers
|
|
||||||
public class DatabaseSnapshotTest {
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Directly start a PSQL database instead of going through the test extensions.
|
|
||||||
*
|
|
||||||
* <p>For reasons unknown, an EntityManagerFactory created by {@code JpaIntegrationTestExtension}
|
|
||||||
* or {@code JpaUnitTestExtension} enters a bad state after exporting the first snapshot. Starting
|
|
||||||
* with the second attempt, exports alternate between error ("cannot export a snapshot from a
|
|
||||||
* subtransaction") and success. The {@link #createSnapshot_twiceNoRead} test below fails with
|
|
||||||
* either extension. EntityManagerFactory created for production does not have this problem.
|
|
||||||
*/
|
|
||||||
@Container
|
|
||||||
private static PostgreSQLContainer sqlContainer =
|
|
||||||
new PostgreSQLContainer<>(NomulusPostgreSql.getDockerTag())
|
|
||||||
.withInitScript("sql/schema/nomulus.golden.sql");
|
|
||||||
|
|
||||||
@RegisterExtension
|
|
||||||
final transient TestPipelineExtension testPipeline =
|
|
||||||
TestPipelineExtension.create().enableAbandonedNodeEnforcement(true);
|
|
||||||
|
|
||||||
static JpaTransactionManager origJpa;
|
|
||||||
static JpaTransactionManager jpa;
|
|
||||||
|
|
||||||
static Tld registry;
|
|
||||||
|
|
||||||
@BeforeAll
|
|
||||||
static void setup() {
|
|
||||||
ImmutableMap<String, String> jpaProperties =
|
|
||||||
new ImmutableMap.Builder<String, String>()
|
|
||||||
.put(Environment.URL, sqlContainer.getJdbcUrl())
|
|
||||||
.put(Environment.USER, sqlContainer.getUsername())
|
|
||||||
.put(Environment.PASS, sqlContainer.getPassword())
|
|
||||||
.putAll(PersistenceModule.provideDefaultDatabaseConfigs())
|
|
||||||
.build();
|
|
||||||
jpa =
|
|
||||||
new JpaTransactionManagerImpl(
|
|
||||||
Persistence.createEntityManagerFactory("nomulus", jpaProperties), new FakeClock());
|
|
||||||
origJpa = tm();
|
|
||||||
TransactionManagerFactory.setJpaTm(() -> jpa);
|
|
||||||
|
|
||||||
Tld tld = DatabaseHelper.newTld("tld", "TLD");
|
|
||||||
tm().transact(() -> tm().put(tld));
|
|
||||||
registry = tm().transact(() -> tm().loadByEntity(tld));
|
|
||||||
}
|
|
||||||
|
|
||||||
@AfterAll
|
|
||||||
static void tearDown() {
|
|
||||||
TransactionManagerFactory.setJpaTm(() -> origJpa);
|
|
||||||
|
|
||||||
if (jpa != null) {
|
|
||||||
jpa.teardown();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void createSnapshot_onceNoRead() {
|
|
||||||
try (DatabaseSnapshot databaseSnapshot = DatabaseSnapshot.createSnapshot()) {}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void createSnapshot_twiceNoRead() {
|
|
||||||
try (DatabaseSnapshot databaseSnapshot = DatabaseSnapshot.createSnapshot()) {}
|
|
||||||
try (DatabaseSnapshot databaseSnapshot = DatabaseSnapshot.createSnapshot()) {}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void readSnapshot() {
|
|
||||||
try (DatabaseSnapshot databaseSnapshot = DatabaseSnapshot.createSnapshot()) {
|
|
||||||
Tld snapshotRegistry =
|
|
||||||
tm().transact(
|
|
||||||
() ->
|
|
||||||
tm().setDatabaseSnapshot(databaseSnapshot.getSnapshotId())
|
|
||||||
.loadByEntity(registry));
|
|
||||||
Truth.assertThat(snapshotRegistry).isEqualTo(registry);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void readSnapshot_withSubsequentChange() {
|
|
||||||
try (DatabaseSnapshot databaseSnapshot = DatabaseSnapshot.createSnapshot()) {
|
|
||||||
Tld updated =
|
|
||||||
registry
|
|
||||||
.asBuilder()
|
|
||||||
.setCreateBillingCost(registry.getCreateBillingCost().plus(1))
|
|
||||||
.build();
|
|
||||||
tm().transact(() -> tm().put(updated));
|
|
||||||
|
|
||||||
Tld persistedUpdate = tm().transact(() -> tm().loadByEntity(registry));
|
|
||||||
Truth.assertThat(persistedUpdate).isNotEqualTo(registry);
|
|
||||||
|
|
||||||
Tld snapshotRegistry =
|
|
||||||
tm().transact(
|
|
||||||
() ->
|
|
||||||
tm().setDatabaseSnapshot(databaseSnapshot.getSnapshotId())
|
|
||||||
.loadByEntity(registry));
|
|
||||||
Truth.assertThat(snapshotRegistry).isEqualTo(registry);
|
|
||||||
} finally {
|
|
||||||
// Revert change to registry in DB, which is shared by all test methods.
|
|
||||||
tm().transact(() -> tm().put(registry));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void readWithRegistryJpaIO() {
|
|
||||||
try (DatabaseSnapshot databaseSnapshot = DatabaseSnapshot.createSnapshot()) {
|
|
||||||
Tld updated =
|
|
||||||
registry
|
|
||||||
.asBuilder()
|
|
||||||
.setCreateBillingCost(registry.getCreateBillingCost().plus(1))
|
|
||||||
.build();
|
|
||||||
tm().transact(() -> tm().put(updated));
|
|
||||||
|
|
||||||
Read<Tld, Tld> read =
|
|
||||||
RegistryJpaIO.read(() -> CriteriaQueryBuilder.create(Tld.class).build(), x -> x)
|
|
||||||
.withCoder(SerializableCoder.of(Tld.class))
|
|
||||||
.withSnapshot(databaseSnapshot.getSnapshotId());
|
|
||||||
PCollection<Tld> registries = testPipeline.apply(read);
|
|
||||||
|
|
||||||
// This assertion depends on Registry being Serializable, which may change if the
|
|
||||||
// UnsafeSerializable interface is removed after migration.
|
|
||||||
PAssert.that(registries).containsInAnyOrder(registry);
|
|
||||||
testPipeline.run();
|
|
||||||
} finally {
|
|
||||||
// Revert change to registry in DB, which is shared by all test methods.
|
|
||||||
tm().transact(() -> tm().put(registry));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -134,7 +134,6 @@ public final class TldTest extends EntityTestCase {
|
||||||
assertThat(yaml).isEqualTo(loadFile(getClass(), "tld.yaml"));
|
assertThat(yaml).isEqualTo(loadFile(getClass(), "tld.yaml"));
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO (sarahbot): re-enable this test after we figure out why it fails in presubmits.
|
|
||||||
@Test
|
@Test
|
||||||
void testYamlToTld() throws Exception {
|
void testYamlToTld() throws Exception {
|
||||||
fakeClock.setTo(START_OF_TIME);
|
fakeClock.setTo(START_OF_TIME);
|
||||||
|
|
|
@ -59,11 +59,6 @@ public class ReplicaSimulatingJpaTransactionManager implements JpaTransactionMan
|
||||||
return delegate.getEntityManager();
|
return delegate.getEntityManager();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public JpaTransactionManager setDatabaseSnapshot(String snapshotId) {
|
|
||||||
return delegate.setDatabaseSnapshot(snapshotId);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <T> TypedQuery<T> query(String sqlString, Class<T> resultClass) {
|
public <T> TypedQuery<T> query(String sqlString, Class<T> resultClass) {
|
||||||
return delegate.query(sqlString, resultClass);
|
return delegate.query(sqlString, resultClass);
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue