Implement read-only transaction manager modes for R3.0 migration (#1241)

This involves:
- Altering both transaction managers to check for a read-only mode at
the start of standard write actions (e.g. delete, put).
- Altering both raw layers (entity manager, ofy) to throw exceptions on
write actions as well
- Implementing bypass routes for reading / setting / removing the schedule itself
so that we don't get "stuck"
This commit is contained in:
gbrodman 2021-08-27 13:59:16 -06:00 committed by GitHub
parent ee18f5cdf6
commit d87856f16c
20 changed files with 954 additions and 51 deletions

View file

@ -215,6 +215,7 @@ PRESUBMITS = {
"RdapDomainSearchAction.java", "RdapDomainSearchAction.java",
"RdapNameserverSearchAction.java", "RdapNameserverSearchAction.java",
"RdapSearchActionBase.java", "RdapSearchActionBase.java",
"ReadOnlyCheckingEntityManager.java",
"RegistryQuery", "RegistryQuery",
}, },
): ):

View file

@ -260,7 +260,7 @@ public class ReplayCommitLogsToSqlAction implements Runnable {
.ifPresent( .ifPresent(
sqlEntity -> { sqlEntity -> {
sqlEntity.beforeSqlSaveOnReplay(); sqlEntity.beforeSqlSaveOnReplay();
jpaTm().put(sqlEntity); jpaTm().putIgnoringReadOnly(sqlEntity);
}); });
} else { } else {
// this should never happen, but we shouldn't fail on it // this should never happen, but we shouldn't fail on it
@ -293,7 +293,7 @@ public class ReplayCommitLogsToSqlAction implements Runnable {
&& !DatastoreOnlyEntity.class.isAssignableFrom(entityClass) && !DatastoreOnlyEntity.class.isAssignableFrom(entityClass)
&& entityClass.getAnnotation(javax.persistence.Entity.class) != null) { && entityClass.getAnnotation(javax.persistence.Entity.class) != null) {
ReplaySpecializer.beforeSqlDelete(entityVKey); ReplaySpecializer.beforeSqlDelete(entityVKey);
jpaTm().delete(entityVKey); jpaTm().deleteIgnoringReadOnly(entityVKey);
} }
} catch (Throwable t) { } catch (Throwable t) {
logger.atSevere().log("Error when deleting key %s", entityVKey); logger.atSevere().log("Error when deleting key %s", entityVKey);

View file

@ -36,8 +36,8 @@ import org.joda.time.DateTime;
/** /**
* A wrapper object representing the stage-to-time mapping of the Registry 3.0 Cloud SQL migration. * A wrapper object representing the stage-to-time mapping of the Registry 3.0 Cloud SQL migration.
* *
* <p>The entity is stored in Datastore throughout the entire migration so as to have a single point * <p>The entity is stored in SQL throughout the entire migration so as to have a single point of
* of access. * access.
*/ */
@Entity @Entity
public class DatabaseMigrationStateSchedule extends CrossTldSingleton implements SqlOnlyEntity { public class DatabaseMigrationStateSchedule extends CrossTldSingleton implements SqlOnlyEntity {
@ -187,12 +187,12 @@ public class DatabaseMigrationStateSchedule extends CrossTldSingleton implements
private DatabaseMigrationStateSchedule() {} private DatabaseMigrationStateSchedule() {}
@VisibleForTesting @VisibleForTesting
DatabaseMigrationStateSchedule( public DatabaseMigrationStateSchedule(
TimedTransitionProperty<MigrationState, MigrationStateTransition> migrationTransitions) { TimedTransitionProperty<MigrationState, MigrationStateTransition> migrationTransitions) {
this.migrationTransitions = migrationTransitions; this.migrationTransitions = migrationTransitions;
} }
/** Sets and persists to Datastore the provided migration transition schedule. */ /** Sets and persists to SQL the provided migration transition schedule. */
public static void set(ImmutableSortedMap<DateTime, MigrationState> migrationTransitionMap) { public static void set(ImmutableSortedMap<DateTime, MigrationState> migrationTransitionMap) {
jpaTm().assertInTransaction(); jpaTm().assertInTransaction();
TimedTransitionProperty<MigrationState, MigrationStateTransition> transitions = TimedTransitionProperty<MigrationState, MigrationStateTransition> transitions =
@ -204,7 +204,7 @@ public class DatabaseMigrationStateSchedule extends CrossTldSingleton implements
MigrationState.DATASTORE_ONLY, MigrationState.DATASTORE_ONLY,
"migrationTransitionMap must start with DATASTORE_ONLY"); "migrationTransitionMap must start with DATASTORE_ONLY");
validateTransitionAtCurrentTime(transitions); validateTransitionAtCurrentTime(transitions);
jpaTm().put(new DatabaseMigrationStateSchedule(transitions)); jpaTm().putIgnoringReadOnly(new DatabaseMigrationStateSchedule(transitions));
CACHE.invalidateAll(); CACHE.invalidateAll();
} }
@ -218,7 +218,7 @@ public class DatabaseMigrationStateSchedule extends CrossTldSingleton implements
return get().getValueAtTime(dateTime); return get().getValueAtTime(dateTime);
} }
/** Loads the currently-set migration schedule from Datastore, or the default if none exists. */ /** Loads the currently-set migration schedule from SQL, or the default if none exists. */
@VisibleForTesting @VisibleForTesting
static TimedTransitionProperty<MigrationState, MigrationStateTransition> getUncached() { static TimedTransitionProperty<MigrationState, MigrationStateTransition> getUncached() {
return jpaTm() return jpaTm()

View file

@ -336,7 +336,7 @@ public class DatastoreTransactionManager implements TransactionManager {
@Override @Override
public <T> QueryComposer<T> createQueryComposer(Class<T> entity) { public <T> QueryComposer<T> createQueryComposer(Class<T> entity) {
return new DatastoreQueryComposerImpl(entity); return new DatastoreQueryComposerImpl<>(entity);
} }
@Override @Override
@ -349,6 +349,16 @@ public class DatastoreTransactionManager implements TransactionManager {
return true; return true;
} }
@Override
public void putIgnoringReadOnly(Object entity) {
syncIfTransactionless(getOfy().saveIgnoringReadOnly().entities(toDatastoreEntity(entity)));
}
@Override
public void deleteIgnoringReadOnly(VKey<?> key) {
syncIfTransactionless(getOfy().deleteIgnoringReadOnly().key(key.getOfyKey()));
}
/** /**
* Executes the given {@link Result} instance synchronously if not in a transaction. * Executes the given {@link Result} instance synchronously if not in a transaction.
* *

View file

@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkState;
import static com.google.common.collect.Maps.uniqueIndex; import static com.google.common.collect.Maps.uniqueIndex;
import static com.googlecode.objectify.ObjectifyService.ofy; import static com.googlecode.objectify.ObjectifyService.ofy;
import static google.registry.config.RegistryConfig.getBaseOfyRetryDuration; import static google.registry.config.RegistryConfig.getBaseOfyRetryDuration;
import static google.registry.persistence.transaction.TransactionManagerFactory.assertNotReadOnlyMode;
import static google.registry.util.CollectionUtils.union; import static google.registry.util.CollectionUtils.union;
import com.google.appengine.api.datastore.DatastoreFailureException; import com.google.appengine.api.datastore.DatastoreFailureException;
@ -131,6 +132,7 @@ public class Ofy {
* <p>We only allow this in transactions so commit logs can be written in tandem with the delete. * <p>We only allow this in transactions so commit logs can be written in tandem with the delete.
*/ */
public Deleter delete() { public Deleter delete() {
assertNotReadOnlyMode();
return new AugmentedDeleter() { return new AugmentedDeleter() {
@Override @Override
protected void handleDeletion(Iterable<Key<?>> keys) { protected void handleDeletion(Iterable<Key<?>> keys) {
@ -148,12 +150,8 @@ public class Ofy {
* <p>No backups get written. * <p>No backups get written.
*/ */
public Deleter deleteWithoutBackup() { public Deleter deleteWithoutBackup() {
return new AugmentedDeleter() { assertNotReadOnlyMode();
@Override return deleteIgnoringReadOnly();
protected void handleDeletion(Iterable<Key<?>> keys) {
checkProhibitedAnnotations(keys, VirtualEntity.class);
}
};
} }
/** /**
@ -163,6 +161,7 @@ public class Ofy {
* <p>We only allow this in transactions so commit logs can be written in tandem with the save. * <p>We only allow this in transactions so commit logs can be written in tandem with the save.
*/ */
public Saver save() { public Saver save() {
assertNotReadOnlyMode();
return new AugmentedSaver() { return new AugmentedSaver() {
@Override @Override
protected void handleSave(Iterable<?> entities) { protected void handleSave(Iterable<?> entities) {
@ -182,6 +181,12 @@ public class Ofy {
* <p>No backups get written. * <p>No backups get written.
*/ */
public Saver saveWithoutBackup() { public Saver saveWithoutBackup() {
assertNotReadOnlyMode();
return saveIgnoringReadOnly();
}
/** Save, ignoring any backups or any read-only settings. */
public Saver saveIgnoringReadOnly() {
return new AugmentedSaver() { return new AugmentedSaver() {
@Override @Override
protected void handleSave(Iterable<?> entities) { protected void handleSave(Iterable<?> entities) {
@ -190,6 +195,16 @@ public class Ofy {
}; };
} }
/** Delete, ignoring any backups or any read-only settings. */
public Deleter deleteIgnoringReadOnly() {
return new AugmentedDeleter() {
@Override
protected void handleDeletion(Iterable<Key<?>> keys) {
checkProhibitedAnnotations(keys, VirtualEntity.class);
}
};
}
private Clock getClock() { private Clock getClock() {
return injectedClock == null ? clock : injectedClock; return injectedClock == null ? clock : injectedClock;
} }

View file

@ -41,6 +41,6 @@ public class SqlReplayCheckpoint extends CrossTldSingleton implements SqlOnlyEnt
SqlReplayCheckpoint checkpoint = new SqlReplayCheckpoint(); SqlReplayCheckpoint checkpoint = new SqlReplayCheckpoint();
checkpoint.lastReplayTime = lastReplayTime; checkpoint.lastReplayTime = lastReplayTime;
// this will overwrite the existing object due to the constant revisionId // this will overwrite the existing object due to the constant revisionId
jpaTm().put(checkpoint); jpaTm().putIgnoringReadOnly(checkpoint);
} }
} }

View file

@ -250,7 +250,7 @@ public class Lock extends ImmutableObject implements DatastoreAndSqlEntity, Seri
resourceName, scope, requestStatusChecker.getLogId(), now, leaseLength); resourceName, scope, requestStatusChecker.getLogId(), now, leaseLength);
// Locks are not parented under an EntityGroupRoot (so as to avoid write // Locks are not parented under an EntityGroupRoot (so as to avoid write
// contention) and don't need to be backed up. // contention) and don't need to be backed up.
tm().putWithoutBackup(newLock); tm().putIgnoringReadOnly(newLock);
return AcquireResult.create(now, lock, newLock, lockState); return AcquireResult.create(now, lock, newLock, lockState);
}); });
@ -269,18 +269,15 @@ public class Lock extends ImmutableObject implements DatastoreAndSqlEntity, Seri
// delete it. If the lock in Datastore was different then this lock is gone already; // delete it. If the lock in Datastore was different then this lock is gone already;
// this can happen if release() is called around the expiration time and the lock // this can happen if release() is called around the expiration time and the lock
// expires underneath us. // expires underneath us.
Lock loadedLock = VKey<Lock> key =
tm().loadByKeyIfPresent( VKey.create(
VKey.create( Lock.class, new LockId(resourceName, tld), Key.create(Lock.class, lockId));
Lock.class, Lock loadedLock = tm().loadByKeyIfPresent(key).orElse(null);
new LockId(resourceName, tld),
Key.create(Lock.class, lockId)))
.orElse(null);
if (Lock.this.equals(loadedLock)) { if (Lock.this.equals(loadedLock)) {
// Use deleteWithoutBackup() so that we don't create a commit log entry for deleting // Use deleteWithoutBackup() so that we don't create a commit log entry for deleting
// the lock. // the lock.
logger.atInfo().log("Deleting lock: %s", lockId); logger.atInfo().log("Deleting lock: %s", lockId);
tm().deleteWithoutBackup(Lock.this); tm().deleteIgnoringReadOnly(key);
lockMetrics.recordRelease( lockMetrics.recordRelease(
resourceName, tld, new Duration(acquiredTime, tm().getTransactionTime())); resourceName, tld, new Duration(acquiredTime, tm().getTransactionTime()));

View file

@ -119,22 +119,23 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
@Override @Override
public EntityManager getEntityManager() { public EntityManager getEntityManager() {
if (transactionInfo.get().entityManager == null) { EntityManager entityManager = transactionInfo.get().entityManager;
if (entityManager == null) {
throw new PersistenceException( throw new PersistenceException(
"No EntityManager has been initialized. getEntityManager() must be invoked in the scope" "No EntityManager has been initialized. getEntityManager() must be invoked in the scope"
+ " of a transaction"); + " of a transaction");
} }
return transactionInfo.get().entityManager; return entityManager;
} }
@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));
} }
@Override @Override
public <T> TypedQuery<T> query(CriteriaQuery<T> criteriaQuery) { public <T> TypedQuery<T> query(CriteriaQuery<T> criteriaQuery) {
return new DetachingTypedQuery(getEntityManager().createQuery(criteriaQuery)); return new DetachingTypedQuery<>(getEntityManager().createQuery(criteriaQuery));
} }
@Override @Override
@ -171,7 +172,7 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
return work.get(); return work.get();
} }
TransactionInfo txnInfo = transactionInfo.get(); TransactionInfo txnInfo = transactionInfo.get();
txnInfo.entityManager = emf.createEntityManager(); txnInfo.entityManager = createReadOnlyCheckingEntityManager();
EntityTransaction txn = txnInfo.entityManager.getTransaction(); EntityTransaction txn = txnInfo.entityManager.getTransaction();
try { try {
txn.begin(); txn.begin();
@ -203,7 +204,7 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
return work.get(); return work.get();
} }
TransactionInfo txnInfo = transactionInfo.get(); TransactionInfo txnInfo = transactionInfo.get();
txnInfo.entityManager = emf.createEntityManager(); txnInfo.entityManager = createReadOnlyCheckingEntityManager();
EntityTransaction txn = txnInfo.entityManager.getTransaction(); EntityTransaction txn = txnInfo.entityManager.getTransaction();
try { try {
txn.begin(); txn.begin();
@ -594,7 +595,7 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
@Override @Override
public <T> QueryComposer<T> createQueryComposer(Class<T> entity) { public <T> QueryComposer<T> createQueryComposer(Class<T> entity) {
return new JpaQueryComposerImpl<T>(entity); return new JpaQueryComposerImpl<>(entity);
} }
@Override @Override
@ -607,6 +608,38 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
return false; return false;
} }
@Override
public void putIgnoringReadOnly(Object entity) {
checkArgumentNotNull(entity);
if (isEntityOfIgnoredClass(entity)) {
return;
}
assertInTransaction();
// Necessary due to the changes in HistoryEntry representation during the migration to SQL
Object toPersist = toSqlEntity(entity);
TransactionInfo txn = transactionInfo.get();
Object merged = txn.entityManager.mergeIgnoringReadOnly(toPersist);
txn.objectsToSave.add(merged);
txn.addUpdate(toPersist);
}
@Override
public void deleteIgnoringReadOnly(VKey<?> key) {
checkArgumentNotNull(key, "key must be specified");
assertInTransaction();
if (IGNORED_ENTITY_CLASSES.contains(key.getKind())) {
return;
}
EntityType<?> entityType = getEntityType(key.getKind());
ImmutableSet<EntityId> entityIds = getEntityIdsFromSqlKey(entityType, key.getSqlKey());
String sql =
String.format("DELETE FROM %s WHERE %s", entityType.getName(), getAndClause(entityIds));
ReadOnlyCheckingQuery query = transactionInfo.get().entityManager.createQuery(sql);
entityIds.forEach(entityId -> query.setParameter(entityId.name, entityId.value));
transactionInfo.get().addDelete(key);
query.executeUpdateIgnoringReadOnly();
}
@Override @Override
public <T> void assertDelete(VKey<T> key) { public <T> void assertDelete(VKey<T> key) {
if (internalDelete(key) != 1) { if (internalDelete(key) != 1) {
@ -615,6 +648,10 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
} }
} }
private ReadOnlyCheckingEntityManager createReadOnlyCheckingEntityManager() {
return new ReadOnlyCheckingEntityManager(emf.createEntityManager());
}
private <T> EntityType<T> getEntityType(Class<T> clazz) { private <T> EntityType<T> getEntityType(Class<T> clazz) {
return emf.getMetamodel().entity(clazz); return emf.getMetamodel().entity(clazz);
} }
@ -750,7 +787,7 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
} }
private static class TransactionInfo { private static class TransactionInfo {
EntityManager entityManager; ReadOnlyCheckingEntityManager entityManager;
boolean inTransaction = false; boolean inTransaction = false;
DateTime transactionTime; DateTime transactionTime;

View file

@ -0,0 +1,320 @@
// 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.persistence.transaction;
import static google.registry.persistence.transaction.TransactionManagerFactory.assertNotReadOnlyMode;
import java.util.List;
import java.util.Map;
import javax.persistence.EntityGraph;
import javax.persistence.EntityManager;
import javax.persistence.EntityManagerFactory;
import javax.persistence.EntityTransaction;
import javax.persistence.FlushModeType;
import javax.persistence.LockModeType;
import javax.persistence.Query;
import javax.persistence.StoredProcedureQuery;
import javax.persistence.TypedQuery;
import javax.persistence.criteria.CriteriaBuilder;
import javax.persistence.criteria.CriteriaDelete;
import javax.persistence.criteria.CriteriaQuery;
import javax.persistence.criteria.CriteriaUpdate;
import javax.persistence.metamodel.Metamodel;
/** An {@link EntityManager} that throws exceptions on write actions if in read-only mode. */
public class ReadOnlyCheckingEntityManager implements EntityManager {
private final EntityManager delegate;
public ReadOnlyCheckingEntityManager(EntityManager delegate) {
this.delegate = delegate;
}
@Override
public void persist(Object entity) {
assertNotReadOnlyMode();
delegate.persist(entity);
}
@Override
public <T> T merge(T entity) {
assertNotReadOnlyMode();
return delegate.merge(entity);
}
@Override
public void remove(Object entity) {
assertNotReadOnlyMode();
delegate.remove(entity);
}
@Override
public <T> T find(Class<T> entityClass, Object primaryKey) {
return delegate.find(entityClass, primaryKey);
}
@Override
public <T> T find(Class<T> entityClass, Object primaryKey, Map<String, Object> properties) {
return delegate.find(entityClass, primaryKey, properties);
}
@Override
public <T> T find(Class<T> entityClass, Object primaryKey, LockModeType lockMode) {
return delegate.find(entityClass, primaryKey, lockMode);
}
@Override
public <T> T find(
Class<T> entityClass,
Object primaryKey,
LockModeType lockMode,
Map<String, Object> properties) {
return delegate.find(entityClass, primaryKey, lockMode, properties);
}
@Override
public <T> T getReference(Class<T> entityClass, Object primaryKey) {
return delegate.getReference(entityClass, primaryKey);
}
@Override
public void flush() {
delegate.flush();
}
@Override
public void setFlushMode(FlushModeType flushMode) {
delegate.setFlushMode(flushMode);
}
@Override
public FlushModeType getFlushMode() {
return delegate.getFlushMode();
}
@Override
public void lock(Object entity, LockModeType lockMode) {
assertNotReadOnlyMode();
delegate.lock(entity, lockMode);
}
@Override
public void lock(Object entity, LockModeType lockMode, Map<String, Object> properties) {
assertNotReadOnlyMode();
delegate.lock(entity, lockMode, properties);
}
@Override
public void refresh(Object entity) {
delegate.refresh(entity);
}
@Override
public void refresh(Object entity, Map<String, Object> properties) {
delegate.refresh(entity, properties);
}
@Override
public void refresh(Object entity, LockModeType lockMode) {
delegate.refresh(entity, lockMode);
}
@Override
public void refresh(Object entity, LockModeType lockMode, Map<String, Object> properties) {
delegate.refresh(entity, lockMode, properties);
}
@Override
public void clear() {
delegate.clear();
}
@Override
public void detach(Object entity) {
delegate.detach(entity);
}
@Override
public boolean contains(Object entity) {
return delegate.contains(entity);
}
@Override
public LockModeType getLockMode(Object entity) {
return delegate.getLockMode(entity);
}
@Override
public void setProperty(String propertyName, Object value) {
delegate.setProperty(propertyName, value);
}
@Override
public Map<String, Object> getProperties() {
return delegate.getProperties();
}
@Override
public ReadOnlyCheckingQuery createQuery(String qlString) {
return new ReadOnlyCheckingQuery(delegate.createQuery(qlString));
}
@Override
public <T> TypedQuery<T> createQuery(CriteriaQuery<T> criteriaQuery) {
return new ReadOnlyCheckingTypedQuery<>(delegate.createQuery(criteriaQuery));
}
@Override
public Query createQuery(CriteriaUpdate updateQuery) {
assertNotReadOnlyMode();
return delegate.createQuery(updateQuery);
}
@Override
public Query createQuery(CriteriaDelete deleteQuery) {
assertNotReadOnlyMode();
return delegate.createQuery(deleteQuery);
}
@Override
public <T> TypedQuery<T> createQuery(String qlString, Class<T> resultClass) {
return new ReadOnlyCheckingTypedQuery<>(delegate.createQuery(qlString, resultClass));
}
@Override
public Query createNamedQuery(String name) {
return new ReadOnlyCheckingQuery(delegate.createNamedQuery(name));
}
@Override
public <T> TypedQuery<T> createNamedQuery(String name, Class<T> resultClass) {
return new ReadOnlyCheckingTypedQuery<>(delegate.createNamedQuery(name, resultClass));
}
@Override
public Query createNativeQuery(String sqlString) {
return new ReadOnlyCheckingQuery(delegate.createNativeQuery(sqlString));
}
@Override
public Query createNativeQuery(String sqlString, Class resultClass) {
return new ReadOnlyCheckingQuery(delegate.createNativeQuery(sqlString, resultClass));
}
@Override
public Query createNativeQuery(String sqlString, String resultSetMapping) {
return new ReadOnlyCheckingQuery(delegate.createNativeQuery(sqlString, resultSetMapping));
}
@Override
public StoredProcedureQuery createNamedStoredProcedureQuery(String name) {
assertNotReadOnlyMode();
return delegate.createNamedStoredProcedureQuery(name);
}
@Override
public StoredProcedureQuery createStoredProcedureQuery(String procedureName) {
assertNotReadOnlyMode();
return delegate.createStoredProcedureQuery(procedureName);
}
@Override
public StoredProcedureQuery createStoredProcedureQuery(
String procedureName, Class... resultClasses) {
assertNotReadOnlyMode();
return delegate.createStoredProcedureQuery(procedureName, resultClasses);
}
@Override
public StoredProcedureQuery createStoredProcedureQuery(
String procedureName, String... resultSetMappings) {
assertNotReadOnlyMode();
return delegate.createStoredProcedureQuery(procedureName, resultSetMappings);
}
@Override
public void joinTransaction() {
delegate.joinTransaction();
}
@Override
public boolean isJoinedToTransaction() {
return delegate.isJoinedToTransaction();
}
@Override
public <T> T unwrap(Class<T> cls) {
return delegate.unwrap(cls);
}
@Override
public Object getDelegate() {
return delegate.getDelegate();
}
@Override
public void close() {
delegate.close();
}
@Override
public boolean isOpen() {
return delegate.isOpen();
}
@Override
public EntityTransaction getTransaction() {
return delegate.getTransaction();
}
@Override
public EntityManagerFactory getEntityManagerFactory() {
return delegate.getEntityManagerFactory();
}
@Override
public CriteriaBuilder getCriteriaBuilder() {
return delegate.getCriteriaBuilder();
}
@Override
public Metamodel getMetamodel() {
return delegate.getMetamodel();
}
@Override
public <T> EntityGraph<T> createEntityGraph(Class<T> rootType) {
return delegate.createEntityGraph(rootType);
}
@Override
public EntityGraph<?> createEntityGraph(String graphName) {
return delegate.createEntityGraph(graphName);
}
@Override
public EntityGraph<?> getEntityGraph(String graphName) {
return delegate.getEntityGraph(graphName);
}
@Override
public <T> List<EntityGraph<? super T>> getEntityGraphs(Class<T> entityClass) {
return delegate.getEntityGraphs(entityClass);
}
public <T> T mergeIgnoringReadOnly(T entity) {
return delegate.merge(entity);
}
}

View file

@ -0,0 +1,203 @@
// 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.persistence.transaction;
import static google.registry.persistence.transaction.TransactionManagerFactory.assertNotReadOnlyMode;
import java.util.Calendar;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.persistence.FlushModeType;
import javax.persistence.LockModeType;
import javax.persistence.Parameter;
import javax.persistence.Query;
import javax.persistence.TemporalType;
/** A {@link Query} that throws exceptions on write actions if in read-only mode. */
class ReadOnlyCheckingQuery implements Query {
private final Query delegate;
ReadOnlyCheckingQuery(Query delegate) {
this.delegate = delegate;
}
@Override
public List getResultList() {
return delegate.getResultList();
}
@Override
public Object getSingleResult() {
return delegate.getSingleResult();
}
@Override
public int executeUpdate() {
assertNotReadOnlyMode();
return delegate.executeUpdate();
}
@Override
public Query setMaxResults(int maxResult) {
return delegate.setMaxResults(maxResult);
}
@Override
public int getMaxResults() {
return delegate.getMaxResults();
}
@Override
public Query setFirstResult(int startPosition) {
return delegate.setFirstResult(startPosition);
}
@Override
public int getFirstResult() {
return delegate.getFirstResult();
}
@Override
public Query setHint(String hintName, Object value) {
return delegate.setHint(hintName, value);
}
@Override
public Map<String, Object> getHints() {
return delegate.getHints();
}
@Override
public <T> Query setParameter(Parameter<T> param, T value) {
return delegate.setParameter(param, value);
}
@Override
public Query setParameter(Parameter<Calendar> param, Calendar value, TemporalType temporalType) {
return delegate.setParameter(param, value, temporalType);
}
@Override
public Query setParameter(Parameter<Date> param, Date value, TemporalType temporalType) {
return delegate.setParameter(param, value, temporalType);
}
@Override
public Query setParameter(String name, Object value) {
return delegate.setParameter(name, value);
}
@Override
public Query setParameter(String name, Calendar value, TemporalType temporalType) {
return delegate.setParameter(name, value, temporalType);
}
@Override
public Query setParameter(String name, Date value, TemporalType temporalType) {
return delegate.setParameter(name, value, temporalType);
}
@Override
public Query setParameter(int position, Object value) {
return delegate.setParameter(position, value);
}
@Override
public Query setParameter(int position, Calendar value, TemporalType temporalType) {
return delegate.setParameter(position, value, temporalType);
}
@Override
public Query setParameter(int position, Date value, TemporalType temporalType) {
return delegate.setParameter(position, value, temporalType);
}
@Override
public Set<Parameter<?>> getParameters() {
return delegate.getParameters();
}
@Override
public Parameter<?> getParameter(String name) {
return delegate.getParameter(name);
}
@Override
public <T> Parameter<T> getParameter(String name, Class<T> type) {
return delegate.getParameter(name, type);
}
@Override
public Parameter<?> getParameter(int position) {
return delegate.getParameter(position);
}
@Override
public <T> Parameter<T> getParameter(int position, Class<T> type) {
return delegate.getParameter(position, type);
}
@Override
public boolean isBound(Parameter<?> param) {
return delegate.isBound(param);
}
@Override
public <T> T getParameterValue(Parameter<T> param) {
return delegate.getParameterValue(param);
}
@Override
public Object getParameterValue(String name) {
return delegate.getParameterValue(name);
}
@Override
public Object getParameterValue(int position) {
return delegate.getParameterValue(position);
}
@Override
public Query setFlushMode(FlushModeType flushMode) {
return delegate.setFlushMode(flushMode);
}
@Override
public FlushModeType getFlushMode() {
return delegate.getFlushMode();
}
@Override
public Query setLockMode(LockModeType lockMode) {
return delegate.setLockMode(lockMode);
}
@Override
public LockModeType getLockMode() {
return delegate.getLockMode();
}
@Override
public <T> T unwrap(Class<T> cls) {
return delegate.unwrap(cls);
}
public int executeUpdateIgnoringReadOnly() {
return delegate.executeUpdate();
}
}

View file

@ -0,0 +1,200 @@
// 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.persistence.transaction;
import static google.registry.persistence.transaction.TransactionManagerFactory.assertNotReadOnlyMode;
import java.util.Calendar;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.persistence.FlushModeType;
import javax.persistence.LockModeType;
import javax.persistence.Parameter;
import javax.persistence.TemporalType;
import javax.persistence.TypedQuery;
/** A {@link TypedQuery <T>} that throws exceptions on write actions if in read-only mode. */
class ReadOnlyCheckingTypedQuery<T> implements TypedQuery<T> {
private final TypedQuery<T> delegate;
ReadOnlyCheckingTypedQuery(TypedQuery<T> delegate) {
this.delegate = delegate;
}
@Override
public List<T> getResultList() {
return delegate.getResultList();
}
@Override
public T getSingleResult() {
return delegate.getSingleResult();
}
@Override
public int executeUpdate() {
assertNotReadOnlyMode();
return delegate.executeUpdate();
}
@Override
public TypedQuery<T> setMaxResults(int maxResult) {
return delegate.setMaxResults(maxResult);
}
@Override
public int getMaxResults() {
return delegate.getMaxResults();
}
@Override
public TypedQuery<T> setFirstResult(int startPosition) {
return delegate.setFirstResult(startPosition);
}
@Override
public int getFirstResult() {
return delegate.getFirstResult();
}
@Override
public TypedQuery<T> setHint(String hintName, Object value) {
return delegate.setHint(hintName, value);
}
@Override
public Map<String, Object> getHints() {
return delegate.getHints();
}
@Override
public <T1> TypedQuery<T> setParameter(Parameter<T1> param, T1 value) {
return delegate.setParameter(param, value);
}
@Override
public TypedQuery<T> setParameter(
Parameter<Calendar> param, Calendar value, TemporalType temporalType) {
return delegate.setParameter(param, value, temporalType);
}
@Override
public TypedQuery<T> setParameter(Parameter<Date> param, Date value, TemporalType temporalType) {
return delegate.setParameter(param, value, temporalType);
}
@Override
public TypedQuery<T> setParameter(String name, Object value) {
return delegate.setParameter(name, value);
}
@Override
public TypedQuery<T> setParameter(String name, Calendar value, TemporalType temporalType) {
return delegate.setParameter(name, value, temporalType);
}
@Override
public TypedQuery<T> setParameter(String name, Date value, TemporalType temporalType) {
return delegate.setParameter(name, value, temporalType);
}
@Override
public TypedQuery<T> setParameter(int position, Object value) {
return delegate.setParameter(position, value);
}
@Override
public TypedQuery<T> setParameter(int position, Calendar value, TemporalType temporalType) {
return delegate.setParameter(position, value, temporalType);
}
@Override
public TypedQuery<T> setParameter(int position, Date value, TemporalType temporalType) {
return delegate.setParameter(position, value, temporalType);
}
@Override
public Set<Parameter<?>> getParameters() {
return delegate.getParameters();
}
@Override
public Parameter<?> getParameter(String name) {
return delegate.getParameter(name);
}
@Override
public <X> Parameter<X> getParameter(String name, Class<X> type) {
return delegate.getParameter(name, type);
}
@Override
public Parameter<?> getParameter(int position) {
return delegate.getParameter(position);
}
@Override
public <X> Parameter<X> getParameter(int position, Class<X> type) {
return delegate.getParameter(position, type);
}
@Override
public boolean isBound(Parameter<?> param) {
return delegate.isBound(param);
}
@Override
public <X> X getParameterValue(Parameter<X> param) {
return delegate.getParameterValue(param);
}
@Override
public Object getParameterValue(String name) {
return delegate.getParameterValue(name);
}
@Override
public Object getParameterValue(int position) {
return delegate.getParameterValue(position);
}
@Override
public TypedQuery<T> setFlushMode(FlushModeType flushMode) {
return delegate.setFlushMode(flushMode);
}
@Override
public FlushModeType getFlushMode() {
return delegate.getFlushMode();
}
@Override
public TypedQuery<T> setLockMode(LockModeType lockMode) {
return delegate.setLockMode(lockMode);
}
@Override
public LockModeType getLockMode() {
return delegate.getLockMode();
}
@Override
public <X> X unwrap(Class<X> cls) {
return delegate.unwrap(cls);
}
}

View file

@ -307,4 +307,10 @@ public interface TransactionManager {
/** Returns true if the transaction manager is DatastoreTransactionManager, false otherwise. */ /** Returns true if the transaction manager is DatastoreTransactionManager, false otherwise. */
boolean isOfy(); boolean isOfy();
/** Performs the given write ignoring any read-only restrictions, for use only in replay. */
void putIgnoringReadOnly(Object entity);
/** Performs the given delete ignoring any read-only restrictions, for use only in replay. */
void deleteIgnoringReadOnly(VKey<?> key);
} }

View file

@ -86,9 +86,11 @@ public class TransactionManagerFactory {
if (tmForTest.isPresent()) { if (tmForTest.isPresent()) {
return tmForTest.get(); return tmForTest.get();
} }
PrimaryDatabase primaryDatabase = return DatabaseMigrationStateSchedule.getValueAtTime(DateTime.now(UTC))
DatabaseMigrationStateSchedule.getValueAtTime(DateTime.now(UTC)).getPrimaryDatabase(); .getPrimaryDatabase()
return primaryDatabase.equals(PrimaryDatabase.DATASTORE) ? ofyTm() : jpaTm(); .equals(PrimaryDatabase.DATASTORE)
? ofyTm()
: jpaTm();
} }
/** /**
@ -141,4 +143,17 @@ public class TransactionManagerFactory {
public static void removeTmOverrideForTest() { public static void removeTmOverrideForTest() {
tmForTest = Optional.empty(); tmForTest = Optional.empty();
} }
public static void assertNotReadOnlyMode() {
if (DatabaseMigrationStateSchedule.getValueAtTime(DateTime.now(UTC)).isReadOnly()) {
throw new ReadOnlyModeException();
}
}
/** Thrown when a write is attempted when the DB is in read-only mode. */
public static class ReadOnlyModeException extends IllegalStateException {
public ReadOnlyModeException() {
super("Registry is currently in read-only mode");
}
}
} }

View file

@ -40,6 +40,7 @@ import com.google.cloud.storage.contrib.nio.testing.LocalStorageHelper;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.ImmutableSortedMap;
import com.google.common.collect.Iterables;
import com.google.common.truth.Truth8; import com.google.common.truth.Truth8;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import com.googlecode.objectify.Key; import com.googlecode.objectify.Key;
@ -357,10 +358,10 @@ public class ReplayCommitLogsToSqlActionTest {
// even though the domain came first in the file // even though the domain came first in the file
// 2. that the allocation token delete occurred after the insertions // 2. that the allocation token delete occurred after the insertions
InOrder inOrder = Mockito.inOrder(spy); InOrder inOrder = Mockito.inOrder(spy);
inOrder.verify(spy).put(any(ContactResource.class)); inOrder.verify(spy).putIgnoringReadOnly(any(ContactResource.class));
inOrder.verify(spy).put(any(DomainBase.class)); inOrder.verify(spy).putIgnoringReadOnly(any(DomainBase.class));
inOrder.verify(spy).delete(toDelete.createVKey()); inOrder.verify(spy).deleteIgnoringReadOnly(toDelete.createVKey());
inOrder.verify(spy).put(any(SqlReplayCheckpoint.class)); inOrder.verify(spy).putIgnoringReadOnly(any(SqlReplayCheckpoint.class));
} }
@Test @Test
@ -399,8 +400,8 @@ public class ReplayCommitLogsToSqlActionTest {
// deletes have higher weight // deletes have higher weight
ArgumentCaptor<Object> putCaptor = ArgumentCaptor.forClass(Object.class); ArgumentCaptor<Object> putCaptor = ArgumentCaptor.forClass(Object.class);
InOrder inOrder = Mockito.inOrder(spy); InOrder inOrder = Mockito.inOrder(spy);
inOrder.verify(spy).delete(contact.createVKey()); inOrder.verify(spy).deleteIgnoringReadOnly(contact.createVKey());
inOrder.verify(spy).put(putCaptor.capture()); inOrder.verify(spy).putIgnoringReadOnly(putCaptor.capture());
assertThat(putCaptor.getValue().getClass()).isEqualTo(ContactResource.class); assertThat(putCaptor.getValue().getClass()).isEqualTo(ContactResource.class);
assertThat(jpaTm().transact(() -> jpaTm().loadByKey(contact.createVKey()).getEmailAddress())) assertThat(jpaTm().transact(() -> jpaTm().loadByKey(contact.createVKey()).getEmailAddress()))
.isEqualTo("replay@example.tld"); .isEqualTo("replay@example.tld");
@ -441,9 +442,9 @@ public class ReplayCommitLogsToSqlActionTest {
} }
}); });
runAndAssertSuccess(now.minusMinutes(1), 1, 1); runAndAssertSuccess(now.minusMinutes(1), 1, 1);
// jpaTm()::put should only have been called with the checkpoint // jpaTm()::putIgnoringReadOnly should only have been called with the checkpoint
verify(spy, times(2)).put(any(SqlReplayCheckpoint.class)); verify(spy, times(2)).putIgnoringReadOnly(any(SqlReplayCheckpoint.class));
verify(spy, times(2)).put(any()); verify(spy, times(2)).putIgnoringReadOnly(any());
} }
@Test @Test
@ -556,6 +557,34 @@ public class ReplayCommitLogsToSqlActionTest {
}); });
} }
@Test
void testReplay_duringReadOnly() throws Exception {
DateTime now = fakeClock.nowUtc();
jpaTm()
.transact(
() -> {
jpaTm().insertWithoutBackup(TestObject.create("previous to delete"));
SqlReplayCheckpoint.set(now.minusMinutes(2));
});
Key<CommitLogManifest> manifestKey =
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
saveDiffFile(
gcsUtils,
createCheckpoint(now.minusMinutes(1)),
CommitLogManifest.create(
getBucketKey(1),
now.minusMinutes(1),
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))),
CommitLogMutation.create(manifestKey, TestObject.create("a")));
DatabaseHelper.setMigrationScheduleToDatastorePrimaryReadOnly(fakeClock);
runAndAssertSuccess(now.minusMinutes(1), 1, 1);
jpaTm()
.transact(
() ->
assertThat(Iterables.getOnlyElement(jpaTm().loadAllOf(TestObject.class)).getId())
.isEqualTo("a"));
}
@Test @Test
void testReplay_deleteAndResaveCascade_withOtherDeletion_noErrors() throws Exception { void testReplay_deleteAndResaveCascade_withOtherDeletion_noErrors() throws Exception {
createTld("tld"); createTld("tld");

View file

@ -23,12 +23,17 @@ import static google.registry.model.common.DatabaseMigrationStateSchedule.Migrat
import static google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState.SQL_PRIMARY_READ_ONLY; import static google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState.SQL_PRIMARY_READ_ONLY;
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm; import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm; import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import static google.registry.testing.DatabaseHelper.createTld;
import static google.registry.testing.DatabaseHelper.persistResource;
import static google.registry.util.DateTimeUtils.START_OF_TIME; import static google.registry.util.DateTimeUtils.START_OF_TIME;
import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertThrows;
import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.ImmutableSortedMap;
import google.registry.model.EntityTestCase; import google.registry.model.EntityTestCase;
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState; import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
import google.registry.model.domain.token.AllocationToken;
import google.registry.model.domain.token.AllocationToken.TokenType;
import google.registry.persistence.transaction.TransactionManagerFactory.ReadOnlyModeException;
import google.registry.testing.DatabaseHelper; import google.registry.testing.DatabaseHelper;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Duration; import org.joda.time.Duration;
@ -152,6 +157,20 @@ public class DatabaseMigrationStateScheduleTest extends EntityTestCase {
assertThat(tm().isOfy()).isFalse(); assertThat(tm().isOfy()).isFalse();
} }
@Test
void testSuccess_factoryUsesReadOnly() {
createTld("tld");
fakeClock.setTo(START_OF_TIME.plusDays(1));
AllocationToken token =
new AllocationToken.Builder().setToken("token").setTokenType(TokenType.SINGLE_USE).build();
runValidTransition(DATASTORE_PRIMARY, DATASTORE_PRIMARY_READ_ONLY);
assertThrows(ReadOnlyModeException.class, () -> persistResource(token));
runValidTransition(DATASTORE_PRIMARY_READ_ONLY, SQL_PRIMARY_READ_ONLY);
assertThrows(ReadOnlyModeException.class, () -> persistResource(token));
runValidTransition(SQL_PRIMARY_READ_ONLY, SQL_PRIMARY);
persistResource(token);
}
private void runValidTransition(MigrationState from, MigrationState to) { private void runValidTransition(MigrationState from, MigrationState to) {
ImmutableSortedMap<DateTime, MigrationState> transitions = ImmutableSortedMap<DateTime, MigrationState> transitions =
createMapEndingWithTransition(from, to); createMapEndingWithTransition(from, to);

View file

@ -47,6 +47,7 @@ import google.registry.model.domain.DomainBase;
import google.registry.model.eppcommon.Trid; import google.registry.model.eppcommon.Trid;
import google.registry.model.replay.EntityTest.EntityForTesting; import google.registry.model.replay.EntityTest.EntityForTesting;
import google.registry.model.reporting.HistoryEntry; import google.registry.model.reporting.HistoryEntry;
import google.registry.persistence.transaction.TransactionManagerFactory.ReadOnlyModeException;
import google.registry.testing.AppEngineExtension; import google.registry.testing.AppEngineExtension;
import google.registry.testing.DatabaseHelper; import google.registry.testing.DatabaseHelper;
import google.registry.testing.FakeClock; import google.registry.testing.FakeClock;
@ -61,9 +62,11 @@ import org.junit.jupiter.api.extension.RegisterExtension;
/** Tests for our wrapper around Objectify. */ /** Tests for our wrapper around Objectify. */
public class OfyTest { public class OfyTest {
private final FakeClock fakeClock = new FakeClock(DateTime.parse("2000-01-01TZ"));
@RegisterExtension @RegisterExtension
public final AppEngineExtension appEngine = public final AppEngineExtension appEngine =
AppEngineExtension.builder().withDatastoreAndCloudSql().build(); AppEngineExtension.builder().withDatastoreAndCloudSql().withClock(fakeClock).build();
/** An entity to use in save and delete tests. */ /** An entity to use in save and delete tests. */
private HistoryEntry someObject; private HistoryEntry someObject;
@ -434,4 +437,12 @@ public class OfyTest {
// Test the normal loading again to verify that we've restored the original session unchanged. // Test the normal loading again to verify that we've restored the original session unchanged.
assertThat(auditedOfy().load().entity(someObject).now()).isEqualTo(someObject.asHistoryEntry()); assertThat(auditedOfy().load().entity(someObject).now()).isEqualTo(someObject.asHistoryEntry());
} }
@Test
void testReadOnly_failsWrite() {
Ofy ofy = new Ofy(fakeClock);
DatabaseHelper.setMigrationScheduleToDatastorePrimaryReadOnly(fakeClock);
assertThrows(ReadOnlyModeException.class, () -> ofy.save().entity(someObject).now());
DatabaseHelper.removeDatabaseMigrationSchedule();
}
} }

View file

@ -212,7 +212,7 @@ public class ReplicateToDatastoreActionTest {
@Test @Test
void testNotInMigrationState_doesNothing() { void testNotInMigrationState_doesNothing() {
// set a schedule that backtracks the current status to DATASTORE_PRIMARY_READ_ONLY // set a schedule that backtracks the current status to DATASTORE_PRIMARY
DateTime now = fakeClock.nowUtc(); DateTime now = fakeClock.nowUtc();
jpaTm() jpaTm()
.transact( .transact(
@ -225,6 +225,7 @@ public class ReplicateToDatastoreActionTest {
.put(START_OF_TIME.plusHours(3), MigrationState.SQL_PRIMARY) .put(START_OF_TIME.plusHours(3), MigrationState.SQL_PRIMARY)
.put(now.plusHours(1), MigrationState.SQL_PRIMARY_READ_ONLY) .put(now.plusHours(1), MigrationState.SQL_PRIMARY_READ_ONLY)
.put(now.plusHours(2), MigrationState.DATASTORE_PRIMARY_READ_ONLY) .put(now.plusHours(2), MigrationState.DATASTORE_PRIMARY_READ_ONLY)
.put(now.plusHours(3), MigrationState.DATASTORE_PRIMARY)
.build())); .build()));
fakeClock.advanceBy(Duration.standardDays(1)); fakeClock.advanceBy(Duration.standardDays(1));
@ -237,6 +238,6 @@ public class ReplicateToDatastoreActionTest {
.hasLogAtLevelWithMessage( .hasLogAtLevelWithMessage(
Level.INFO, Level.INFO,
"Skipping ReplicateToDatastoreAction because we are in migration phase " "Skipping ReplicateToDatastoreAction because we are in migration phase "
+ "DATASTORE_PRIMARY_READ_ONLY."); + "DATASTORE_PRIMARY.");
} }
} }

View file

@ -638,7 +638,9 @@ class JpaTransactionManagerImplTest {
jpaTm() jpaTm()
.transact( .transact(
() -> () ->
jpaTm().query("FROM TestEntity", TestEntity.class).getResultList().stream() jpaTm()
.query("FROM TestEntity", TestEntity.class)
.getResultList()
.forEach(e -> assertThat(jpaTm().getEntityManager().contains(e)).isFalse())); .forEach(e -> assertThat(jpaTm().getEntityManager().contains(e)).isFalse()));
jpaTm() jpaTm()
.transact( .transact(

View file

@ -31,7 +31,9 @@ import google.registry.model.ImmutableObject;
import google.registry.model.ofy.DatastoreTransactionManager; import google.registry.model.ofy.DatastoreTransactionManager;
import google.registry.model.ofy.Ofy; import google.registry.model.ofy.Ofy;
import google.registry.persistence.VKey; import google.registry.persistence.VKey;
import google.registry.persistence.transaction.TransactionManagerFactory.ReadOnlyModeException;
import google.registry.testing.AppEngineExtension; import google.registry.testing.AppEngineExtension;
import google.registry.testing.DatabaseHelper;
import google.registry.testing.DualDatabaseTest; import google.registry.testing.DualDatabaseTest;
import google.registry.testing.FakeClock; import google.registry.testing.FakeClock;
import google.registry.testing.InjectExtension; import google.registry.testing.InjectExtension;
@ -406,6 +408,13 @@ public class TransactionManagerTest {
assertThat(tm().transact(() -> tm().loadByKey(theEntity.key())).data).isEqualTo("foo"); assertThat(tm().transact(() -> tm().loadByKey(theEntity.key())).data).isEqualTo("foo");
} }
@TestOfyAndSql
void testReadOnly_writeFails() {
DatabaseHelper.setMigrationScheduleToDatastorePrimaryReadOnly(fakeClock);
assertThrows(ReadOnlyModeException.class, () -> tm().transact(() -> tm().put(theEntity)));
DatabaseHelper.removeDatabaseMigrationSchedule();
}
private static void assertEntityExists(TestEntity entity) { private static void assertEntityExists(TestEntity entity) {
assertThat(tm().transact(() -> tm().exists(entity))).isTrue(); assertThat(tm().transact(() -> tm().exists(entity))).isTrue();
} }

View file

@ -1360,6 +1360,33 @@ public class DatabaseHelper {
return entity; return entity;
} }
/**
* Sets a DATASTORE_PRIMARY_READ_ONLY state on the {@link DatabaseMigrationStateSchedule}.
*
* <p>In order to allow for tests to manipulate the clock how they need, we start the transitions
* one millisecond after the clock's current time (in case the clock's current value is
* START_OF_TIME). We then advance the clock one second so that we're in the
* DATASTORE_PRIMARY_READ_ONLY phase.
*
* <p>We must use the current time, otherwise the setting of the migration state will fail due to
* an invalid transition.
*/
public static void setMigrationScheduleToDatastorePrimaryReadOnly(FakeClock fakeClock) {
DateTime now = fakeClock.nowUtc();
jpaTm()
.transact(
() ->
DatabaseMigrationStateSchedule.set(
ImmutableSortedMap.of(
START_OF_TIME,
MigrationState.DATASTORE_ONLY,
now.plusMillis(1),
MigrationState.DATASTORE_PRIMARY,
now.plusMillis(2),
MigrationState.DATASTORE_PRIMARY_READ_ONLY)));
fakeClock.advanceBy(Duration.standardSeconds(1));
}
/** /**
* Sets a SQL_PRIMARY state on the {@link DatabaseMigrationStateSchedule}. * Sets a SQL_PRIMARY state on the {@link DatabaseMigrationStateSchedule}.
* *
@ -1395,8 +1422,9 @@ public class DatabaseHelper {
.transact( .transact(
() -> () ->
jpaTm() jpaTm()
.loadSingleton(DatabaseMigrationStateSchedule.class) .putIgnoringReadOnly(
.ifPresent(jpaTm()::delete)); new DatabaseMigrationStateSchedule(
DatabaseMigrationStateSchedule.DEFAULT_TRANSITION_MAP)));
DatabaseMigrationStateSchedule.CACHE.invalidateAll(); DatabaseMigrationStateSchedule.CACHE.invalidateAll();
} }