mirror of
https://github.com/google/nomulus.git
synced 2025-07-23 11:16:04 +02:00
Remove functional SQL<->DS replay code (#1659)
This includes: - removing the actions that do the replay - removing the tests for the replay - removing the ReplayExtension and adjusting the various tests that used it appropriately - removing functionality relating to "things that happen during replay", e.g. beforeSqlSaveOnReplay This does not include: - removing the InitSqlPipeline or similar tasks - removing e.g. SqlEntity (it's used in other places) - removing Transforms/RegistryJpaIO and other SQL-pipeline-creation code
This commit is contained in:
parent
73b3cec982
commit
2879f3dac5
86 changed files with 29 additions and 5851 deletions
|
@ -790,11 +790,6 @@ if (environment == 'alpha') {
|
|||
mainClass: 'google.registry.beam.rde.RdePipeline',
|
||||
metaData : 'google/registry/beam/rde_pipeline_metadata.json'
|
||||
],
|
||||
validateDatabase :
|
||||
[
|
||||
mainClass: 'google.registry.beam.comparedb.ValidateDatabasePipeline',
|
||||
metaData: 'google/registry/beam/validate_database_pipeline_metadata.json'
|
||||
],
|
||||
resaveAllEppResources:
|
||||
[
|
||||
mainClass: 'google.registry.beam.resave.ResaveAllEppResourcesPipeline',
|
||||
|
|
|
@ -21,7 +21,6 @@ import static google.registry.backup.ExportCommitLogDiffAction.UPPER_CHECKPOINT_
|
|||
import static google.registry.backup.RestoreCommitLogsAction.BUCKET_OVERRIDE_PARAM;
|
||||
import static google.registry.backup.RestoreCommitLogsAction.FROM_TIME_PARAM;
|
||||
import static google.registry.backup.RestoreCommitLogsAction.TO_TIME_PARAM;
|
||||
import static google.registry.backup.SyncDatastoreToSqlSnapshotAction.SQL_SNAPSHOT_ID_PARAM;
|
||||
import static google.registry.request.RequestParameters.extractOptionalParameter;
|
||||
import static google.registry.request.RequestParameters.extractRequiredDatetimeParameter;
|
||||
import static google.registry.request.RequestParameters.extractRequiredParameter;
|
||||
|
@ -99,12 +98,6 @@ public final class BackupModule {
|
|||
return extractRequiredDatetimeParameter(req, TO_TIME_PARAM);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Parameter(SQL_SNAPSHOT_ID_PARAM)
|
||||
static String provideSqlSnapshotId(HttpServletRequest req) {
|
||||
return extractRequiredParameter(req, SQL_SNAPSHOT_ID_PARAM);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Backups
|
||||
static ListeningExecutorService provideListeningExecutorService() {
|
||||
|
|
|
@ -1,321 +0,0 @@
|
|||
// 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.backup;
|
||||
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static google.registry.backup.ExportCommitLogDiffAction.DIFF_FILE_PREFIX;
|
||||
import static google.registry.backup.RestoreCommitLogsAction.DRY_RUN_PARAM;
|
||||
import static google.registry.model.ofy.EntityWritePriorities.getEntityPriority;
|
||||
import static google.registry.model.ofy.ObjectifyService.auditedOfy;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
import static google.registry.util.DateTimeUtils.isAtOrAfter;
|
||||
import static google.registry.util.DateTimeUtils.isBeforeOrAt;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
import static org.joda.time.Duration.standardHours;
|
||||
|
||||
import com.google.appengine.api.datastore.Entity;
|
||||
import com.google.appengine.api.datastore.Key;
|
||||
import com.google.cloud.storage.BlobInfo;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.gcs.GcsUtils;
|
||||
import google.registry.model.UpdateAutoTimestamp;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.ReplayDirection;
|
||||
import google.registry.model.replay.DatastoreEntity;
|
||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
||||
import google.registry.model.replay.NonReplicatedEntity;
|
||||
import google.registry.model.replay.ReplaySpecializer;
|
||||
import google.registry.model.replay.SqlReplayCheckpoint;
|
||||
import google.registry.model.server.Lock;
|
||||
import google.registry.model.translators.VKeyTranslatorFactory;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.request.Action;
|
||||
import google.registry.request.Action.Method;
|
||||
import google.registry.request.Parameter;
|
||||
import google.registry.request.Response;
|
||||
import google.registry.request.auth.Auth;
|
||||
import google.registry.util.Clock;
|
||||
import google.registry.util.RequestStatusChecker;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Optional;
|
||||
import javax.inject.Inject;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Seconds;
|
||||
|
||||
/** Action that replays commit logs to Cloud SQL to keep it up to date. */
|
||||
@Action(
|
||||
service = Action.Service.BACKEND,
|
||||
path = ReplayCommitLogsToSqlAction.PATH,
|
||||
method = Method.POST,
|
||||
automaticallyPrintOk = true,
|
||||
auth = Auth.AUTH_INTERNAL_OR_ADMIN)
|
||||
@DeleteAfterMigration
|
||||
public class ReplayCommitLogsToSqlAction implements Runnable {
|
||||
|
||||
static final String PATH = "/_dr/task/replayCommitLogsToSql";
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
public static final String REPLAY_TO_SQL_LOCK_NAME =
|
||||
ReplayCommitLogsToSqlAction.class.getSimpleName();
|
||||
|
||||
public static final Duration REPLAY_TO_SQL_LOCK_LEASE_LENGTH = standardHours(1);
|
||||
// Stop / pause where we are if we've been replaying for more than five minutes to avoid GAE
|
||||
// request timeouts
|
||||
private static final Duration REPLAY_TIMEOUT_DURATION = Duration.standardMinutes(5);
|
||||
|
||||
@Inject GcsUtils gcsUtils;
|
||||
@Inject Response response;
|
||||
@Inject RequestStatusChecker requestStatusChecker;
|
||||
@Inject GcsDiffFileLister diffLister;
|
||||
@Inject Clock clock;
|
||||
|
||||
@Inject
|
||||
@Config("commitLogGcsBucket")
|
||||
String gcsBucket;
|
||||
|
||||
/** If true, will exit after logging the commit log files that would otherwise be replayed. */
|
||||
@Inject
|
||||
@Parameter(DRY_RUN_PARAM)
|
||||
boolean dryRun;
|
||||
|
||||
@Inject
|
||||
ReplayCommitLogsToSqlAction() {}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
DateTime startTime = clock.nowUtc();
|
||||
MigrationState state = DatabaseMigrationStateSchedule.getValueAtTime(startTime);
|
||||
if (!state.getReplayDirection().equals(ReplayDirection.DATASTORE_TO_SQL)) {
|
||||
String message =
|
||||
String.format(
|
||||
"Skipping ReplayCommitLogsToSqlAction because we are in migration phase %s.", state);
|
||||
logger.atInfo().log(message);
|
||||
// App Engine will retry on any non-2xx status code, which we don't want in this case.
|
||||
response.setStatus(SC_NO_CONTENT);
|
||||
response.setPayload(message);
|
||||
return;
|
||||
}
|
||||
Optional<Lock> lock =
|
||||
Lock.acquireSql(
|
||||
REPLAY_TO_SQL_LOCK_NAME,
|
||||
null,
|
||||
REPLAY_TO_SQL_LOCK_LEASE_LENGTH,
|
||||
requestStatusChecker,
|
||||
false);
|
||||
if (!lock.isPresent()) {
|
||||
String message = "Can't acquire SQL commit log replay lock, aborting.";
|
||||
logger.atSevere().log(message);
|
||||
// App Engine will retry on any non-2xx status code, which we don't want in this case.
|
||||
// Let the next run after the next export happen naturally.
|
||||
response.setStatus(SC_NO_CONTENT);
|
||||
response.setPayload(message);
|
||||
return;
|
||||
}
|
||||
try {
|
||||
logger.atInfo().log("Beginning replay of commit logs.");
|
||||
String resultMessage;
|
||||
if (dryRun) {
|
||||
resultMessage = executeDryRun();
|
||||
} else {
|
||||
resultMessage = replayFiles(startTime);
|
||||
}
|
||||
response.setStatus(SC_OK);
|
||||
response.setPayload(resultMessage);
|
||||
logger.atInfo().log(resultMessage);
|
||||
} catch (Throwable t) {
|
||||
String message = "Errored out replaying files.";
|
||||
logger.atSevere().withCause(t).log(message);
|
||||
response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
|
||||
response.setPayload(message);
|
||||
} finally {
|
||||
lock.ifPresent(Lock::releaseSql);
|
||||
}
|
||||
}
|
||||
|
||||
private String executeDryRun() {
|
||||
// Start at the first millisecond we haven't seen yet
|
||||
DateTime searchStartTime = jpaTm().transact(() -> SqlReplayCheckpoint.get().plusMillis(1));
|
||||
// Search through the end of the hour
|
||||
DateTime searchEndTime =
|
||||
searchStartTime.withMinuteOfHour(59).withSecondOfMinute(59).withMillisOfSecond(999);
|
||||
ImmutableList<String> fileBatch =
|
||||
diffLister.listDiffFiles(gcsBucket, searchStartTime, searchEndTime).stream()
|
||||
.map(BlobInfo::getName)
|
||||
.collect(toImmutableList());
|
||||
return String.format(
|
||||
"Running in dry-run mode, the first set of commit log files processed would be from "
|
||||
+ "searching from %s to %s and would contain %d file(s). They are (limit 10): \n%s",
|
||||
searchStartTime,
|
||||
searchEndTime,
|
||||
fileBatch.size(),
|
||||
fileBatch.stream().limit(10).collect(toImmutableList()));
|
||||
}
|
||||
|
||||
private String replayFiles(DateTime startTime) {
|
||||
DateTime replayTimeoutTime = startTime.plus(REPLAY_TIMEOUT_DURATION);
|
||||
DateTime searchStartTime = jpaTm().transact(() -> SqlReplayCheckpoint.get().plusMillis(1));
|
||||
int filesProcessed = 0;
|
||||
int transactionsProcessed = 0;
|
||||
// Starting from one millisecond after the last file we processed, search for and import files
|
||||
// one hour at a time until we catch up to the current time or we hit the replay timeout (in
|
||||
// which case the next run will pick up from where we leave off).
|
||||
//
|
||||
// We use hour-long batches because GCS supports filename prefix-based searches.
|
||||
while (true) {
|
||||
if (isAtOrAfter(clock.nowUtc(), replayTimeoutTime)) {
|
||||
return createResponseString(
|
||||
"Reached max execution time", startTime, filesProcessed, transactionsProcessed);
|
||||
}
|
||||
if (isBeforeOrAt(clock.nowUtc(), searchStartTime)) {
|
||||
return createResponseString(
|
||||
"Caught up to current time", startTime, filesProcessed, transactionsProcessed);
|
||||
}
|
||||
// Search through the end of the hour
|
||||
DateTime searchEndTime =
|
||||
searchStartTime.withMinuteOfHour(59).withSecondOfMinute(59).withMillisOfSecond(999);
|
||||
ImmutableList<BlobInfo> fileBatch =
|
||||
diffLister.listDiffFiles(gcsBucket, searchStartTime, searchEndTime);
|
||||
if (fileBatch.isEmpty()) {
|
||||
logger.atInfo().log(
|
||||
"No remaining files found in hour %s, continuing search in the next hour.",
|
||||
searchStartTime.toString("yyyy-MM-dd HH"));
|
||||
}
|
||||
for (BlobInfo file : fileBatch) {
|
||||
transactionsProcessed += processFile(file);
|
||||
filesProcessed++;
|
||||
if (clock.nowUtc().isAfter(replayTimeoutTime)) {
|
||||
return createResponseString(
|
||||
"Reached max execution time", startTime, filesProcessed, transactionsProcessed);
|
||||
}
|
||||
}
|
||||
searchStartTime = searchEndTime.plusMillis(1);
|
||||
}
|
||||
}
|
||||
|
||||
private String createResponseString(
|
||||
String msg, DateTime startTime, int filesProcessed, int transactionsProcessed) {
|
||||
double tps =
|
||||
(double) transactionsProcessed
|
||||
/ (double) Seconds.secondsBetween(startTime, clock.nowUtc()).getSeconds();
|
||||
return String.format(
|
||||
"%s after replaying %d file(s) containing %d total transaction(s) (%.2f tx/s).",
|
||||
msg, filesProcessed, transactionsProcessed, tps);
|
||||
}
|
||||
|
||||
/**
|
||||
* Replays the commit logs in the given commit log file and returns the number of transactions
|
||||
* committed.
|
||||
*/
|
||||
private int processFile(BlobInfo metadata) {
|
||||
try (InputStream input = gcsUtils.openInputStream(metadata.getBlobId())) {
|
||||
// Load and process the Datastore transactions one at a time
|
||||
ImmutableList<ImmutableList<VersionedEntity>> allTransactions =
|
||||
CommitLogImports.loadEntitiesByTransaction(input);
|
||||
try (UpdateAutoTimestamp.DisableAutoUpdateResource disabler =
|
||||
UpdateAutoTimestamp.disableAutoUpdate()) {
|
||||
allTransactions.forEach(
|
||||
transaction -> jpaTm().transact(() -> replayTransaction(transaction)));
|
||||
}
|
||||
// if we succeeded, set the last-seen time
|
||||
DateTime checkpoint = DateTime.parse(metadata.getName().substring(DIFF_FILE_PREFIX.length()));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(checkpoint));
|
||||
logger.atInfo().log(
|
||||
"Replayed %d transactions from commit log file %s with size %d B.",
|
||||
allTransactions.size(), metadata.getName(), metadata.getSize());
|
||||
return allTransactions.size();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(
|
||||
"Errored out while replaying commit log file " + metadata.getName(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private void replayTransaction(ImmutableList<VersionedEntity> transaction) {
|
||||
transaction.stream()
|
||||
.sorted(ReplayCommitLogsToSqlAction::compareByWeight)
|
||||
.forEach(
|
||||
versionedEntity -> {
|
||||
if (versionedEntity.getEntity().isPresent()) {
|
||||
handleEntityPut(versionedEntity.getEntity().get());
|
||||
} else {
|
||||
handleEntityDelete(versionedEntity);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void handleEntityPut(Entity entity) {
|
||||
Object ofyPojo = auditedOfy().toPojo(entity);
|
||||
try {
|
||||
if (ofyPojo instanceof DatastoreEntity) {
|
||||
DatastoreEntity datastoreEntity = (DatastoreEntity) ofyPojo;
|
||||
datastoreEntity
|
||||
.toSqlEntity()
|
||||
.ifPresent(
|
||||
sqlEntity -> {
|
||||
sqlEntity.beforeSqlSaveOnReplay();
|
||||
jpaTm().putIgnoringReadOnlyWithoutBackup(sqlEntity);
|
||||
});
|
||||
} else {
|
||||
// this should never happen, but we shouldn't fail on it
|
||||
logger.atSevere().log(
|
||||
"%s does not implement DatastoreEntity, which is necessary for SQL replay.",
|
||||
ofyPojo.getClass());
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
logger.atSevere().log("Error when replaying object %s.", ofyPojo);
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
|
||||
private void handleEntityDelete(VersionedEntity entityToDelete) {
|
||||
Key key = entityToDelete.key();
|
||||
VKey<?> entityVKey;
|
||||
try {
|
||||
entityVKey = VKeyTranslatorFactory.createVKey(key);
|
||||
} catch (RuntimeException e) {
|
||||
// This means that the key wasn't convertible to VKey through the standard methods or via
|
||||
// a createVKey method. This means that the object isn't persisted in SQL so we ignore it.
|
||||
logger.atInfo().log(
|
||||
"Skipping SQL delete for kind %s since it is not convertible.", key.getKind());
|
||||
return;
|
||||
}
|
||||
try {
|
||||
Class<?> entityClass = entityVKey.getKind();
|
||||
// Delete the key iff the class represents a JPA entity that is replicated
|
||||
if (!NonReplicatedEntity.class.isAssignableFrom(entityClass)
|
||||
&& !DatastoreOnlyEntity.class.isAssignableFrom(entityClass)
|
||||
&& entityClass.getAnnotation(javax.persistence.Entity.class) != null) {
|
||||
ReplaySpecializer.beforeSqlDelete(entityVKey);
|
||||
jpaTm().deleteIgnoringReadOnlyWithoutBackup(entityVKey);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
logger.atSevere().log("Error when deleting key %s.", entityVKey);
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
|
||||
private static int compareByWeight(VersionedEntity a, VersionedEntity b) {
|
||||
return getEntityPriority(a.key().getKind(), !a.getEntity().isPresent())
|
||||
- getEntityPriority(b.key().getKind(), !b.getEntity().isPresent());
|
||||
}
|
||||
}
|
|
@ -1,187 +0,0 @@
|
|||
// Copyright 2022 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.backup;
|
||||
|
||||
import static javax.servlet.http.HttpServletResponse.SC_INTERNAL_SERVER_ERROR;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import google.registry.beam.comparedb.LatestDatastoreSnapshotFinder;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.model.ofy.CommitLogCheckpoint;
|
||||
import google.registry.model.replay.ReplicateToDatastoreAction;
|
||||
import google.registry.request.Action;
|
||||
import google.registry.request.Action.Service;
|
||||
import google.registry.request.Parameter;
|
||||
import google.registry.request.Response;
|
||||
import google.registry.request.auth.Auth;
|
||||
import google.registry.util.Sleeper;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Optional;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/**
|
||||
* Synchronizes Datastore to a given SQL snapshot when SQL is the primary database.
|
||||
*
|
||||
* <p>The caller takes the responsibility for:
|
||||
*
|
||||
* <ul>
|
||||
* <li>verifying the current migration stage
|
||||
* <li>acquiring the {@link ReplicateToDatastoreAction#REPLICATE_TO_DATASTORE_LOCK_NAME
|
||||
* replication lock}, and
|
||||
* <li>while holding the lock, creating an SQL snapshot and invoking this action with the snapshot
|
||||
* id
|
||||
* </ul>
|
||||
*
|
||||
* The caller may release the replication lock upon receiving the response from this action. Please
|
||||
* refer to {@link google.registry.tools.ValidateDatastoreCommand} for more information on usage.
|
||||
*
|
||||
* <p>This action plays SQL transactions up to the user-specified snapshot, creates a new CommitLog
|
||||
* checkpoint, and exports all CommitLogs to GCS up to this checkpoint. The timestamp of this
|
||||
* checkpoint can be used to recreate a Datastore snapshot that is equivalent to the given SQL
|
||||
* snapshot. If this action succeeds, the checkpoint timestamp is included in the response (the
|
||||
* format of which is defined by {@link #SUCCESS_RESPONSE_TEMPLATE}).
|
||||
*/
|
||||
@Action(
|
||||
service = Service.BACKEND,
|
||||
path = SyncDatastoreToSqlSnapshotAction.PATH,
|
||||
method = Action.Method.POST,
|
||||
auth = Auth.AUTH_INTERNAL_OR_ADMIN)
|
||||
@DeleteAfterMigration
|
||||
public class SyncDatastoreToSqlSnapshotAction implements Runnable {
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
public static final String PATH = "/_dr/task/syncDatastoreToSqlSnapshot";
|
||||
|
||||
public static final String SUCCESS_RESPONSE_TEMPLATE =
|
||||
"Datastore is up-to-date with provided SQL snapshot (%s). CommitLog timestamp is (%s).";
|
||||
|
||||
static final String SQL_SNAPSHOT_ID_PARAM = "sqlSnapshotId";
|
||||
|
||||
private static final int COMMITLOGS_PRESENCE_CHECK_ATTEMPTS = 10;
|
||||
private static final Duration COMMITLOGS_PRESENCE_CHECK_DELAY = Duration.standardSeconds(6);
|
||||
|
||||
private final Response response;
|
||||
private final Sleeper sleeper;
|
||||
|
||||
@Config("commitLogGcsBucket")
|
||||
private final String gcsBucket;
|
||||
|
||||
private final GcsDiffFileLister gcsDiffFileLister;
|
||||
private final LatestDatastoreSnapshotFinder datastoreSnapshotFinder;
|
||||
private final CommitLogCheckpointAction commitLogCheckpointAction;
|
||||
private final String sqlSnapshotId;
|
||||
|
||||
@Inject
|
||||
SyncDatastoreToSqlSnapshotAction(
|
||||
Response response,
|
||||
Sleeper sleeper,
|
||||
@Config("commitLogGcsBucket") String gcsBucket,
|
||||
GcsDiffFileLister gcsDiffFileLister,
|
||||
LatestDatastoreSnapshotFinder datastoreSnapshotFinder,
|
||||
CommitLogCheckpointAction commitLogCheckpointAction,
|
||||
@Parameter(SQL_SNAPSHOT_ID_PARAM) String sqlSnapshotId) {
|
||||
this.response = response;
|
||||
this.sleeper = sleeper;
|
||||
this.gcsBucket = gcsBucket;
|
||||
this.gcsDiffFileLister = gcsDiffFileLister;
|
||||
this.datastoreSnapshotFinder = datastoreSnapshotFinder;
|
||||
this.commitLogCheckpointAction = commitLogCheckpointAction;
|
||||
this.sqlSnapshotId = sqlSnapshotId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
logger.atInfo().log("Datastore validation invoked. SqlSnapshotId is %s.", sqlSnapshotId);
|
||||
|
||||
try {
|
||||
CommitLogCheckpoint checkpoint = ensureDatabasesComparable(sqlSnapshotId);
|
||||
response.setStatus(SC_OK);
|
||||
response.setPayload(
|
||||
String.format(SUCCESS_RESPONSE_TEMPLATE, sqlSnapshotId, checkpoint.getCheckpointTime()));
|
||||
return;
|
||||
} catch (Throwable e) {
|
||||
logger.atSevere().withCause(e).log("Failed to sync Datastore to SQL.");
|
||||
response.setStatus(SC_INTERNAL_SERVER_ERROR);
|
||||
response.setPayload(getStackTrace(e));
|
||||
}
|
||||
}
|
||||
|
||||
private static String getStackTrace(Throwable e) {
|
||||
try {
|
||||
ByteArrayOutputStream bis = new ByteArrayOutputStream();
|
||||
PrintStream printStream = new PrintStream(bis);
|
||||
e.printStackTrace(printStream);
|
||||
printStream.close();
|
||||
return bis.toString();
|
||||
} catch (RuntimeException re) {
|
||||
return re.getMessage();
|
||||
}
|
||||
}
|
||||
|
||||
private CommitLogCheckpoint ensureDatabasesComparable(String sqlSnapshotId) {
|
||||
// Replicate SQL transaction to Datastore, up to when this snapshot is taken.
|
||||
int playbacks = ReplicateToDatastoreAction.replayAllTransactions(Optional.of(sqlSnapshotId));
|
||||
logger.atInfo().log("Played %s SQL transactions.", playbacks);
|
||||
|
||||
Optional<CommitLogCheckpoint> checkpoint = exportCommitLogs();
|
||||
if (!checkpoint.isPresent()) {
|
||||
throw new RuntimeException("Cannot create CommitLog checkpoint");
|
||||
}
|
||||
logger.atInfo().log(
|
||||
"CommitLog checkpoint created at %s.", checkpoint.get().getCheckpointTime());
|
||||
verifyCommitLogsPersisted(checkpoint.get());
|
||||
return checkpoint.get();
|
||||
}
|
||||
|
||||
private Optional<CommitLogCheckpoint> exportCommitLogs() {
|
||||
// Trigger an async CommitLog export to GCS. Will check file availability later.
|
||||
// Although we can add support to synchronous execution, it can disrupt the export cadence
|
||||
// when the system is busy
|
||||
Optional<CommitLogCheckpoint> checkpoint =
|
||||
commitLogCheckpointAction.createCheckPointAndStartAsyncExport();
|
||||
|
||||
// Failure to create checkpoint most likely caused by race with cron-triggered checkpointing.
|
||||
// Retry once.
|
||||
if (!checkpoint.isPresent()) {
|
||||
commitLogCheckpointAction.createCheckPointAndStartAsyncExport();
|
||||
}
|
||||
return checkpoint;
|
||||
}
|
||||
|
||||
private void verifyCommitLogsPersisted(CommitLogCheckpoint checkpoint) {
|
||||
DateTime exportStartTime =
|
||||
datastoreSnapshotFinder
|
||||
.getSnapshotInfo(checkpoint.getCheckpointTime().toInstant())
|
||||
.exportInterval()
|
||||
.getStart();
|
||||
logger.atInfo().log("Found Datastore export at %s", exportStartTime);
|
||||
for (int attempts = 0; attempts < COMMITLOGS_PRESENCE_CHECK_ATTEMPTS; attempts++) {
|
||||
try {
|
||||
gcsDiffFileLister.listDiffFiles(gcsBucket, exportStartTime, checkpoint.getCheckpointTime());
|
||||
return;
|
||||
} catch (IllegalStateException e) {
|
||||
// Gap in commitlog files. Fall through to sleep and retry.
|
||||
logger.atInfo().log("Commitlog files not yet found on GCS.");
|
||||
}
|
||||
sleeper.sleepInterruptibly(COMMITLOGS_PRESENCE_CHECK_DELAY);
|
||||
}
|
||||
throw new RuntimeException("Cannot find all commitlog files.");
|
||||
}
|
||||
}
|
|
@ -1,193 +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.comparedb;
|
||||
|
||||
import static google.registry.beam.comparedb.ValidateSqlUtils.createSqlEntityTupleTag;
|
||||
import static google.registry.beam.initsql.Transforms.createTagForKind;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Verify;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.backup.VersionedEntity;
|
||||
import google.registry.beam.initsql.Transforms;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.model.billing.BillingEvent;
|
||||
import google.registry.model.common.Cursor;
|
||||
import google.registry.model.contact.ContactHistory;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.domain.DomainHistory;
|
||||
import google.registry.model.domain.token.AllocationToken;
|
||||
import google.registry.model.host.HostHistory;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.model.poll.PollMessage;
|
||||
import google.registry.model.registrar.Registrar;
|
||||
import google.registry.model.registrar.RegistrarContact;
|
||||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.model.tld.Registry;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import javax.annotation.Nullable;
|
||||
import org.apache.beam.sdk.Pipeline;
|
||||
import org.apache.beam.sdk.transforms.DoFn;
|
||||
import org.apache.beam.sdk.transforms.ParDo;
|
||||
import org.apache.beam.sdk.values.PCollection;
|
||||
import org.apache.beam.sdk.values.PCollectionTuple;
|
||||
import org.apache.beam.sdk.values.TupleTag;
|
||||
import org.apache.beam.sdk.values.TupleTagList;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Utilities for loading Datastore snapshots. */
|
||||
@DeleteAfterMigration
|
||||
public final class DatastoreSnapshots {
|
||||
|
||||
private DatastoreSnapshots() {}
|
||||
|
||||
/**
|
||||
* Datastore kinds eligible for validation. This set must be consistent with {@link
|
||||
* SqlSnapshots#ALL_SQL_ENTITIES}.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static final ImmutableSet<Class<?>> ALL_DATASTORE_KINDS =
|
||||
ImmutableSet.of(
|
||||
Registry.class,
|
||||
Cursor.class,
|
||||
Registrar.class,
|
||||
ContactResource.class,
|
||||
RegistrarContact.class,
|
||||
HostResource.class,
|
||||
HistoryEntry.class,
|
||||
AllocationToken.class,
|
||||
BillingEvent.Recurring.class,
|
||||
BillingEvent.OneTime.class,
|
||||
BillingEvent.Cancellation.class,
|
||||
PollMessage.class,
|
||||
DomainBase.class);
|
||||
|
||||
/**
|
||||
* Returns the Datastore snapshot right before {@code commitLogToTime} for the user specified
|
||||
* {@code kinds}. The resulting snapshot has all changes that happened before {@code
|
||||
* commitLogToTime}, and none at or after {@code commitLogToTime}.
|
||||
*
|
||||
* <p>If {@code HistoryEntry} is included in {@code kinds}, the result will contain {@code
|
||||
* PCollections} for the child entities, {@code DomainHistory}, {@code ContactHistory}, and {@code
|
||||
* HostHistory}.
|
||||
*/
|
||||
static PCollectionTuple loadDatastoreSnapshotByKind(
|
||||
Pipeline pipeline,
|
||||
String exportDir,
|
||||
String commitLogDir,
|
||||
DateTime commitLogFromTime,
|
||||
DateTime commitLogToTime,
|
||||
Set<Class<?>> kinds,
|
||||
Optional<DateTime> compareStartTime) {
|
||||
PCollectionTuple snapshot =
|
||||
pipeline.apply(
|
||||
"Load Datastore snapshot.",
|
||||
Transforms.loadDatastoreSnapshot(
|
||||
exportDir,
|
||||
commitLogDir,
|
||||
commitLogFromTime,
|
||||
commitLogToTime,
|
||||
kinds.stream().map(Key::getKind).collect(ImmutableSet.toImmutableSet())));
|
||||
|
||||
PCollectionTuple perTypeSnapshots = PCollectionTuple.empty(pipeline);
|
||||
for (Class<?> kind : kinds) {
|
||||
PCollection<VersionedEntity> perKindSnapshot =
|
||||
snapshot.get(createTagForKind(Key.getKind(kind)));
|
||||
if (SqlEntity.class.isAssignableFrom(kind)) {
|
||||
perTypeSnapshots =
|
||||
perTypeSnapshots.and(
|
||||
createSqlEntityTupleTag((Class<? extends SqlEntity>) kind),
|
||||
datastoreEntityToPojo(perKindSnapshot, kind.getSimpleName(), compareStartTime));
|
||||
continue;
|
||||
}
|
||||
Verify.verify(kind == HistoryEntry.class, "Unexpected Non-SqlEntity class: %s", kind);
|
||||
PCollectionTuple historyEntriesByType = splitHistoryEntry(perKindSnapshot, compareStartTime);
|
||||
for (Map.Entry<TupleTag<?>, PCollection<?>> entry :
|
||||
historyEntriesByType.getAll().entrySet()) {
|
||||
perTypeSnapshots = perTypeSnapshots.and(entry.getKey().getId(), entry.getValue());
|
||||
}
|
||||
}
|
||||
return perTypeSnapshots;
|
||||
}
|
||||
|
||||
/**
|
||||
* Splits a {@link PCollection} of {@link HistoryEntry HistoryEntries} into three collections of
|
||||
* its child entities by type.
|
||||
*/
|
||||
static PCollectionTuple splitHistoryEntry(
|
||||
PCollection<VersionedEntity> historyEntries, Optional<DateTime> compareStartTime) {
|
||||
DateTime nullableStartTime = compareStartTime.orElse(null);
|
||||
return historyEntries.apply(
|
||||
"Split HistoryEntry by Resource Type",
|
||||
ParDo.of(
|
||||
new DoFn<VersionedEntity, SqlEntity>() {
|
||||
@ProcessElement
|
||||
public void processElement(
|
||||
@Element VersionedEntity historyEntry, MultiOutputReceiver out) {
|
||||
Optional.ofNullable(Transforms.convertVersionedEntityToSqlEntity(historyEntry))
|
||||
.filter(e -> isEntityIncludedForComparison(e, nullableStartTime))
|
||||
.ifPresent(
|
||||
sqlEntity ->
|
||||
out.get(createSqlEntityTupleTag(sqlEntity.getClass()))
|
||||
.output(sqlEntity));
|
||||
}
|
||||
})
|
||||
.withOutputTags(
|
||||
createSqlEntityTupleTag(DomainHistory.class),
|
||||
TupleTagList.of(createSqlEntityTupleTag(ContactHistory.class))
|
||||
.and(createSqlEntityTupleTag(HostHistory.class))));
|
||||
}
|
||||
|
||||
/**
|
||||
* Transforms a {@link PCollection} of {@link VersionedEntity VersionedEntities} to Ofy Java
|
||||
* objects.
|
||||
*/
|
||||
static PCollection<SqlEntity> datastoreEntityToPojo(
|
||||
PCollection<VersionedEntity> entities, String desc, Optional<DateTime> compareStartTime) {
|
||||
DateTime nullableStartTime = compareStartTime.orElse(null);
|
||||
return entities.apply(
|
||||
"Datastore Entity to Pojo " + desc,
|
||||
ParDo.of(
|
||||
new DoFn<VersionedEntity, SqlEntity>() {
|
||||
@ProcessElement
|
||||
public void processElement(
|
||||
@Element VersionedEntity entity, OutputReceiver<SqlEntity> out) {
|
||||
Optional.ofNullable(Transforms.convertVersionedEntityToSqlEntity(entity))
|
||||
.filter(e -> isEntityIncludedForComparison(e, nullableStartTime))
|
||||
.ifPresent(out::output);
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
static boolean isEntityIncludedForComparison(
|
||||
SqlEntity entity, @Nullable DateTime compareStartTime) {
|
||||
if (compareStartTime == null) {
|
||||
return true;
|
||||
}
|
||||
if (entity instanceof HistoryEntry) {
|
||||
return compareStartTime.isBefore(((HistoryEntry) entity).getModificationTime());
|
||||
}
|
||||
if (entity instanceof EppResource) {
|
||||
return compareStartTime.isBefore(((EppResource) entity).getUpdateTimestamp().getTimestamp());
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -1,154 +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.comparedb;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.cloud.storage.BlobId;
|
||||
import com.google.cloud.storage.BlobInfo;
|
||||
import dagger.Component;
|
||||
import google.registry.config.CloudTasksUtilsModule;
|
||||
import google.registry.config.CredentialModule;
|
||||
import google.registry.config.RegistryConfig;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.config.RegistryConfig.ConfigModule;
|
||||
import google.registry.gcs.GcsUtils;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.util.Clock;
|
||||
import google.registry.util.UtilsModule;
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Optional;
|
||||
import javax.inject.Inject;
|
||||
import javax.inject.Singleton;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Instant;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/** Finds the necessary information for loading the most recent Datastore snapshot. */
|
||||
@DeleteAfterMigration
|
||||
public class LatestDatastoreSnapshotFinder {
|
||||
private final String projectId;
|
||||
private final GcsUtils gcsUtils;
|
||||
private final Clock clock;
|
||||
|
||||
@Inject
|
||||
LatestDatastoreSnapshotFinder(
|
||||
@Config("projectId") String projectId, GcsUtils gcsUtils, Clock clock) {
|
||||
this.projectId = projectId;
|
||||
this.gcsUtils = gcsUtils;
|
||||
this.clock = clock;
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds information of the most recent Datastore snapshot that ends strictly before {@code
|
||||
* exportEndTimeUpperBound}, including the GCS folder of the exported data files and the start and
|
||||
* stop times of the export. The folder of the CommitLogs is also included in the return.
|
||||
*/
|
||||
public DatastoreSnapshotInfo getSnapshotInfo(Instant exportEndTimeUpperBound) {
|
||||
String bucketName = RegistryConfig.getDatastoreBackupsBucket().substring("gs://".length());
|
||||
/**
|
||||
* Find the bucket-relative path to the overall metadata file of the last Datastore export.
|
||||
* Since Datastore export is saved daily, we may need to look back to yesterday. If found, the
|
||||
* return value is like
|
||||
* "2021-11-19T06:00:00_76493/2021-11-19T06:00:00_76493.overall_export_metadata".
|
||||
*/
|
||||
Optional<String> metaFilePathOptional =
|
||||
findNewestExportMetadataFileBeforeTime(bucketName, exportEndTimeUpperBound, 5);
|
||||
if (!metaFilePathOptional.isPresent()) {
|
||||
throw new NoSuchElementException("No exports found over the past 2 days.");
|
||||
}
|
||||
String metaFilePath = metaFilePathOptional.get();
|
||||
String metaFileFolder = metaFilePath.substring(0, metaFilePath.indexOf('/'));
|
||||
Instant exportStartTime = Instant.parse(metaFileFolder.replace('_', '.') + 'Z');
|
||||
BlobInfo blobInfo = gcsUtils.getBlobInfo(BlobId.of(bucketName, metaFilePath));
|
||||
Instant exportEndTime = new Instant(blobInfo.getCreateTime());
|
||||
return DatastoreSnapshotInfo.create(
|
||||
String.format("gs://%s/%s", bucketName, metaFileFolder),
|
||||
getCommitLogDir(),
|
||||
new Interval(exportStartTime, exportEndTime));
|
||||
}
|
||||
|
||||
public String getCommitLogDir() {
|
||||
return "gs://" + projectId + "-commits";
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the latest Datastore export that ends strictly before {@code endTimeUpperBound} and
|
||||
* returns the bucket-relative path of the overall export metadata file, in the given bucket. The
|
||||
* search goes back for up to {@code lookBackDays} days in time, including today.
|
||||
*
|
||||
* <p>The overall export metadata file is the last file created during a Datastore export. All
|
||||
* data has been exported by the creation time of this file. The name of this file, like that of
|
||||
* all files in the same export, begins with the timestamp when the export starts.
|
||||
*
|
||||
* <p>An example return value: {@code
|
||||
* 2021-11-19T06:00:00_76493/2021-11-19T06:00:00_76493.overall_export_metadata}.
|
||||
*/
|
||||
private Optional<String> findNewestExportMetadataFileBeforeTime(
|
||||
String bucketName, Instant endTimeUpperBound, int lookBackDays) {
|
||||
DateTime today = clock.nowUtc();
|
||||
for (int day = 0; day < lookBackDays; day++) {
|
||||
String dateString = today.minusDays(day).toString("yyyy-MM-dd");
|
||||
try {
|
||||
Optional<String> metaFilePath =
|
||||
gcsUtils.listFolderObjects(bucketName, dateString).stream()
|
||||
.filter(s -> s.endsWith("overall_export_metadata"))
|
||||
.map(s -> dateString + s)
|
||||
.sorted(Comparator.<String>naturalOrder().reversed())
|
||||
.findFirst();
|
||||
if (metaFilePath.isPresent()) {
|
||||
BlobInfo blobInfo = gcsUtils.getBlobInfo(BlobId.of(bucketName, metaFilePath.get()));
|
||||
Instant exportEndTime = new Instant(blobInfo.getCreateTime());
|
||||
if (exportEndTime.isBefore(endTimeUpperBound)) {
|
||||
return metaFilePath;
|
||||
}
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
/** Holds information about a Datastore snapshot. */
|
||||
@AutoValue
|
||||
public abstract static class DatastoreSnapshotInfo {
|
||||
public abstract String exportDir();
|
||||
|
||||
public abstract String commitLogDir();
|
||||
|
||||
public abstract Interval exportInterval();
|
||||
|
||||
static DatastoreSnapshotInfo create(
|
||||
String exportDir, String commitLogDir, Interval exportOperationInterval) {
|
||||
return new AutoValue_LatestDatastoreSnapshotFinder_DatastoreSnapshotInfo(
|
||||
exportDir, commitLogDir, exportOperationInterval);
|
||||
}
|
||||
}
|
||||
|
||||
@Singleton
|
||||
@Component(
|
||||
modules = {
|
||||
CredentialModule.class,
|
||||
ConfigModule.class,
|
||||
CloudTasksUtilsModule.class,
|
||||
UtilsModule.class
|
||||
})
|
||||
interface LatestDatastoreSnapshotFinderFinderComponent {
|
||||
|
||||
LatestDatastoreSnapshotFinder datastoreSnapshotInfoFinder();
|
||||
}
|
||||
}
|
|
@ -1,539 +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.comparedb;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkState;
|
||||
import static google.registry.beam.comparedb.ValidateSqlUtils.createSqlEntityTupleTag;
|
||||
import static google.registry.beam.comparedb.ValidateSqlUtils.getMedianIdForHistoryTable;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.base.Verify;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.ImmutableSetMultimap;
|
||||
import com.google.common.collect.Streams;
|
||||
import google.registry.beam.common.RegistryJpaIO;
|
||||
import google.registry.beam.common.RegistryJpaIO.Read;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.UpdateAutoTimestamp;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.model.billing.BillingEvent;
|
||||
import google.registry.model.bulkquery.BulkQueryEntities;
|
||||
import google.registry.model.bulkquery.DomainBaseLite;
|
||||
import google.registry.model.bulkquery.DomainHistoryHost;
|
||||
import google.registry.model.bulkquery.DomainHistoryLite;
|
||||
import google.registry.model.bulkquery.DomainHost;
|
||||
import google.registry.model.common.Cursor;
|
||||
import google.registry.model.contact.ContactHistory;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.domain.DomainHistory;
|
||||
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
||||
import google.registry.model.domain.GracePeriod;
|
||||
import google.registry.model.domain.GracePeriod.GracePeriodHistory;
|
||||
import google.registry.model.domain.secdns.DelegationSignerData;
|
||||
import google.registry.model.domain.secdns.DomainDsDataHistory;
|
||||
import google.registry.model.domain.token.AllocationToken;
|
||||
import google.registry.model.host.HostHistory;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.model.poll.PollMessage;
|
||||
import google.registry.model.registrar.Registrar;
|
||||
import google.registry.model.registrar.RegistrarContact;
|
||||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.model.reporting.DomainTransactionRecord;
|
||||
import google.registry.model.tld.Registry;
|
||||
import google.registry.persistence.transaction.CriteriaQueryBuilder;
|
||||
import google.registry.util.DateTimeUtils;
|
||||
import java.io.Serializable;
|
||||
import java.util.Optional;
|
||||
import javax.persistence.Entity;
|
||||
import org.apache.beam.sdk.Pipeline;
|
||||
import org.apache.beam.sdk.transforms.DoFn;
|
||||
import org.apache.beam.sdk.transforms.Flatten;
|
||||
import org.apache.beam.sdk.transforms.GroupByKey;
|
||||
import org.apache.beam.sdk.transforms.MapElements;
|
||||
import org.apache.beam.sdk.transforms.ParDo;
|
||||
import org.apache.beam.sdk.transforms.SerializableFunction;
|
||||
import org.apache.beam.sdk.values.KV;
|
||||
import org.apache.beam.sdk.values.PCollection;
|
||||
import org.apache.beam.sdk.values.PCollectionList;
|
||||
import org.apache.beam.sdk.values.PCollectionTuple;
|
||||
import org.apache.beam.sdk.values.TypeDescriptor;
|
||||
import org.apache.beam.sdk.values.TypeDescriptors;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/**
|
||||
* Utilities for loading SQL snapshots.
|
||||
*
|
||||
* <p>For {@link DomainBase} and {@link DomainHistory}, this class assumes the presence of the
|
||||
* {@link google.registry.persistence.PersistenceModule.JpaTransactionManagerType#BULK_QUERY
|
||||
* bulk-query-capable JpaTransactionManager}, and takes advantage of it for higher throughput.
|
||||
*
|
||||
* <p>For now this class is meant for use during the database migration period only. Therefore, it
|
||||
* contains optimizations specifically for the production database at the current size, e.g.,
|
||||
* parallel queries for select tables.
|
||||
*/
|
||||
@DeleteAfterMigration
|
||||
public final class SqlSnapshots {
|
||||
|
||||
private SqlSnapshots() {}
|
||||
|
||||
/**
|
||||
* SQL entity types that are eligible for validation. This set must be consistent with {@link
|
||||
* DatastoreSnapshots#ALL_DATASTORE_KINDS}.
|
||||
*/
|
||||
static final ImmutableSet<Class<? extends SqlEntity>> ALL_SQL_ENTITIES =
|
||||
ImmutableSet.of(
|
||||
Registry.class,
|
||||
Cursor.class,
|
||||
Registrar.class,
|
||||
ContactResource.class,
|
||||
RegistrarContact.class,
|
||||
HostResource.class,
|
||||
AllocationToken.class,
|
||||
BillingEvent.Recurring.class,
|
||||
BillingEvent.OneTime.class,
|
||||
BillingEvent.Cancellation.class,
|
||||
PollMessage.class,
|
||||
DomainBase.class,
|
||||
ContactHistory.class,
|
||||
HostHistory.class,
|
||||
DomainHistory.class);
|
||||
|
||||
/**
|
||||
* Loads a SQL snapshot for the given {@code sqlEntityTypes}.
|
||||
*
|
||||
* <p>If {@code snapshotId} is present, all queries use the specified database snapshot,
|
||||
* guaranteeing a consistent result.
|
||||
*/
|
||||
public static PCollectionTuple loadCloudSqlSnapshotByType(
|
||||
Pipeline pipeline,
|
||||
ImmutableSet<Class<? extends SqlEntity>> sqlEntityTypes,
|
||||
Optional<String> snapshotId,
|
||||
Optional<DateTime> compareStartTime) {
|
||||
PCollectionTuple perTypeSnapshots = PCollectionTuple.empty(pipeline);
|
||||
for (Class<? extends SqlEntity> clazz : sqlEntityTypes) {
|
||||
if (clazz == DomainBase.class) {
|
||||
perTypeSnapshots =
|
||||
perTypeSnapshots.and(
|
||||
createSqlEntityTupleTag(DomainBase.class),
|
||||
loadAndAssembleDomainBase(pipeline, snapshotId, compareStartTime));
|
||||
continue;
|
||||
}
|
||||
if (clazz == DomainHistory.class) {
|
||||
perTypeSnapshots =
|
||||
perTypeSnapshots.and(
|
||||
createSqlEntityTupleTag(DomainHistory.class),
|
||||
loadAndAssembleDomainHistory(pipeline, snapshotId, compareStartTime));
|
||||
continue;
|
||||
}
|
||||
if (clazz == ContactHistory.class) {
|
||||
perTypeSnapshots =
|
||||
perTypeSnapshots.and(
|
||||
createSqlEntityTupleTag(ContactHistory.class),
|
||||
loadContactHistory(pipeline, snapshotId, compareStartTime));
|
||||
continue;
|
||||
}
|
||||
if (clazz == HostHistory.class) {
|
||||
perTypeSnapshots =
|
||||
perTypeSnapshots.and(
|
||||
createSqlEntityTupleTag(HostHistory.class),
|
||||
loadHostHistory(
|
||||
pipeline, snapshotId, compareStartTime.orElse(DateTimeUtils.START_OF_TIME)));
|
||||
continue;
|
||||
}
|
||||
if (EppResource.class.isAssignableFrom(clazz) && compareStartTime.isPresent()) {
|
||||
perTypeSnapshots =
|
||||
perTypeSnapshots.and(
|
||||
createSqlEntityTupleTag(clazz),
|
||||
pipeline.apply(
|
||||
"SQL Load " + clazz.getSimpleName(),
|
||||
buildEppResourceQueryWithTimeFilter(
|
||||
clazz, SqlEntity.class, snapshotId, compareStartTime.get())
|
||||
.withSnapshot(snapshotId.orElse(null))));
|
||||
continue;
|
||||
}
|
||||
perTypeSnapshots =
|
||||
perTypeSnapshots.and(
|
||||
createSqlEntityTupleTag(clazz),
|
||||
pipeline.apply(
|
||||
"SQL Load " + clazz.getSimpleName(),
|
||||
RegistryJpaIO.read(
|
||||
() -> CriteriaQueryBuilder.create(clazz).build(), SqlEntity.class::cast)
|
||||
.withSnapshot(snapshotId.orElse(null))));
|
||||
}
|
||||
return perTypeSnapshots;
|
||||
}
|
||||
|
||||
/**
|
||||
* Bulk-loads parts of {@link DomainBase} and assembles them in the pipeline.
|
||||
*
|
||||
* @see BulkQueryEntities
|
||||
*/
|
||||
public static PCollection<SqlEntity> loadAndAssembleDomainBase(
|
||||
Pipeline pipeline, Optional<String> snapshotId, Optional<DateTime> compareStartTime) {
|
||||
PCollection<KV<String, Serializable>> baseObjects =
|
||||
readAllAndAssignKey(
|
||||
pipeline,
|
||||
DomainBaseLite.class,
|
||||
DomainBaseLite::getRepoId,
|
||||
snapshotId,
|
||||
compareStartTime);
|
||||
PCollection<KV<String, Serializable>> gracePeriods =
|
||||
readAllAndAssignKey(
|
||||
pipeline,
|
||||
GracePeriod.class,
|
||||
GracePeriod::getDomainRepoId,
|
||||
snapshotId,
|
||||
compareStartTime);
|
||||
PCollection<KV<String, Serializable>> delegationSigners =
|
||||
readAllAndAssignKey(
|
||||
pipeline,
|
||||
DelegationSignerData.class,
|
||||
DelegationSignerData::getDomainRepoId,
|
||||
snapshotId,
|
||||
compareStartTime);
|
||||
PCollection<KV<String, Serializable>> domainHosts =
|
||||
readAllAndAssignKey(
|
||||
pipeline, DomainHost.class, DomainHost::getDomainRepoId, snapshotId, compareStartTime);
|
||||
|
||||
DateTime nullableCompareStartTime = compareStartTime.orElse(null);
|
||||
return PCollectionList.of(
|
||||
ImmutableList.of(baseObjects, gracePeriods, delegationSigners, domainHosts))
|
||||
.apply("SQL Merge DomainBase parts", Flatten.pCollections())
|
||||
.apply("Group by Domain Parts by RepoId", GroupByKey.create())
|
||||
.apply(
|
||||
"Assemble DomainBase",
|
||||
ParDo.of(
|
||||
new DoFn<KV<String, Iterable<Serializable>>, SqlEntity>() {
|
||||
@ProcessElement
|
||||
public void processElement(
|
||||
@Element KV<String, Iterable<Serializable>> kv,
|
||||
OutputReceiver<SqlEntity> outputReceiver) {
|
||||
TypedClassifier partsByType = new TypedClassifier(kv.getValue());
|
||||
ImmutableSet<DomainBaseLite> baseObjects =
|
||||
partsByType.getAllOf(DomainBaseLite.class);
|
||||
if (nullableCompareStartTime != null) {
|
||||
Verify.verify(
|
||||
baseObjects.size() <= 1,
|
||||
"Found duplicate DomainBaseLite object per repoId: " + kv.getKey());
|
||||
if (baseObjects.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
Verify.verify(
|
||||
baseObjects.size() == 1,
|
||||
"Expecting one DomainBaseLite object per repoId: " + kv.getKey());
|
||||
outputReceiver.output(
|
||||
BulkQueryEntities.assembleDomainBase(
|
||||
baseObjects.iterator().next(),
|
||||
partsByType.getAllOf(GracePeriod.class),
|
||||
partsByType.getAllOf(DelegationSignerData.class),
|
||||
partsByType.getAllOf(DomainHost.class).stream()
|
||||
.map(DomainHost::getHostVKey)
|
||||
.collect(ImmutableSet.toImmutableSet())));
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads all {@link ContactHistory} entities from the database.
|
||||
*
|
||||
* <p>This method uses two queries to load data in parallel. This is a performance optimization
|
||||
* specifically for the production database.
|
||||
*/
|
||||
static PCollection<SqlEntity> loadContactHistory(
|
||||
Pipeline pipeline, Optional<String> snapshotId, Optional<DateTime> compareStartTime) {
|
||||
PartitionedQuery partitionedQuery =
|
||||
buildPartitonedHistoryQuery(ContactHistory.class, compareStartTime);
|
||||
PCollection<SqlEntity> part1 =
|
||||
pipeline.apply(
|
||||
"SQL Load ContactHistory first half",
|
||||
RegistryJpaIO.read(
|
||||
partitionedQuery.firstHalfQuery(),
|
||||
partitionedQuery.parameters(),
|
||||
false,
|
||||
SqlEntity.class::cast)
|
||||
.withSnapshot(snapshotId.orElse(null)));
|
||||
PCollection<SqlEntity> part2 =
|
||||
pipeline.apply(
|
||||
"SQL Load ContactHistory second half",
|
||||
RegistryJpaIO.read(
|
||||
partitionedQuery.secondHalfQuery(),
|
||||
partitionedQuery.parameters(),
|
||||
false,
|
||||
SqlEntity.class::cast)
|
||||
.withSnapshot(snapshotId.orElse(null)));
|
||||
return PCollectionList.of(part1)
|
||||
.and(part2)
|
||||
.apply("Combine ContactHistory parts", Flatten.pCollections());
|
||||
}
|
||||
|
||||
/** Loads all {@link HostHistory} entities from the database. */
|
||||
static PCollection<SqlEntity> loadHostHistory(
|
||||
Pipeline pipeline, Optional<String> snapshotId, DateTime compareStartTime) {
|
||||
return pipeline.apply(
|
||||
"SQL Load HostHistory",
|
||||
RegistryJpaIO.read(
|
||||
"select c from HostHistory c where :compareStartTime < modificationTime",
|
||||
ImmutableMap.of("compareStartTime", compareStartTime),
|
||||
false,
|
||||
SqlEntity.class::cast)
|
||||
.withSnapshot(snapshotId.orElse(null)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Bulk-loads all parts of {@link DomainHistory} and assembles them in the pipeline.
|
||||
*
|
||||
* <p>This method uses two queries to load {@link DomainBaseLite} in parallel. This is a
|
||||
* performance optimization specifically for the production database.
|
||||
*
|
||||
* @see BulkQueryEntities
|
||||
*/
|
||||
static PCollection<SqlEntity> loadAndAssembleDomainHistory(
|
||||
Pipeline pipeline, Optional<String> snapshotId, Optional<DateTime> compareStartTime) {
|
||||
PartitionedQuery partitionedQuery =
|
||||
buildPartitonedHistoryQuery(DomainHistoryLite.class, compareStartTime);
|
||||
PCollection<KV<String, Serializable>> baseObjectsPart1 =
|
||||
queryAndAssignKey(
|
||||
pipeline,
|
||||
"first half",
|
||||
partitionedQuery.firstHalfQuery(),
|
||||
partitionedQuery.parameters(),
|
||||
DomainHistoryLite.class,
|
||||
compose(DomainHistoryLite::getDomainHistoryId, DomainHistoryId::toString),
|
||||
snapshotId);
|
||||
PCollection<KV<String, Serializable>> baseObjectsPart2 =
|
||||
queryAndAssignKey(
|
||||
pipeline,
|
||||
"second half",
|
||||
partitionedQuery.secondHalfQuery(),
|
||||
partitionedQuery.parameters(),
|
||||
DomainHistoryLite.class,
|
||||
compose(DomainHistoryLite::getDomainHistoryId, DomainHistoryId::toString),
|
||||
snapshotId);
|
||||
PCollection<KV<String, Serializable>> gracePeriods =
|
||||
readAllAndAssignKey(
|
||||
pipeline,
|
||||
GracePeriodHistory.class,
|
||||
compose(GracePeriodHistory::getDomainHistoryId, DomainHistoryId::toString),
|
||||
snapshotId,
|
||||
compareStartTime);
|
||||
PCollection<KV<String, Serializable>> delegationSigners =
|
||||
readAllAndAssignKey(
|
||||
pipeline,
|
||||
DomainDsDataHistory.class,
|
||||
compose(DomainDsDataHistory::getDomainHistoryId, DomainHistoryId::toString),
|
||||
snapshotId,
|
||||
compareStartTime);
|
||||
PCollection<KV<String, Serializable>> domainHosts =
|
||||
readAllAndAssignKey(
|
||||
pipeline,
|
||||
DomainHistoryHost.class,
|
||||
compose(DomainHistoryHost::getDomainHistoryId, DomainHistoryId::toString),
|
||||
snapshotId,
|
||||
compareStartTime);
|
||||
PCollection<KV<String, Serializable>> transactionRecords =
|
||||
readAllAndAssignKey(
|
||||
pipeline,
|
||||
DomainTransactionRecord.class,
|
||||
compose(DomainTransactionRecord::getDomainHistoryId, DomainHistoryId::toString),
|
||||
snapshotId,
|
||||
compareStartTime);
|
||||
|
||||
DateTime nullableCompareStartTime = compareStartTime.orElse(null);
|
||||
return PCollectionList.of(
|
||||
ImmutableList.of(
|
||||
baseObjectsPart1,
|
||||
baseObjectsPart2,
|
||||
gracePeriods,
|
||||
delegationSigners,
|
||||
domainHosts,
|
||||
transactionRecords))
|
||||
.apply("Merge DomainHistory parts", Flatten.pCollections())
|
||||
.apply("Group by DomainHistory Parts by DomainHistoryId string", GroupByKey.create())
|
||||
.apply(
|
||||
"Assemble DomainHistory",
|
||||
ParDo.of(
|
||||
new DoFn<KV<String, Iterable<Serializable>>, SqlEntity>() {
|
||||
@ProcessElement
|
||||
public void processElement(
|
||||
@Element KV<String, Iterable<Serializable>> kv,
|
||||
OutputReceiver<SqlEntity> outputReceiver) {
|
||||
TypedClassifier partsByType = new TypedClassifier(kv.getValue());
|
||||
ImmutableSet<DomainHistoryLite> baseObjects =
|
||||
partsByType.getAllOf(DomainHistoryLite.class);
|
||||
if (nullableCompareStartTime != null) {
|
||||
Verify.verify(
|
||||
baseObjects.size() <= 1,
|
||||
"Found duplicate DomainHistoryLite object per domainHistoryId: "
|
||||
+ kv.getKey());
|
||||
if (baseObjects.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
Verify.verify(
|
||||
baseObjects.size() == 1,
|
||||
"Expecting one DomainHistoryLite object per domainHistoryId: "
|
||||
+ kv.getKey());
|
||||
outputReceiver.output(
|
||||
BulkQueryEntities.assembleDomainHistory(
|
||||
baseObjects.iterator().next(),
|
||||
partsByType.getAllOf(DomainDsDataHistory.class),
|
||||
partsByType.getAllOf(DomainHistoryHost.class).stream()
|
||||
.map(DomainHistoryHost::getHostVKey)
|
||||
.collect(ImmutableSet.toImmutableSet()),
|
||||
partsByType.getAllOf(GracePeriodHistory.class),
|
||||
partsByType.getAllOf(DomainTransactionRecord.class)));
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
static <R, T> PCollection<KV<String, Serializable>> readAllAndAssignKey(
|
||||
Pipeline pipeline,
|
||||
Class<R> type,
|
||||
SerializableFunction<R, String> keyFunction,
|
||||
Optional<String> snapshotId,
|
||||
Optional<DateTime> compareStartTime) {
|
||||
Read<R, R> queryObject;
|
||||
if (compareStartTime.isPresent() && EppResource.class.isAssignableFrom(type)) {
|
||||
queryObject =
|
||||
buildEppResourceQueryWithTimeFilter(type, type, snapshotId, compareStartTime.get());
|
||||
} else {
|
||||
queryObject =
|
||||
RegistryJpaIO.read(() -> CriteriaQueryBuilder.create(type).build())
|
||||
.withSnapshot(snapshotId.orElse(null));
|
||||
}
|
||||
return pipeline
|
||||
.apply("SQL Load " + type.getSimpleName(), queryObject)
|
||||
.apply(
|
||||
"Assign Key to " + type.getSimpleName(),
|
||||
MapElements.into(
|
||||
TypeDescriptors.kvs(
|
||||
TypeDescriptors.strings(), TypeDescriptor.of(Serializable.class)))
|
||||
.via(obj -> KV.of(keyFunction.apply(obj), (Serializable) obj)));
|
||||
}
|
||||
|
||||
static <R, T> PCollection<KV<String, Serializable>> queryAndAssignKey(
|
||||
Pipeline pipeline,
|
||||
String diffrentiator,
|
||||
String jplQuery,
|
||||
ImmutableMap<String, Object> queryParameters,
|
||||
Class<R> type,
|
||||
SerializableFunction<R, String> keyFunction,
|
||||
Optional<String> snapshotId) {
|
||||
return pipeline
|
||||
.apply(
|
||||
"SQL Load " + type.getSimpleName() + " " + diffrentiator,
|
||||
RegistryJpaIO.read(jplQuery, queryParameters, false, type::cast)
|
||||
.withSnapshot(snapshotId.orElse(null)))
|
||||
.apply(
|
||||
"Assign Key to " + type.getSimpleName() + " " + diffrentiator,
|
||||
MapElements.into(
|
||||
TypeDescriptors.kvs(
|
||||
TypeDescriptors.strings(), TypeDescriptor.of(Serializable.class)))
|
||||
.via(obj -> KV.of(keyFunction.apply(obj), (Serializable) obj)));
|
||||
}
|
||||
|
||||
// TODO(b/205988530): don't use beam serializablefunction, make one that extends Java's Function.
|
||||
private static <R, I, T> SerializableFunction<R, T> compose(
|
||||
SerializableFunction<R, I> f1, SerializableFunction<I, T> f2) {
|
||||
return r -> f2.apply(f1.apply(r));
|
||||
}
|
||||
|
||||
static <R, T> Read<R, T> buildEppResourceQueryWithTimeFilter(
|
||||
Class<R> entityType,
|
||||
Class<T> castOutputAsType,
|
||||
Optional<String> snapshotId,
|
||||
DateTime compareStartTime) {
|
||||
String tableName = getJpaEntityName(entityType);
|
||||
String jpql =
|
||||
String.format("select c from %s c where :compareStartTime < updateTimestamp", tableName);
|
||||
return RegistryJpaIO.read(
|
||||
jpql,
|
||||
ImmutableMap.of("compareStartTime", UpdateAutoTimestamp.create(compareStartTime)),
|
||||
false,
|
||||
(R x) -> castOutputAsType.cast(x))
|
||||
.withSnapshot(snapshotId.orElse(null));
|
||||
}
|
||||
|
||||
static PartitionedQuery buildPartitonedHistoryQuery(
|
||||
Class<?> entityType, Optional<DateTime> compareStartTime) {
|
||||
String tableName = getJpaEntityName(entityType);
|
||||
Verify.verify(
|
||||
!Strings.isNullOrEmpty(tableName), "Invalid entity type %s", entityType.getSimpleName());
|
||||
long medianId =
|
||||
getMedianIdForHistoryTable(tableName)
|
||||
.orElseThrow(() -> new IllegalStateException("Not a valid database: no " + tableName));
|
||||
String firstHalfQuery = String.format("select c from %s c where id <= :historyId", tableName);
|
||||
String secondHalfQuery = String.format("select c from %s c where id > :historyId", tableName);
|
||||
if (compareStartTime.isPresent()) {
|
||||
String timeFilter = " and :compareStartTime < modificationTime";
|
||||
firstHalfQuery += timeFilter;
|
||||
secondHalfQuery += timeFilter;
|
||||
return PartitionedQuery.createPartitionedQuery(
|
||||
firstHalfQuery,
|
||||
secondHalfQuery,
|
||||
ImmutableMap.of("historyId", medianId, "compareStartTime", compareStartTime.get()));
|
||||
} else {
|
||||
return PartitionedQuery.createPartitionedQuery(
|
||||
firstHalfQuery, secondHalfQuery, ImmutableMap.of("historyId", medianId));
|
||||
}
|
||||
}
|
||||
|
||||
private static String getJpaEntityName(Class entityType) {
|
||||
Entity entityAnnotation = (Entity) entityType.getAnnotation(Entity.class);
|
||||
checkState(
|
||||
entityAnnotation != null, "Unexpected non-entity type %s", entityType.getSimpleName());
|
||||
return Strings.isNullOrEmpty(entityAnnotation.name())
|
||||
? entityType.getSimpleName()
|
||||
: entityAnnotation.name();
|
||||
}
|
||||
|
||||
/** Contains two queries that partition the target table in two. */
|
||||
@AutoValue
|
||||
abstract static class PartitionedQuery {
|
||||
abstract String firstHalfQuery();
|
||||
|
||||
abstract String secondHalfQuery();
|
||||
|
||||
abstract ImmutableMap<String, Object> parameters();
|
||||
|
||||
public static PartitionedQuery createPartitionedQuery(
|
||||
String firstHalfQuery, String secondHalfQuery, ImmutableMap<String, Object> parameters) {
|
||||
return new AutoValue_SqlSnapshots_PartitionedQuery(
|
||||
firstHalfQuery, secondHalfQuery, parameters);
|
||||
}
|
||||
}
|
||||
|
||||
/** Container that receives mixed-typed data and groups them by {@link Class}. */
|
||||
static class TypedClassifier {
|
||||
private final ImmutableSetMultimap<Class<?>, Object> classifiedEntities;
|
||||
|
||||
TypedClassifier(Iterable<Serializable> inputs) {
|
||||
this.classifiedEntities =
|
||||
Streams.stream(inputs)
|
||||
.collect(ImmutableSetMultimap.toImmutableSetMultimap(Object::getClass, x -> x));
|
||||
}
|
||||
|
||||
<T> ImmutableSet<T> getAllOf(Class<T> clazz) {
|
||||
return classifiedEntities.get(clazz).stream()
|
||||
.map(clazz::cast)
|
||||
.collect(ImmutableSet.toImmutableSet());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,206 +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.comparedb;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static org.apache.beam.sdk.values.TypeDescriptors.strings;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import google.registry.beam.common.RegistryPipelineOptions;
|
||||
import google.registry.beam.common.RegistryPipelineWorkerInitializer;
|
||||
import google.registry.beam.comparedb.LatestDatastoreSnapshotFinder.DatastoreSnapshotInfo;
|
||||
import google.registry.beam.comparedb.ValidateSqlUtils.CompareSqlEntity;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.domain.DomainHistory;
|
||||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.persistence.PersistenceModule.JpaTransactionManagerType;
|
||||
import google.registry.persistence.PersistenceModule.TransactionIsolationLevel;
|
||||
import google.registry.util.SystemClock;
|
||||
import java.io.Serializable;
|
||||
import java.util.Optional;
|
||||
import org.apache.beam.sdk.Pipeline;
|
||||
import org.apache.beam.sdk.coders.SerializableCoder;
|
||||
import org.apache.beam.sdk.io.TextIO;
|
||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||
import org.apache.beam.sdk.transforms.Flatten;
|
||||
import org.apache.beam.sdk.transforms.GroupByKey;
|
||||
import org.apache.beam.sdk.transforms.ParDo;
|
||||
import org.apache.beam.sdk.transforms.WithKeys;
|
||||
import org.apache.beam.sdk.values.PCollectionList;
|
||||
import org.apache.beam.sdk.values.PCollectionTuple;
|
||||
import org.apache.beam.sdk.values.TupleTag;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/**
|
||||
* Validates the asynchronous data replication process between Datastore and Cloud SQL.
|
||||
*
|
||||
* <p>This pipeline is to be launched by {@link google.registry.tools.ValidateDatastoreCommand} or
|
||||
* {@link google.registry.tools.ValidateSqlCommand}.
|
||||
*/
|
||||
@DeleteAfterMigration
|
||||
public class ValidateDatabasePipeline {
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
/** Specifies the extra CommitLogs to load before the start of a Database export. */
|
||||
private static final Duration COMMITLOG_START_TIME_MARGIN = Duration.standardMinutes(10);
|
||||
|
||||
private final ValidateDatabasePipelineOptions options;
|
||||
private final LatestDatastoreSnapshotFinder datastoreSnapshotFinder;
|
||||
|
||||
public ValidateDatabasePipeline(
|
||||
ValidateDatabasePipelineOptions options,
|
||||
LatestDatastoreSnapshotFinder datastoreSnapshotFinder) {
|
||||
this.options = options;
|
||||
this.datastoreSnapshotFinder = datastoreSnapshotFinder;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void run(Pipeline pipeline) {
|
||||
DateTime latestCommitLogTime = DateTime.parse(options.getLatestCommitLogTimestamp());
|
||||
DatastoreSnapshotInfo mostRecentExport =
|
||||
datastoreSnapshotFinder.getSnapshotInfo(latestCommitLogTime.toInstant());
|
||||
|
||||
logger.atInfo().log(
|
||||
"Comparing datastore export at %s and commitlog timestamp %s.",
|
||||
mostRecentExport.exportDir(), latestCommitLogTime);
|
||||
|
||||
Optional<String> outputPath =
|
||||
Optional.ofNullable(options.getDiffOutputGcsBucket())
|
||||
.map(
|
||||
bucket ->
|
||||
String.format(
|
||||
"gs://%s/validate_database/%s/diffs.txt",
|
||||
bucket, new SystemClock().nowUtc()));
|
||||
outputPath.ifPresent(path -> logger.atInfo().log("Discrepancies will be logged to %s", path));
|
||||
|
||||
setupPipeline(
|
||||
pipeline,
|
||||
Optional.ofNullable(options.getSqlSnapshotId()),
|
||||
mostRecentExport,
|
||||
latestCommitLogTime,
|
||||
Optional.ofNullable(options.getComparisonStartTimestamp()).map(DateTime::parse),
|
||||
outputPath);
|
||||
|
||||
pipeline.run();
|
||||
}
|
||||
|
||||
static void setupPipeline(
|
||||
Pipeline pipeline,
|
||||
Optional<String> sqlSnapshotId,
|
||||
DatastoreSnapshotInfo mostRecentExport,
|
||||
DateTime latestCommitLogTime,
|
||||
Optional<DateTime> compareStartTime,
|
||||
Optional<String> diffOutputPath) {
|
||||
pipeline
|
||||
.getCoderRegistry()
|
||||
.registerCoderForClass(SqlEntity.class, SerializableCoder.of(Serializable.class));
|
||||
|
||||
PCollectionTuple datastoreSnapshot =
|
||||
DatastoreSnapshots.loadDatastoreSnapshotByKind(
|
||||
pipeline,
|
||||
mostRecentExport.exportDir(),
|
||||
mostRecentExport.commitLogDir(),
|
||||
mostRecentExport.exportInterval().getStart().minus(COMMITLOG_START_TIME_MARGIN),
|
||||
// Increase by 1ms since we want to include commitLogs latestCommitLogTime but
|
||||
// this parameter is exclusive.
|
||||
latestCommitLogTime.plusMillis(1),
|
||||
DatastoreSnapshots.ALL_DATASTORE_KINDS,
|
||||
compareStartTime);
|
||||
|
||||
PCollectionTuple cloudSqlSnapshot =
|
||||
SqlSnapshots.loadCloudSqlSnapshotByType(
|
||||
pipeline, SqlSnapshots.ALL_SQL_ENTITIES, sqlSnapshotId, compareStartTime);
|
||||
|
||||
verify(
|
||||
datastoreSnapshot.getAll().keySet().equals(cloudSqlSnapshot.getAll().keySet()),
|
||||
"Expecting the same set of types in both snapshots.");
|
||||
|
||||
PCollectionList<String> diffLogs = PCollectionList.empty(pipeline);
|
||||
|
||||
for (Class<? extends SqlEntity> clazz : SqlSnapshots.ALL_SQL_ENTITIES) {
|
||||
TupleTag<SqlEntity> tag = ValidateSqlUtils.createSqlEntityTupleTag(clazz);
|
||||
verify(
|
||||
datastoreSnapshot.has(tag), "Missing %s in Datastore snapshot.", clazz.getSimpleName());
|
||||
verify(cloudSqlSnapshot.has(tag), "Missing %s in Cloud SQL snapshot.", clazz.getSimpleName());
|
||||
diffLogs =
|
||||
diffLogs.and(
|
||||
PCollectionList.of(datastoreSnapshot.get(tag))
|
||||
.and(cloudSqlSnapshot.get(tag))
|
||||
.apply(
|
||||
"Combine from both snapshots: " + clazz.getSimpleName(),
|
||||
Flatten.pCollections())
|
||||
.apply(
|
||||
"Assign primary key to merged " + clazz.getSimpleName(),
|
||||
WithKeys.of(ValidateDatabasePipeline::getPrimaryKeyString)
|
||||
.withKeyType(strings()))
|
||||
.apply("Group by primary key " + clazz.getSimpleName(), GroupByKey.create())
|
||||
.apply("Compare " + clazz.getSimpleName(), ParDo.of(new CompareSqlEntity())));
|
||||
}
|
||||
if (diffOutputPath.isPresent()) {
|
||||
diffLogs
|
||||
.apply("Gather diff logs", Flatten.pCollections())
|
||||
.apply(
|
||||
"Output diffs",
|
||||
TextIO.write()
|
||||
.to(diffOutputPath.get())
|
||||
/**
|
||||
* Output to a single file for ease of use since diffs should be few. If this
|
||||
* assumption turns out not to be false, user should abort the pipeline and
|
||||
* investigate why.
|
||||
*/
|
||||
.withoutSharding()
|
||||
.withDelimiter((Strings.repeat("-", 80) + "\n").toCharArray()));
|
||||
}
|
||||
}
|
||||
|
||||
private static String getPrimaryKeyString(SqlEntity sqlEntity) {
|
||||
// SqlEntity.getPrimaryKeyString only works with entities registered with Hibernate.
|
||||
// We are using the BulkQueryJpaTransactionManager, which does not recognize DomainBase and
|
||||
// DomainHistory. See BulkQueryEntities.java for more information.
|
||||
if (sqlEntity instanceof DomainBase) {
|
||||
return "DomainBase_" + ((DomainBase) sqlEntity).getRepoId();
|
||||
}
|
||||
if (sqlEntity instanceof DomainHistory) {
|
||||
return "DomainHistory_" + ((DomainHistory) sqlEntity).getDomainHistoryId().toString();
|
||||
}
|
||||
return sqlEntity.getPrimaryKeyString();
|
||||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
ValidateDatabasePipelineOptions options =
|
||||
PipelineOptionsFactory.fromArgs(args)
|
||||
.withValidation()
|
||||
.as(ValidateDatabasePipelineOptions.class);
|
||||
RegistryPipelineOptions.validateRegistryPipelineOptions(options);
|
||||
|
||||
// Defensively set important options.
|
||||
options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_REPEATABLE_READ);
|
||||
options.setJpaTransactionManagerType(JpaTransactionManagerType.BULK_QUERY);
|
||||
|
||||
// Set up JPA in the pipeline harness (the locally executed part of the main() method). Reuse
|
||||
// code in RegistryPipelineWorkerInitializer, which only applies to pipeline worker VMs.
|
||||
new RegistryPipelineWorkerInitializer().beforeProcessing(options);
|
||||
|
||||
LatestDatastoreSnapshotFinder datastoreSnapshotFinder =
|
||||
DaggerLatestDatastoreSnapshotFinder_LatestDatastoreSnapshotFinderFinderComponent.create()
|
||||
.datastoreSnapshotInfoFinder();
|
||||
|
||||
new ValidateDatabasePipeline(options, datastoreSnapshotFinder).run(Pipeline.create(options));
|
||||
}
|
||||
}
|
|
@ -1,55 +0,0 @@
|
|||
// Copyright 2022 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.comparedb;
|
||||
|
||||
import google.registry.beam.common.RegistryPipelineOptions;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import javax.annotation.Nullable;
|
||||
import org.apache.beam.sdk.options.Description;
|
||||
import org.apache.beam.sdk.options.Validation;
|
||||
|
||||
/** BEAM pipeline options for {@link ValidateDatabasePipeline}. */
|
||||
@DeleteAfterMigration
|
||||
public interface ValidateDatabasePipelineOptions extends RegistryPipelineOptions {
|
||||
|
||||
@Description(
|
||||
"The id of the SQL snapshot to be compared with Datastore. "
|
||||
+ "If null, the current state of the SQL database is used.")
|
||||
@Nullable
|
||||
String getSqlSnapshotId();
|
||||
|
||||
void setSqlSnapshotId(String snapshotId);
|
||||
|
||||
@Description("The latest CommitLogs to load, in ISO8601 format.")
|
||||
@Validation.Required
|
||||
String getLatestCommitLogTimestamp();
|
||||
|
||||
void setLatestCommitLogTimestamp(String commitLogEndTimestamp);
|
||||
|
||||
@Description(
|
||||
"For history entries and EPP resources, only those modified strictly after this time are "
|
||||
+ "included in comparison. Value is in ISO8601 format. "
|
||||
+ "Other entity types are not affected.")
|
||||
@Nullable
|
||||
String getComparisonStartTimestamp();
|
||||
|
||||
void setComparisonStartTimestamp(String comparisonStartTimestamp);
|
||||
|
||||
@Description("The GCS bucket where discrepancies found during comparison should be logged.")
|
||||
@Nullable
|
||||
String getDiffOutputGcsBucket();
|
||||
|
||||
void setDiffOutputGcsBucket(String gcsBucket);
|
||||
}
|
|
@ -1,427 +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.comparedb;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Maps.EntryTransformer;
|
||||
import google.registry.beam.initsql.Transforms;
|
||||
import google.registry.config.RegistryEnvironment;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.ImmutableObject;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.model.billing.BillingEvent;
|
||||
import google.registry.model.contact.ContactBase;
|
||||
import google.registry.model.contact.ContactHistory;
|
||||
import google.registry.model.domain.DomainContent;
|
||||
import google.registry.model.domain.DomainHistory;
|
||||
import google.registry.model.eppcommon.AuthInfo;
|
||||
import google.registry.model.host.HostHistory;
|
||||
import google.registry.model.poll.PollMessage;
|
||||
import google.registry.model.registrar.Registrar;
|
||||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.util.DiffUtils;
|
||||
import java.io.Serializable;
|
||||
import java.lang.reflect.Field;
|
||||
import java.math.BigInteger;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.beam.sdk.metrics.Counter;
|
||||
import org.apache.beam.sdk.metrics.Metrics;
|
||||
import org.apache.beam.sdk.transforms.DoFn;
|
||||
import org.apache.beam.sdk.values.KV;
|
||||
import org.apache.beam.sdk.values.TupleTag;
|
||||
import org.checkerframework.checker.nullness.qual.Nullable;
|
||||
|
||||
/** Helpers for use by {@link ValidateDatabasePipeline}. */
|
||||
@DeleteAfterMigration
|
||||
final class ValidateSqlUtils {
|
||||
|
||||
private ValidateSqlUtils() {}
|
||||
|
||||
private static final ImmutableSet<String> PROBER_CELLS = ImmutableSet.of("IQ", "LG", "TL");
|
||||
private static final ImmutableSet<String> PROBER_TYPES =
|
||||
ImmutableSet.of("ANYT", "ANYTES", "CANARY");
|
||||
|
||||
/**
|
||||
* Query template for finding the median value of the {@code history_revision_id} column in one of
|
||||
* the History tables.
|
||||
*
|
||||
* <p>The {@link ValidateDatabasePipeline} uses this query to parallelize the query to some of the
|
||||
* history tables. Although the {@code repo_id} column is the leading column in the primary keys
|
||||
* of these tables, in practice and with production data, division by {@code history_revision_id}
|
||||
* works slightly faster for unknown reasons.
|
||||
*/
|
||||
private static final String MEDIAN_ID_QUERY_TEMPLATE =
|
||||
"SELECT history_revision_id FROM ( "
|
||||
+ " SELECT"
|
||||
+ " ROW_NUMBER() OVER (ORDER BY history_revision_id ASC) AS rownumber,"
|
||||
+ " history_revision_id"
|
||||
+ " FROM \"%TABLE%\""
|
||||
+ ") AS foo\n"
|
||||
+ "WHERE rownumber in (select count(*) / 2 + 1 from \"%TABLE%\")";
|
||||
|
||||
static Optional<Long> getMedianIdForHistoryTable(String tableName) {
|
||||
Preconditions.checkArgument(
|
||||
tableName.endsWith("History"), "Table must be one of the History tables.");
|
||||
String sqlText = MEDIAN_ID_QUERY_TEMPLATE.replace("%TABLE%", tableName);
|
||||
List results =
|
||||
jpaTm()
|
||||
.transact(() -> jpaTm().getEntityManager().createNativeQuery(sqlText).getResultList());
|
||||
verify(results.size() < 2, "MidPoint query should have at most one result.");
|
||||
if (results.isEmpty()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(((BigInteger) results.get(0)).longValue());
|
||||
}
|
||||
|
||||
static TupleTag<SqlEntity> createSqlEntityTupleTag(Class<? extends SqlEntity> actualType) {
|
||||
return new TupleTag<SqlEntity>(actualType.getSimpleName()) {};
|
||||
}
|
||||
|
||||
static class CompareSqlEntity extends DoFn<KV<String, Iterable<SqlEntity>>, String> {
|
||||
private final HashMap<String, Counter> totalCounters = new HashMap<>();
|
||||
private final HashMap<String, Counter> missingCounters = new HashMap<>();
|
||||
private final HashMap<String, Counter> unequalCounters = new HashMap<>();
|
||||
private final HashMap<String, Counter> badEntityCounters = new HashMap<>();
|
||||
private final HashMap<String, Counter> duplicateEntityCounters = new HashMap<>();
|
||||
|
||||
private String getCounterKey(Class<?> clazz) {
|
||||
return PollMessage.class.isAssignableFrom(clazz) ? "PollMessage" : clazz.getSimpleName();
|
||||
}
|
||||
|
||||
private synchronized void ensureCounterExists(String counterKey) {
|
||||
if (totalCounters.containsKey(counterKey)) {
|
||||
return;
|
||||
}
|
||||
totalCounters.put(counterKey, Metrics.counter("CompareDB", "Total Compared: " + counterKey));
|
||||
missingCounters.put(
|
||||
counterKey, Metrics.counter("CompareDB", "Missing In One DB: " + counterKey));
|
||||
unequalCounters.put(counterKey, Metrics.counter("CompareDB", "Not Equal:" + counterKey));
|
||||
badEntityCounters.put(counterKey, Metrics.counter("CompareDB", "Bad Entities:" + counterKey));
|
||||
duplicateEntityCounters.put(
|
||||
counterKey, Metrics.counter("CompareDB", "Duplicate Entities:" + counterKey));
|
||||
}
|
||||
|
||||
String duplicateEntityLog(String key, ImmutableList<SqlEntity> entities) {
|
||||
return String.format("%s: %d entities.", key, entities.size());
|
||||
}
|
||||
|
||||
String unmatchedEntityLog(String key, SqlEntity entry) {
|
||||
// For a PollMessage only found in Datastore, key is not enough to query for it.
|
||||
return String.format("Missing in one DB:\n%s", entry instanceof PollMessage ? entry : key);
|
||||
}
|
||||
|
||||
/**
|
||||
* A rudimentary debugging helper that prints the first pair of unequal entities in each worker.
|
||||
* This will be removed when we start exporting such entities to GCS.
|
||||
*/
|
||||
String unEqualEntityLog(String key, SqlEntity entry0, SqlEntity entry1) {
|
||||
Map<String, Object> fields0 = ((ImmutableObject) entry0).toDiffableFieldMap();
|
||||
Map<String, Object> fields1 = ((ImmutableObject) entry1).toDiffableFieldMap();
|
||||
return key + " " + DiffUtils.prettyPrintEntityDeepDiff(fields0, fields1);
|
||||
}
|
||||
|
||||
String badEntitiesLog(String key, SqlEntity entry0, SqlEntity entry1) {
|
||||
Map<String, Object> fields0 = ((ImmutableObject) entry0).toDiffableFieldMap();
|
||||
Map<String, Object> fields1 = ((ImmutableObject) entry1).toDiffableFieldMap();
|
||||
return String.format(
|
||||
"Failed to parse one or both entities for key %s:\n%s\n",
|
||||
key, DiffUtils.prettyPrintEntityDeepDiff(fields0, fields1));
|
||||
}
|
||||
|
||||
@ProcessElement
|
||||
public void processElement(
|
||||
@Element KV<String, Iterable<SqlEntity>> kv, OutputReceiver<String> out) {
|
||||
ImmutableList<SqlEntity> entities = ImmutableList.copyOf(kv.getValue());
|
||||
|
||||
verify(!entities.isEmpty(), "Can't happen: no value for key %s.", kv.getKey());
|
||||
|
||||
String counterKey = getCounterKey(entities.get(0).getClass());
|
||||
ensureCounterExists(counterKey);
|
||||
totalCounters.get(counterKey).inc();
|
||||
|
||||
if (entities.size() > 2) {
|
||||
// Duplicates may happen with Cursors if imported across projects. Its key in Datastore, the
|
||||
// id field, encodes the project name and is not fixed by the importing job.
|
||||
duplicateEntityCounters.get(counterKey).inc();
|
||||
out.output(duplicateEntityLog(kv.getKey(), entities) + "\n");
|
||||
return;
|
||||
}
|
||||
|
||||
if (entities.size() == 1) {
|
||||
if (isSpecialCaseProberEntity(entities.get(0))) {
|
||||
return;
|
||||
}
|
||||
missingCounters.get(counterKey).inc();
|
||||
out.output(unmatchedEntityLog(kv.getKey(), entities.get(0)) + "\n");
|
||||
return;
|
||||
}
|
||||
SqlEntity entity0 = entities.get(0);
|
||||
SqlEntity entity1 = entities.get(1);
|
||||
|
||||
if (isSpecialCaseProberEntity(entity0) && isSpecialCaseProberEntity(entity1)) {
|
||||
// Ignore prober-related data: their deletions are not propagated from Datastore to SQL.
|
||||
// When code reaches here, in most cases it involves one soft deleted entity in Datastore
|
||||
// and an SQL entity with its pre-deletion status.
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
entity0 = normalizeEntity(entity0);
|
||||
entity1 = normalizeEntity(entity1);
|
||||
} catch (Exception e) {
|
||||
badEntityCounters.get(counterKey).inc();
|
||||
out.output(badEntitiesLog(kv.getKey(), entity0, entity1));
|
||||
return;
|
||||
}
|
||||
|
||||
Map<String, Object> fields0 =
|
||||
Maps.transformEntries(
|
||||
((ImmutableObject) entity0).toDiffableFieldMap(), new DiffableFieldNormalizer());
|
||||
Map<String, Object> fields1 =
|
||||
Maps.transformEntries(
|
||||
((ImmutableObject) entity1).toDiffableFieldMap(), new DiffableFieldNormalizer());
|
||||
if (!Objects.equals(fields0, fields1)) {
|
||||
unequalCounters.get(counterKey).inc();
|
||||
out.output(kv.getKey() + " " + DiffUtils.prettyPrintEntityDeepDiff(fields0, fields1));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalizes trivial differences between objects persisted in Datastore and SQL.
|
||||
*
|
||||
* <p>This class works on a map generated by {@link ImmutableObject#toDiffableFieldMap}, and
|
||||
* performs the following changes:
|
||||
*
|
||||
* <ul>
|
||||
* <li>If a value is an empty {@link Collection}, it is converted to null
|
||||
* <li>For each {@link google.registry.model.eppcommon.Address} object, empty strings are
|
||||
* removed from its {@code string} field, which is a {@link List}.
|
||||
* </ul>
|
||||
*/
|
||||
static class DiffableFieldNormalizer
|
||||
implements EntryTransformer<String, Object, Object>, Serializable {
|
||||
|
||||
@Override
|
||||
public Object transformEntry(String key, @Nullable Object value) {
|
||||
if (value instanceof Collection && ((Collection<?>) value).isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
if (key.equals("street") && value instanceof List) {
|
||||
return ((List<?>) value)
|
||||
.stream().filter(v -> v != null && !Objects.equals("", v)).collect(Collectors.toList());
|
||||
}
|
||||
// Short-term hack: LinkedHashMap<String, ?> represents a child ImmutableObject instance.
|
||||
if (value instanceof LinkedHashMap
|
||||
&& ((LinkedHashMap<?, ?>) value).keySet().stream().anyMatch(e -> e instanceof String)) {
|
||||
return Maps.transformEntries((Map<String, Object>) value, this);
|
||||
}
|
||||
return value;
|
||||
}
|
||||
}
|
||||
|
||||
static SqlEntity normalizeEntity(SqlEntity sqlEntity) {
|
||||
if (sqlEntity instanceof EppResource) {
|
||||
return normalizeEppResource(sqlEntity);
|
||||
}
|
||||
if (sqlEntity instanceof HistoryEntry) {
|
||||
return (SqlEntity) normalizeHistoryEntry((HistoryEntry) sqlEntity);
|
||||
}
|
||||
return sqlEntity;
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalizes an {@link EppResource} instance for comparison.
|
||||
*
|
||||
* <p>This method may modify the input object using reflection instead of making a copy with
|
||||
* {@code eppResource.asBuilder().build()}, because when {@code eppResource} is a {@link
|
||||
* google.registry.model.domain.DomainBase}, the {@code build} method accesses the Database, which
|
||||
* we want to avoid.
|
||||
*/
|
||||
static SqlEntity normalizeEppResource(SqlEntity eppResource) {
|
||||
try {
|
||||
Field authField =
|
||||
eppResource instanceof DomainContent
|
||||
? DomainContent.class.getDeclaredField("authInfo")
|
||||
: eppResource instanceof ContactBase
|
||||
? ContactBase.class.getDeclaredField("authInfo")
|
||||
: null;
|
||||
if (authField != null) {
|
||||
authField.setAccessible(true);
|
||||
AuthInfo authInfo = (AuthInfo) authField.get(eppResource);
|
||||
// When AuthInfo is missing, the authInfo field is null if the object is loaded from
|
||||
// Datastore, or a PasswordAuth with null properties if loaded from SQL. In the second case
|
||||
// we set the authInfo field to null.
|
||||
if (authInfo != null
|
||||
&& authInfo.getPw() != null
|
||||
&& authInfo.getPw().getRepoId() == null
|
||||
&& authInfo.getPw().getValue() == null) {
|
||||
authField.set(eppResource, null);
|
||||
}
|
||||
}
|
||||
|
||||
Field field = EppResource.class.getDeclaredField("revisions");
|
||||
field.setAccessible(true);
|
||||
field.set(eppResource, null);
|
||||
return eppResource;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalizes a {@link HistoryEntry} for comparison.
|
||||
*
|
||||
* <p>This method modifies the input using reflection because relevant builder methods performs
|
||||
* unwanted checks and changes.
|
||||
*/
|
||||
static HistoryEntry normalizeHistoryEntry(HistoryEntry historyEntry) {
|
||||
// History objects from Datastore do not have details of their EppResource objects
|
||||
// (domainContent, contactBase, hostBase).
|
||||
try {
|
||||
if (historyEntry instanceof DomainHistory) {
|
||||
Field domainContent = DomainHistory.class.getDeclaredField("domainContent");
|
||||
domainContent.setAccessible(true);
|
||||
domainContent.set(historyEntry, null);
|
||||
// Convert empty domainTransactionRecords to null for comparison.
|
||||
Field domainTransactionRecords =
|
||||
HistoryEntry.class.getDeclaredField("domainTransactionRecords");
|
||||
domainTransactionRecords.setAccessible(true);
|
||||
Set<?> domainTransactionRecordsValue = (Set<?>) domainTransactionRecords.get(historyEntry);
|
||||
if (domainTransactionRecordsValue != null && domainTransactionRecordsValue.isEmpty()) {
|
||||
domainTransactionRecords.set(historyEntry, null);
|
||||
}
|
||||
// DomainHistory in Datastore does not have the following properties either:
|
||||
Field nsHosts = DomainHistory.class.getDeclaredField("nsHosts");
|
||||
nsHosts.setAccessible(true);
|
||||
nsHosts.set(historyEntry, null);
|
||||
Field dsDataHistories = DomainHistory.class.getDeclaredField("dsDataHistories");
|
||||
dsDataHistories.setAccessible(true);
|
||||
dsDataHistories.set(historyEntry, null);
|
||||
Field gracePeriodHistories = DomainHistory.class.getDeclaredField("gracePeriodHistories");
|
||||
gracePeriodHistories.setAccessible(true);
|
||||
gracePeriodHistories.set(historyEntry, null);
|
||||
} else if (historyEntry instanceof ContactHistory) {
|
||||
Field contactBase = ContactHistory.class.getDeclaredField("contactBase");
|
||||
contactBase.setAccessible(true);
|
||||
contactBase.set(historyEntry, null);
|
||||
} else if (historyEntry instanceof HostHistory) {
|
||||
Field hostBase = HostHistory.class.getDeclaredField("hostBase");
|
||||
hostBase.setAccessible(true);
|
||||
hostBase.set(historyEntry, null);
|
||||
}
|
||||
return historyEntry;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns {@code true} if {@code entity} is created by the prober and needs special treatment.
|
||||
*
|
||||
* <p>{@link EppResource} entities created by the prober are deleted by a cron job that bypasses
|
||||
* the CommitLog mechanism. As a result, their deletions are not propagated to SQL, creating two
|
||||
* types of mismatches: an entity exists in both databases but differs in lastUpdateTime and
|
||||
* deletionTime; an entity only exists in the SQL database.
|
||||
*
|
||||
* <p>In production, there are few placeholder {@link Registrar registrars} that do not exist in
|
||||
* Datastore. They were manually created to in SQL to solve a one-time problem (see b/187946868
|
||||
* for details). They can be ignored in the database comparison.
|
||||
*/
|
||||
static boolean isSpecialCaseProberEntity(Object entity) {
|
||||
if (entity instanceof EppResource) {
|
||||
EppResource host = (EppResource) entity;
|
||||
if (host.getPersistedCurrentSponsorRegistrarId().startsWith("prober-")) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
if (entity instanceof HistoryEntry) {
|
||||
HistoryEntry historyEntry = (HistoryEntry) entity;
|
||||
if (historyEntry.getRegistrarId().startsWith("prober-")) {
|
||||
// Not all prober entities have "prober-" as registrar prefix.
|
||||
return true;
|
||||
}
|
||||
if (Objects.equals(historyEntry.getReason(), "Deletion of prober data")) {
|
||||
// Soft-delete event in Datastore that is not propagated to SQL.
|
||||
return true;
|
||||
}
|
||||
}
|
||||
if (entity instanceof DomainHistory) {
|
||||
DomainHistory domainHistory = (DomainHistory) entity;
|
||||
if (domainHistory.getDomainContent().isPresent()
|
||||
&& domainHistory.getDomainContent().get().getDomainName().startsWith("prober-")) {
|
||||
// Asynchronously replicated event in SQL.
|
||||
return true;
|
||||
}
|
||||
if (domainHistory.getDomainRepoId() != null) {
|
||||
// Some synthetic events only have domainRepoId.
|
||||
String repoId = domainHistory.getDomainRepoId();
|
||||
if (Transforms.IGNORED_DOMAINS.contains(repoId)) {
|
||||
return true;
|
||||
}
|
||||
String suffix = repoId.substring(repoId.indexOf('-') + 1);
|
||||
String cell = suffix.substring(0, 2);
|
||||
suffix = suffix.substring(2);
|
||||
if (PROBER_CELLS.contains(cell) && PROBER_TYPES.contains(suffix)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (entity instanceof ContactHistory) {
|
||||
if (Transforms.IGNORED_CONTACTS.contains(((ContactHistory) entity).getContactRepoId())) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
if (entity instanceof HostHistory) {
|
||||
if (Transforms.IGNORED_HOSTS.contains(((HostHistory) entity).getHostRepoId())) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
if (entity instanceof BillingEvent) {
|
||||
BillingEvent event = (BillingEvent) entity;
|
||||
if (event.getRegistrarId().startsWith("prober-")) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
if (entity instanceof PollMessage) {
|
||||
if (((PollMessage) entity).getRegistrarId().startsWith("prober-")) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
if (RegistryEnvironment.get().equals(RegistryEnvironment.PRODUCTION)
|
||||
&& entity instanceof Registrar) {
|
||||
Registrar registrar = (Registrar) entity;
|
||||
if (registrar.getRegistrarId().startsWith("prober-wj-")) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -157,12 +157,6 @@
|
|||
<url-pattern>/_dr/cron/readDnsQueue</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<!-- Replicates SQL transactions to Datastore during the Registry 3.0 migration. -->
|
||||
<servlet-mapping>
|
||||
<servlet-name>backend-servlet</servlet-name>
|
||||
<url-pattern>/_dr/cron/replicateToDatastore</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<!-- Publishes DNS updates. -->
|
||||
<servlet-mapping>
|
||||
<servlet-name>backend-servlet</servlet-name>
|
||||
|
@ -243,12 +237,6 @@
|
|||
<url-pattern>/_dr/task/killCommitLogs</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<!-- Replays Datastore commit logs to SQL. -->
|
||||
<servlet-mapping>
|
||||
<servlet-name>backend-servlet</servlet-name>
|
||||
<url-pattern>/_dr/task/replayCommitLogsToSql</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<!-- MapReduce servlet. -->
|
||||
<servlet>
|
||||
<servlet-name>mapreduce</servlet-name>
|
||||
|
@ -413,12 +401,6 @@ have been in the database for a certain period of time. -->
|
|||
<url-pattern>/_dr/task/wipeOutDatastore</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<!-- Action to sync Datastore to a snapshot of the primary SQL database. -->
|
||||
<servlet-mapping>
|
||||
<servlet-name>backend-servlet</servlet-name>
|
||||
<url-pattern>/_dr/task/syncDatastoreToSqlSnapshot</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<!-- Security config -->
|
||||
<security-constraint>
|
||||
<web-resource-collection>
|
||||
|
|
|
@ -20,7 +20,6 @@ import static com.google.common.collect.Sets.difference;
|
|||
import static com.google.common.collect.Sets.union;
|
||||
import static google.registry.config.RegistryConfig.getEppResourceCachingDuration;
|
||||
import static google.registry.config.RegistryConfig.getEppResourceMaxCachedEntries;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.replicaTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.util.CollectionUtils.nullToEmpty;
|
||||
|
@ -39,8 +38,6 @@ import com.googlecode.objectify.annotation.Index;
|
|||
import google.registry.config.RegistryConfig;
|
||||
import google.registry.model.CacheUtils.AppEngineEnvironmentCacheLoader;
|
||||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.model.index.EppResourceIndex;
|
||||
import google.registry.model.index.ForeignKeyIndex;
|
||||
import google.registry.model.ofy.CommitLogManifest;
|
||||
import google.registry.model.transfer.TransferData;
|
||||
import google.registry.persistence.VKey;
|
||||
|
@ -226,12 +223,6 @@ public abstract class EppResource extends BackupGroupRoot implements Buildable {
|
|||
@Override
|
||||
public abstract Builder<?, ?> asBuilder();
|
||||
|
||||
/** Used when replaying from SQL to DS to populate the Datastore indexes. */
|
||||
protected void saveIndexesToDatastore() {
|
||||
ofyTm().putIgnoringReadOnlyWithBackup(ForeignKeyIndex.create(this, getDeletionTime()));
|
||||
ofyTm().putIgnoringReadOnlyWithBackup(EppResourceIndex.create(Key.create(this)));
|
||||
}
|
||||
|
||||
/** EppResources that are loaded via foreign keys should implement this marker interface. */
|
||||
public interface ForeignKeyedEppResource {}
|
||||
|
||||
|
|
|
@ -14,8 +14,6 @@
|
|||
|
||||
package google.registry.model.contact;
|
||||
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
|
||||
import com.googlecode.objectify.Key;
|
||||
import com.googlecode.objectify.annotation.EntitySubclass;
|
||||
import google.registry.model.EppResource;
|
||||
|
@ -136,14 +134,6 @@ public class ContactHistory extends HistoryEntry implements SqlEntity, UnsafeSer
|
|||
return Optional.of(asHistoryEntry());
|
||||
}
|
||||
|
||||
// Used to fill out the contactBase field during asynchronous replay
|
||||
@Override
|
||||
public void beforeSqlSaveOnReplay() {
|
||||
if (contactBase == null) {
|
||||
contactBase = jpaTm().getEntityManager().find(ContactResource.class, getContactRepoId());
|
||||
}
|
||||
}
|
||||
|
||||
/** Class to represent the composite primary key of {@link ContactHistory} entity. */
|
||||
public static class ContactHistoryId extends ImmutableObject implements Serializable {
|
||||
|
||||
|
|
|
@ -66,11 +66,6 @@ public class ContactResource extends ContactBase
|
|||
return ContactBase.cloneContactProjectedAtTime(this, now);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeDatastoreSaveOnReplay() {
|
||||
saveIndexesToDatastore();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Builder asBuilder() {
|
||||
return new Builder(clone(this));
|
||||
|
|
|
@ -14,8 +14,6 @@
|
|||
|
||||
package google.registry.model.domain;
|
||||
|
||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.EppResource.ForeignKeyedEppResource;
|
||||
|
@ -26,7 +24,6 @@ import google.registry.model.host.HostResource;
|
|||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.persistence.WithStringVKey;
|
||||
import google.registry.util.DomainNameUtils;
|
||||
import java.util.Set;
|
||||
import javax.persistence.Access;
|
||||
import javax.persistence.AccessType;
|
||||
|
@ -173,20 +170,6 @@ public class DomainBase extends DomainContent
|
|||
return cloneDomainProjectedAtTime(this, now);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeSqlSaveOnReplay() {
|
||||
fullyQualifiedDomainName = DomainNameUtils.canonicalizeHostname(fullyQualifiedDomainName);
|
||||
dsData =
|
||||
dsData.stream()
|
||||
.filter(datum -> datum.getDigest() != null && datum.getDigest().length > 0)
|
||||
.collect(toImmutableSet());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeDatastoreSaveOnReplay() {
|
||||
saveIndexesToDatastore();
|
||||
}
|
||||
|
||||
public static VKey<DomainBase> createVKey(Key<DomainBase> key) {
|
||||
return VKey.create(DomainBase.class, key.getName(), key);
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import static com.google.common.collect.Sets.difference;
|
|||
import static com.google.common.collect.Sets.intersection;
|
||||
import static google.registry.model.EppResourceUtils.projectResourceOntoBuilderAtTime;
|
||||
import static google.registry.model.EppResourceUtils.setAutomaticTransferSuccessProperties;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerUtil.transactIfJpaTm;
|
||||
import static google.registry.util.CollectionUtils.forceEmptyToNull;
|
||||
|
@ -63,7 +62,6 @@ import google.registry.model.domain.secdns.DelegationSignerData;
|
|||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.model.poll.PollMessage;
|
||||
import google.registry.model.replay.ReplaySpecializer;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.model.tld.Registry;
|
||||
import google.registry.model.transfer.DomainTransferData;
|
||||
|
@ -403,23 +401,6 @@ public class DomainContent extends EppResource
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback to delete grace periods and DelegationSignerData records prior to domain delete.
|
||||
*
|
||||
* <p>See {@link ReplaySpecializer}.
|
||||
*/
|
||||
public static void beforeSqlDelete(VKey<DomainBase> key) {
|
||||
// Delete all grace periods associated with the domain.
|
||||
jpaTm()
|
||||
.query("DELETE FROM GracePeriod WHERE domain_repo_id = :repo_id")
|
||||
.setParameter("repo_id", key.getSqlKey())
|
||||
.executeUpdate();
|
||||
jpaTm()
|
||||
.query("DELETE FROM DelegationSignerData WHERE domain_repo_id = :repo_id")
|
||||
.setParameter("repo_id", key.getSqlKey())
|
||||
.executeUpdate();
|
||||
}
|
||||
|
||||
public static <T> VKey<T> restoreOfyFrom(Key<DomainBase> domainKey, VKey<T> key, Long historyId) {
|
||||
if (historyId == null) {
|
||||
// This is a legacy key (or a null key, in which case this works too)
|
||||
|
|
|
@ -15,7 +15,6 @@
|
|||
package google.registry.model.domain;
|
||||
|
||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
import static google.registry.util.CollectionUtils.nullToEmptyImmutableCopy;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
@ -33,7 +32,6 @@ import google.registry.model.replay.SqlEntity;
|
|||
import google.registry.model.reporting.DomainTransactionRecord;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.util.DomainNameUtils;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashSet;
|
||||
import java.util.Optional;
|
||||
|
@ -303,17 +301,6 @@ public class DomainHistory extends HistoryEntry implements SqlEntity {
|
|||
return Optional.of(asHistoryEntry());
|
||||
}
|
||||
|
||||
// Used to fill out the domainContent field during asynchronous replay
|
||||
@Override
|
||||
public void beforeSqlSaveOnReplay() {
|
||||
if (domainContent == null) {
|
||||
domainContent = jpaTm().getEntityManager().find(DomainBase.class, getDomainRepoId());
|
||||
domainContent.fullyQualifiedDomainName =
|
||||
DomainNameUtils.canonicalizeHostname(domainContent.fullyQualifiedDomainName);
|
||||
fillAuxiliaryFieldsFromDomain(this);
|
||||
}
|
||||
}
|
||||
|
||||
private static void fillAuxiliaryFieldsFromDomain(DomainHistory domainHistory) {
|
||||
if (domainHistory.domainContent != null) {
|
||||
domainHistory.nsHosts = nullToEmptyImmutableCopy(domainHistory.domainContent.nsHosts);
|
||||
|
|
|
@ -14,8 +14,6 @@
|
|||
|
||||
package google.registry.model.host;
|
||||
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
|
||||
import com.googlecode.objectify.Key;
|
||||
import com.googlecode.objectify.annotation.EntitySubclass;
|
||||
import google.registry.model.EppResource;
|
||||
|
@ -26,7 +24,6 @@ import google.registry.model.replay.DatastoreEntity;
|
|||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.util.DomainNameUtils;
|
||||
import java.io.Serializable;
|
||||
import java.util.Optional;
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -137,16 +134,6 @@ public class HostHistory extends HistoryEntry implements SqlEntity, UnsafeSerial
|
|||
return Optional.of(asHistoryEntry());
|
||||
}
|
||||
|
||||
// Used to fill out the hostBase field during asynchronous replay
|
||||
@Override
|
||||
public void beforeSqlSaveOnReplay() {
|
||||
if (hostBase == null) {
|
||||
hostBase = jpaTm().getEntityManager().find(HostResource.class, getHostRepoId());
|
||||
hostBase.fullyQualifiedHostName =
|
||||
DomainNameUtils.canonicalizeHostname(hostBase.fullyQualifiedHostName);
|
||||
}
|
||||
}
|
||||
|
||||
/** Class to represent the composite primary key of {@link HostHistory} entity. */
|
||||
public static class HostHistoryId extends ImmutableObject implements Serializable {
|
||||
|
||||
|
|
|
@ -22,7 +22,6 @@ import google.registry.model.annotations.ReportedOn;
|
|||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.persistence.WithStringVKey;
|
||||
import google.registry.util.DomainNameUtils;
|
||||
import javax.persistence.Access;
|
||||
import javax.persistence.AccessType;
|
||||
|
||||
|
@ -71,16 +70,6 @@ public class HostResource extends HostBase
|
|||
return VKey.create(HostResource.class, getRepoId(), Key.create(this));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeSqlSaveOnReplay() {
|
||||
fullyQualifiedHostName = DomainNameUtils.canonicalizeHostname(fullyQualifiedHostName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeDatastoreSaveOnReplay() {
|
||||
saveIndexesToDatastore();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Builder asBuilder() {
|
||||
return new Builder(clone(this));
|
||||
|
|
|
@ -165,7 +165,6 @@ public class CommitLoggedWork<R> implements Runnable {
|
|||
.addAll(untouchedRootsWithTouchedChildren)
|
||||
.build())
|
||||
.now();
|
||||
ReplayQueue.addInTests(info);
|
||||
}
|
||||
|
||||
/** Check that the timestamp of each BackupGroupRoot is in the past. */
|
||||
|
|
|
@ -1,117 +0,0 @@
|
|||
// 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.model.ofy;
|
||||
|
||||
import static google.registry.model.ofy.EntityWritePriorities.getEntityPriority;
|
||||
import static google.registry.model.ofy.ObjectifyService.auditedOfy;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.config.RegistryEnvironment;
|
||||
import google.registry.model.UpdateAutoTimestamp;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.model.replay.DatastoreEntity;
|
||||
import google.registry.model.replay.ReplaySpecializer;
|
||||
import google.registry.persistence.VKey;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
|
||||
/**
|
||||
* Implements simplified datastore to SQL transaction replay.
|
||||
*
|
||||
* <p>This code is to be removed when the actual replay cron job is implemented.
|
||||
*/
|
||||
@DeleteAfterMigration
|
||||
public class ReplayQueue {
|
||||
|
||||
static ConcurrentLinkedQueue<ImmutableMap<Key<?>, Object>> queue =
|
||||
new ConcurrentLinkedQueue<ImmutableMap<Key<?>, Object>>();
|
||||
|
||||
static void addInTests(TransactionInfo info) {
|
||||
if (RegistryEnvironment.get() == RegistryEnvironment.UNITTEST) {
|
||||
// Transform the entities to be persisted to the set of values as they were actually
|
||||
// persisted.
|
||||
ImmutableMap.Builder<Key<?>, Object> builder = new ImmutableMap.Builder<Key<?>, Object>();
|
||||
for (ImmutableMap.Entry<Key<?>, Object> entry : info.getChanges().entrySet()) {
|
||||
if (entry.getValue().equals(TransactionInfo.Delete.SENTINEL)) {
|
||||
builder.put(entry.getKey(), entry.getValue());
|
||||
} else {
|
||||
// The value is an entity object that has not yet been persisted, and thus some of the
|
||||
// special transformations that we do (notably the auto-timestamp transformations) have
|
||||
// not been applied. Converting the object to an entity and then back again performs
|
||||
// those transformations so that we persist the same values to SQL that we have in
|
||||
// Datastore.
|
||||
builder.put(entry.getKey(), auditedOfy().toPojo(auditedOfy().toEntity(entry.getValue())));
|
||||
}
|
||||
}
|
||||
queue.add(builder.build());
|
||||
}
|
||||
}
|
||||
|
||||
/** Replay all transactions, return the set of keys that were replayed. */
|
||||
public static ImmutableMap<Key<?>, Object> replay() {
|
||||
// We can't use an ImmutableMap.Builder here, we need to be able to overwrite existing values
|
||||
// and the builder doesn't support that.
|
||||
Map<Key<?>, Object> result = new HashMap<Key<?>, Object>();
|
||||
ImmutableMap<Key<?>, Object> changes;
|
||||
while ((changes = queue.poll()) != null) {
|
||||
saveToJpa(changes);
|
||||
result.putAll(changes);
|
||||
}
|
||||
|
||||
return ImmutableMap.copyOf(result);
|
||||
}
|
||||
|
||||
public static void clear() {
|
||||
queue.clear();
|
||||
}
|
||||
|
||||
/** Returns the priority of the entity type in the map entry. */
|
||||
private static int getPriority(ImmutableMap.Entry<Key<?>, Object> entry) {
|
||||
return getEntityPriority(
|
||||
entry.getKey().getKind(), entry.getValue().equals(TransactionInfo.Delete.SENTINEL));
|
||||
}
|
||||
|
||||
private static int compareByPriority(
|
||||
ImmutableMap.Entry<Key<?>, Object> a, ImmutableMap.Entry<Key<?>, Object> b) {
|
||||
return getPriority(a) - getPriority(b);
|
||||
}
|
||||
|
||||
private static void saveToJpa(ImmutableMap<Key<?>, Object> changes) {
|
||||
try (UpdateAutoTimestamp.DisableAutoUpdateResource disabler =
|
||||
UpdateAutoTimestamp.disableAutoUpdate()) {
|
||||
// Sort the changes into an order that will work for insertion into the database.
|
||||
jpaTm()
|
||||
.transact(
|
||||
() ->
|
||||
changes.entrySet().stream()
|
||||
.sorted(ReplayQueue::compareByPriority)
|
||||
.forEach(
|
||||
entry -> {
|
||||
if (entry.getValue().equals(TransactionInfo.Delete.SENTINEL)) {
|
||||
VKey<?> vkey = VKey.from(entry.getKey());
|
||||
ReplaySpecializer.beforeSqlDelete(vkey);
|
||||
jpaTm().delete(vkey);
|
||||
} else {
|
||||
((DatastoreEntity) entry.getValue())
|
||||
.toSqlEntity()
|
||||
.ifPresent(jpaTm()::put);
|
||||
}
|
||||
}));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -29,7 +29,4 @@ import java.util.Optional;
|
|||
public interface DatastoreEntity {
|
||||
|
||||
Optional<SqlEntity> toSqlEntity();
|
||||
|
||||
/** A method called before the object is saved to Datastore in asynchronous replay. */
|
||||
default void beforeDatastoreSaveOnReplay() {}
|
||||
}
|
||||
|
|
|
@ -1,52 +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.model.replay;
|
||||
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.persistence.VKey;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
|
||||
/**
|
||||
* Applies class-specific functions for model objects during transaction replays.
|
||||
*
|
||||
* <p>There are certain cases where changes to an entity require changes to other entities that are
|
||||
* not directly present in the other database. This class allows us to do that by using reflection
|
||||
* to invoke special class methods if they are present.
|
||||
*/
|
||||
@DeleteAfterMigration
|
||||
public class ReplaySpecializer {
|
||||
|
||||
public static void beforeSqlDelete(VKey<?> key) {
|
||||
String methodName = "beforeSqlDelete";
|
||||
Class<?> clazz = key.getKind();
|
||||
try {
|
||||
Method method = clazz.getMethod(methodName, VKey.class);
|
||||
method.invoke(null, key);
|
||||
} catch (NoSuchMethodException e) {
|
||||
// Ignore, this just means that the class doesn't need this hook.
|
||||
} catch (IllegalAccessException e) {
|
||||
throw new RuntimeException(
|
||||
String.format(
|
||||
"%s() method is defined for class %s but is not public.",
|
||||
methodName, clazz.getName()),
|
||||
e);
|
||||
} catch (InvocationTargetException e) {
|
||||
throw new RuntimeException(
|
||||
String.format("%s() method for class %s threw an exception", methodName, clazz.getName()),
|
||||
e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,377 +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.model.replay;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkState;
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static google.registry.model.ofy.ObjectifyService.auditedOfy;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||
import static google.registry.request.Action.Method.GET;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_INTERNAL_SERVER_ERROR;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
import static org.joda.time.Duration.standardHours;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import google.registry.model.UpdateAutoTimestamp;
|
||||
import google.registry.model.annotations.DeleteAfterMigration;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.ReplayDirection;
|
||||
import google.registry.model.server.Lock;
|
||||
import google.registry.persistence.transaction.Transaction;
|
||||
import google.registry.persistence.transaction.TransactionEntity;
|
||||
import google.registry.request.Action;
|
||||
import google.registry.request.Response;
|
||||
import google.registry.request.auth.Auth;
|
||||
import google.registry.util.Clock;
|
||||
import google.registry.util.RequestStatusChecker;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import javax.inject.Inject;
|
||||
import javax.persistence.NoResultException;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/** Cron task to replicate from Cloud SQL to datastore. */
|
||||
@Action(
|
||||
service = Action.Service.BACKEND,
|
||||
path = ReplicateToDatastoreAction.PATH,
|
||||
method = GET,
|
||||
automaticallyPrintOk = true,
|
||||
auth = Auth.AUTH_INTERNAL_OR_ADMIN)
|
||||
@VisibleForTesting
|
||||
@DeleteAfterMigration
|
||||
public class ReplicateToDatastoreAction implements Runnable {
|
||||
public static final String PATH = "/_dr/cron/replicateToDatastore";
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
/** Name of the lock that ensures sequential execution of replays. */
|
||||
public static final String REPLICATE_TO_DATASTORE_LOCK_NAME =
|
||||
ReplicateToDatastoreAction.class.getSimpleName();
|
||||
|
||||
/**
|
||||
* Number of transactions to fetch from SQL. The rationale for 200 is that we're processing these
|
||||
* every minute and our production instance currently does about 2 mutations per second, so this
|
||||
* should generally be enough to scoop up all of the transactions for the past minute.
|
||||
*/
|
||||
public static final int BATCH_SIZE = 200;
|
||||
|
||||
/**
|
||||
* The longest time that we'll keep trying to resolve a gap in the Transaction table in
|
||||
* milliseconds, after which, the gap record will be deleted.
|
||||
*/
|
||||
public static final long MAX_GAP_RETENTION_MILLIS = 300000;
|
||||
|
||||
/**
|
||||
* The maximum number of entitities to be mutated per transaction. For our purposes, the entities
|
||||
* that we're keeping track of are each in their own entity group. Per datastore documentation, we
|
||||
* should be allowed to update up to 25 of them. In practice, we get an error if we go beyond 24
|
||||
* (possibly due to something in our own infrastructure).
|
||||
*/
|
||||
private static final int MAX_ENTITIES_PER_TXN = 24;
|
||||
|
||||
public static final Duration REPLICATE_TO_DATASTORE_LOCK_LEASE_LENGTH = standardHours(1);
|
||||
|
||||
private final Clock clock;
|
||||
private final RequestStatusChecker requestStatusChecker;
|
||||
private final Response response;
|
||||
|
||||
@Inject
|
||||
public ReplicateToDatastoreAction(
|
||||
Clock clock, RequestStatusChecker requestStatusChecker, Response response) {
|
||||
this.clock = clock;
|
||||
this.requestStatusChecker = requestStatusChecker;
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public List<TransactionEntity> getTransactionBatchAtSnapshot() {
|
||||
return getTransactionBatchAtSnapshot(Optional.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the next batch of transactions, optionally from a specific SQL database snapshot.
|
||||
*
|
||||
* <p>Note that this method may also apply transactions from previous batches that had not yet
|
||||
* been committed at the time the previous batch was retrieved.
|
||||
*/
|
||||
static List<TransactionEntity> getTransactionBatchAtSnapshot(Optional<String> snapshotId) {
|
||||
// Get the next batch of transactions that we haven't replicated.
|
||||
LastSqlTransaction lastSqlTxnBeforeBatch = ofyTm().transact(LastSqlTransaction::load);
|
||||
try {
|
||||
return jpaTm()
|
||||
.transactWithoutBackup(
|
||||
() -> {
|
||||
snapshotId.ifPresent(jpaTm()::setDatabaseSnapshot);
|
||||
|
||||
// Fill in any gaps in the transaction log that have since become available before
|
||||
// processing the next batch.
|
||||
applyMissingTransactions();
|
||||
|
||||
return jpaTm()
|
||||
.query(
|
||||
"SELECT txn FROM TransactionEntity txn WHERE id >" + " :lastId ORDER BY id",
|
||||
TransactionEntity.class)
|
||||
.setParameter("lastId", lastSqlTxnBeforeBatch.getTransactionId())
|
||||
.setMaxResults(BATCH_SIZE)
|
||||
.getResultList();
|
||||
});
|
||||
} catch (NoResultException e) {
|
||||
return ImmutableList.of();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterate over the recent gaps in the Transaction table and apply any that have been filled in.
|
||||
*
|
||||
* <p>Must be called from within a JPA transaction.
|
||||
*
|
||||
* <p>Gap rewriting is a complicated matter, and the algorithm is the product of some very deep
|
||||
* consideration by mmuller and weiminyu. Basically, the constraints are:
|
||||
*
|
||||
* <ol>
|
||||
* <li>Replay has to work against a database snapshot (gap replay would break this, so we don't
|
||||
* call this method when replaying against a snapshot)
|
||||
* </ol>
|
||||
*/
|
||||
private static void applyMissingTransactions() {
|
||||
long now = jpaTm().getTransactionTime().getMillis();
|
||||
ImmutableList<ReplayGap> gaps = ofyTm().loadAllOf(ReplayGap.class);
|
||||
jpaTm()
|
||||
.query("SELECT txn from TransactionEntity txn WHERE id IN :gapIds", TransactionEntity.class)
|
||||
.setParameter(
|
||||
"gapIds", gaps.stream().map(gap -> gap.getTransactionId()).collect(toImmutableList()))
|
||||
.getResultStream()
|
||||
.forEach(
|
||||
txn -> {
|
||||
// Transcribe the transaction and delete the gap record in the same ofy transaction.
|
||||
ofyTm()
|
||||
.transact(
|
||||
() -> {
|
||||
// Write the transaction to datastore.
|
||||
try {
|
||||
Transaction.deserialize(txn.getContents()).writeToDatastore();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Error during transaction deserialization", e);
|
||||
}
|
||||
|
||||
// Find and delete the gap record.
|
||||
ImmutableList<ReplayGap> filledGaps =
|
||||
gaps.stream()
|
||||
.filter(gap -> gap.getTransactionId() == txn.getId())
|
||||
.collect(toImmutableList());
|
||||
checkState(
|
||||
filledGaps.size() == 1,
|
||||
"Bad list of gaps for discovered id: %s",
|
||||
filledGaps);
|
||||
auditedOfy().deleteIgnoringReadOnlyWithoutBackup().entity(gaps.get(0));
|
||||
});
|
||||
logger.atInfo().log("Applied missing transaction %s", txn.getId());
|
||||
});
|
||||
|
||||
// Clean up any gaps that have expired (in batches because they're each in their own entity
|
||||
// group).
|
||||
ArrayList<ReplayGap> gapBatch = new ArrayList<>();
|
||||
gaps.stream()
|
||||
.forEach(
|
||||
gap -> {
|
||||
if (now - gap.getTimestamp().getMillis() > MAX_GAP_RETENTION_MILLIS) {
|
||||
gapBatch.add(gap);
|
||||
}
|
||||
if (gapBatch.size() == MAX_ENTITIES_PER_TXN) {
|
||||
deleteReplayGaps(gapBatch);
|
||||
gapBatch.clear();
|
||||
}
|
||||
});
|
||||
if (!gapBatch.isEmpty()) {
|
||||
deleteReplayGaps(gapBatch);
|
||||
}
|
||||
}
|
||||
|
||||
private static void deleteReplayGaps(ArrayList<ReplayGap> gapsToDelete) {
|
||||
logger.atInfo().log(
|
||||
"deleting gap records for %s",
|
||||
gapsToDelete.stream().map(g -> g.getTransactionId()).collect(toImmutableList()));
|
||||
ofyTm()
|
||||
.transact(() -> auditedOfy().deleteIgnoringReadOnlyWithoutBackup().entities(gapsToDelete));
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply a transaction to Datastore.
|
||||
*
|
||||
* <p>Throws an exception if a fatal error occurred and the batch should be aborted.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static void applyTransaction(TransactionEntity txnEntity) {
|
||||
logger.atInfo().log("Applying a single transaction Cloud SQL -> Cloud Datastore.");
|
||||
boolean done = false;
|
||||
try (UpdateAutoTimestamp.DisableAutoUpdateResource disabler =
|
||||
UpdateAutoTimestamp.disableAutoUpdate()) {
|
||||
|
||||
// We put this in a do/while loop because we can potentially clean out some range of gaps
|
||||
// first and we want to do those in their own transaction so as not to run up the entity group
|
||||
// count. (This is a highly pathological case: consecutive gaps are rare, but the fact that
|
||||
// they can occur potentially increases the entity group count to beyond what we can
|
||||
// accommodate in a single transaction.)
|
||||
do {
|
||||
done =
|
||||
ofyTm()
|
||||
.transact(
|
||||
() -> {
|
||||
// Reload the last transaction id, which could possibly have changed.
|
||||
LastSqlTransaction lastSqlTxn = LastSqlTransaction.load();
|
||||
long nextTxnId = lastSqlTxn.getTransactionId() + 1;
|
||||
|
||||
// Skip missing transactions. Missed transactions can happen normally. If a
|
||||
// transaction gets rolled back, the sequence counter doesn't.
|
||||
int gapCount = 0;
|
||||
while (nextTxnId < txnEntity.getId()) {
|
||||
logger.atWarning().log(
|
||||
"Ignoring transaction %s, which does not exist.", nextTxnId);
|
||||
auditedOfy()
|
||||
.saveIgnoringReadOnlyWithoutBackup()
|
||||
.entity(new ReplayGap(ofyTm().getTransactionTime(), nextTxnId));
|
||||
++nextTxnId;
|
||||
|
||||
// Don't exceed the entity group count trying to clean these up (we stop at
|
||||
// max
|
||||
// - 1 because we also want to save the lastSqlTransaction).
|
||||
if (++gapCount == MAX_ENTITIES_PER_TXN - 1) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// Don't write gap records in the same transaction as the SQL transaction that
|
||||
// we're replaying. Return false to force us to go through and repeat in a
|
||||
// new
|
||||
// transaction.
|
||||
if (gapCount > 0) {
|
||||
// We haven't replayed the transaction, but we've determined that we can
|
||||
// ignore everything before it so update lastSqlTransaction accordingly.
|
||||
auditedOfy()
|
||||
.saveIgnoringReadOnlyWithoutBackup()
|
||||
.entity(lastSqlTxn.cloneWithNewTransactionId(nextTxnId - 1));
|
||||
return false;
|
||||
}
|
||||
|
||||
if (nextTxnId > txnEntity.getId()) {
|
||||
// We've already replayed this transaction. This shouldn't happen, as GAE
|
||||
// cron
|
||||
// is supposed to avoid overruns and this action shouldn't be executed from
|
||||
// any
|
||||
// other context, but it's not harmful as we can just ignore the
|
||||
// transaction.
|
||||
// Log it so that we know about it and move on.
|
||||
logger.atWarning().log(
|
||||
"Ignoring transaction %s, which appears to have already been applied.",
|
||||
txnEntity.getId());
|
||||
return true;
|
||||
}
|
||||
|
||||
logger.atInfo().log(
|
||||
"Applying transaction %s to Cloud Datastore.", txnEntity.getId());
|
||||
|
||||
// At this point, we know txnEntity is the correct next transaction, so write
|
||||
// it
|
||||
// to Datastore.
|
||||
try {
|
||||
Transaction.deserialize(txnEntity.getContents()).writeToDatastore();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Error during transaction deserialization", e);
|
||||
}
|
||||
|
||||
// Write the updated last transaction id to Datastore as part of this
|
||||
// Datastore
|
||||
// transaction.
|
||||
auditedOfy()
|
||||
.saveIgnoringReadOnlyWithoutBackup()
|
||||
.entity(lastSqlTxn.cloneWithNewTransactionId(nextTxnId));
|
||||
logger.atInfo().log(
|
||||
"Finished applying single transaction Cloud SQL -> Cloud Datastore.");
|
||||
return true;
|
||||
});
|
||||
} while (!done);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
MigrationState state = DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc());
|
||||
if (!state.getReplayDirection().equals(ReplayDirection.SQL_TO_DATASTORE)) {
|
||||
String message =
|
||||
String.format(
|
||||
"Skipping ReplicateToDatastoreAction because we are in migration phase %s.", state);
|
||||
logger.atInfo().log(message);
|
||||
// App Engine will retry on any non-2xx status code, which we don't want in this case.
|
||||
response.setStatus(SC_NO_CONTENT);
|
||||
response.setPayload(message);
|
||||
return;
|
||||
}
|
||||
Optional<Lock> lock =
|
||||
Lock.acquireSql(
|
||||
REPLICATE_TO_DATASTORE_LOCK_NAME,
|
||||
null,
|
||||
REPLICATE_TO_DATASTORE_LOCK_LEASE_LENGTH,
|
||||
requestStatusChecker,
|
||||
false);
|
||||
if (!lock.isPresent()) {
|
||||
String message = "Can't acquire ReplicateToDatastoreAction lock, aborting.";
|
||||
logger.atSevere().log(message);
|
||||
// App Engine will retry on any non-2xx status code, which we don't want in this case.
|
||||
response.setStatus(SC_NO_CONTENT);
|
||||
response.setPayload(message);
|
||||
return;
|
||||
}
|
||||
try {
|
||||
logger.atInfo().log("Processing transaction replay batch Cloud SQL -> Cloud Datastore.");
|
||||
int numTransactionsReplayed = replayAllTransactions();
|
||||
String resultMessage =
|
||||
String.format(
|
||||
"Replayed %d transaction(s) from Cloud SQL -> Datastore.", numTransactionsReplayed);
|
||||
logger.atInfo().log(resultMessage);
|
||||
response.setPayload(resultMessage);
|
||||
response.setStatus(SC_OK);
|
||||
} catch (Throwable t) {
|
||||
String message = "Errored out replaying files.";
|
||||
logger.atSevere().withCause(t).log(message);
|
||||
response.setStatus(SC_INTERNAL_SERVER_ERROR);
|
||||
response.setPayload(message);
|
||||
} finally {
|
||||
lock.ifPresent(Lock::releaseSql);
|
||||
}
|
||||
}
|
||||
|
||||
private int replayAllTransactions() {
|
||||
return replayAllTransactions(Optional.empty());
|
||||
}
|
||||
|
||||
public static int replayAllTransactions(Optional<String> snapshotId) {
|
||||
int numTransactionsReplayed = 0;
|
||||
List<TransactionEntity> transactionBatch;
|
||||
do {
|
||||
transactionBatch = getTransactionBatchAtSnapshot(snapshotId);
|
||||
for (TransactionEntity transaction : transactionBatch) {
|
||||
applyTransaction(transaction);
|
||||
numTransactionsReplayed++;
|
||||
}
|
||||
} while (!transactionBatch.isEmpty());
|
||||
return numTransactionsReplayed;
|
||||
}
|
||||
}
|
|
@ -31,9 +31,6 @@ public interface SqlEntity {
|
|||
|
||||
Optional<DatastoreEntity> toDatastoreEntity();
|
||||
|
||||
/** A method that will be called before the object is saved to SQL in asynchronous replay. */
|
||||
default void beforeSqlSaveOnReplay() {}
|
||||
|
||||
/** Returns this entity's primary key field(s) in a string. */
|
||||
default String getPrimaryKeyString() {
|
||||
return jpaTm()
|
||||
|
|
|
@ -20,8 +20,6 @@ import google.registry.backup.BackupModule;
|
|||
import google.registry.backup.CommitLogCheckpointAction;
|
||||
import google.registry.backup.DeleteOldCommitLogsAction;
|
||||
import google.registry.backup.ExportCommitLogDiffAction;
|
||||
import google.registry.backup.ReplayCommitLogsToSqlAction;
|
||||
import google.registry.backup.SyncDatastoreToSqlSnapshotAction;
|
||||
import google.registry.batch.BatchModule;
|
||||
import google.registry.batch.DeleteExpiredDomainsAction;
|
||||
import google.registry.batch.DeleteLoadTestDataAction;
|
||||
|
@ -62,7 +60,6 @@ import google.registry.export.sheet.SyncRegistrarsSheetAction;
|
|||
import google.registry.flows.FlowComponent;
|
||||
import google.registry.flows.custom.CustomLogicModule;
|
||||
import google.registry.mapreduce.MapreduceModule;
|
||||
import google.registry.model.replay.ReplicateToDatastoreAction;
|
||||
import google.registry.monitoring.whitebox.WhiteboxModule;
|
||||
import google.registry.rdap.UpdateRegistrarRdapBaseUrlsAction;
|
||||
import google.registry.rde.BrdaCopyAction;
|
||||
|
@ -187,10 +184,6 @@ interface BackendRequestComponent {
|
|||
|
||||
RelockDomainAction relockDomainAction();
|
||||
|
||||
ReplayCommitLogsToSqlAction replayCommitLogsToSqlAction();
|
||||
|
||||
ReplicateToDatastoreAction replicateToDatastoreAction();
|
||||
|
||||
ResaveAllEppResourcesAction resaveAllEppResourcesAction();
|
||||
|
||||
ResaveAllEppResourcesPipelineAction resaveAllEppResourcesPipelineAction();
|
||||
|
@ -199,8 +192,6 @@ interface BackendRequestComponent {
|
|||
|
||||
SendExpiringCertificateNotificationEmailAction sendExpiringCertificateNotificationEmailAction();
|
||||
|
||||
SyncDatastoreToSqlSnapshotAction syncDatastoreToSqlSnapshotAction();
|
||||
|
||||
SyncGroupMembersAction syncGroupMembersAction();
|
||||
|
||||
SyncRegistrarsSheetAction syncRegistrarsSheetAction();
|
||||
|
|
|
@ -106,11 +106,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
|||
private static final ThreadLocal<TransactionInfo> transactionInfo =
|
||||
ThreadLocal.withInitial(TransactionInfo::new);
|
||||
|
||||
// If this value is present, use it to determine whether or not to replay SQL transactions to
|
||||
// Datastore, rather than using the schedule stored in Datastore.
|
||||
private static final ThreadLocal<Optional<Boolean>> replaySqlToDatastoreOverrideForTest =
|
||||
ThreadLocal.withInitial(Optional::empty);
|
||||
|
||||
public JpaTransactionManagerImpl(EntityManagerFactory emf, Clock clock) {
|
||||
this.emf = emf;
|
||||
this.clock = clock;
|
||||
|
@ -825,16 +820,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
|||
return entity;
|
||||
}
|
||||
|
||||
/** Sets the override to always/never replay SQL transactions to Datastore. */
|
||||
public static void setReplaySqlToDatastoreOverrideForTest(boolean replaySqlToDs) {
|
||||
replaySqlToDatastoreOverrideForTest.set(Optional.of(replaySqlToDs));
|
||||
}
|
||||
|
||||
/** Removes the replay-SQL-to-Datastore override; the migration schedule will then be used. */
|
||||
public static void removeReplaySqlToDsOverrideForTest() {
|
||||
replaySqlToDatastoreOverrideForTest.set(Optional.empty());
|
||||
}
|
||||
|
||||
/** Returns true if the entity class should be replicated from SQL to datastore. */
|
||||
private static boolean shouldReplicate(Class<?> entityClass) {
|
||||
return !NonReplicatedEntity.class.isAssignableFrom(entityClass)
|
||||
|
@ -860,14 +845,12 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
|||
checkArgumentNotNull(clock);
|
||||
inTransaction = true;
|
||||
transactionTime = clock.nowUtc();
|
||||
if (withBackup
|
||||
&& replaySqlToDatastoreOverrideForTest
|
||||
.get()
|
||||
.orElseGet(
|
||||
() ->
|
||||
DatabaseMigrationStateSchedule.getValueAtTime(transactionTime)
|
||||
.getReplayDirection()
|
||||
.equals(ReplayDirection.SQL_TO_DATASTORE))) {
|
||||
Supplier<Boolean> sqlToDsReplaySupplier =
|
||||
() ->
|
||||
DatabaseMigrationStateSchedule.getValueAtTime(transactionTime)
|
||||
.getReplayDirection()
|
||||
.equals(ReplayDirection.SQL_TO_DATASTORE);
|
||||
if (withBackup && sqlToDsReplaySupplier.get()) {
|
||||
contentsBuilder = new Transaction.Builder();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.storage.onestore.v3.OnestoreEntity.EntityProto;
|
||||
import google.registry.model.Buildable;
|
||||
import google.registry.model.ImmutableObject;
|
||||
import google.registry.model.replay.DatastoreEntity;
|
||||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.persistence.VKey;
|
||||
import java.io.ByteArrayInputStream;
|
||||
|
@ -241,10 +240,6 @@ public class Transaction extends ImmutableObject implements Buildable {
|
|||
|
||||
@Override
|
||||
public void writeToDatastore() {
|
||||
// this should always be the case, but check just in case
|
||||
if (entity instanceof DatastoreEntity) {
|
||||
((DatastoreEntity) entity).beforeDatastoreSaveOnReplay();
|
||||
}
|
||||
ofyTm().putIgnoringReadOnlyWithBackup(entity);
|
||||
}
|
||||
|
||||
|
|
|
@ -127,10 +127,8 @@ public final class RegistryTool {
|
|||
.put("update_server_locks", UpdateServerLocksCommand.class)
|
||||
.put("update_tld", UpdateTldCommand.class)
|
||||
.put("upload_claims_list", UploadClaimsListCommand.class)
|
||||
.put("validate_datastore", ValidateDatastoreCommand.class)
|
||||
.put("validate_escrow_deposit", ValidateEscrowDepositCommand.class)
|
||||
.put("validate_login_credentials", ValidateLoginCredentialsCommand.class)
|
||||
.put("validate_sql", ValidateSqlCommand.class)
|
||||
.put("verify_ote", VerifyOteCommand.class)
|
||||
.put("whois_query", WhoisQueryCommand.class)
|
||||
.build();
|
||||
|
|
|
@ -175,14 +175,10 @@ interface RegistryToolComponent {
|
|||
|
||||
void inject(UpdateTldCommand command);
|
||||
|
||||
void inject(ValidateDatastoreCommand command);
|
||||
|
||||
void inject(ValidateEscrowDepositCommand command);
|
||||
|
||||
void inject(ValidateLoginCredentialsCommand command);
|
||||
|
||||
void inject(ValidateSqlCommand command);
|
||||
|
||||
void inject(WhoisQueryCommand command);
|
||||
|
||||
AppEngineConnection appEngineConnection();
|
||||
|
|
|
@ -1,232 +0,0 @@
|
|||
// Copyright 2022 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.tools;
|
||||
|
||||
import static google.registry.beam.BeamUtils.createJobName;
|
||||
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.google.api.services.dataflow.Dataflow;
|
||||
import com.google.api.services.dataflow.model.Job;
|
||||
import com.google.api.services.dataflow.model.LaunchFlexTemplateParameter;
|
||||
import com.google.api.services.dataflow.model.LaunchFlexTemplateRequest;
|
||||
import com.google.api.services.dataflow.model.LaunchFlexTemplateResponse;
|
||||
import com.google.common.base.Ascii;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.beam.common.DatabaseSnapshot;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.tools.params.DateTimeParameter;
|
||||
import google.registry.util.Clock;
|
||||
import google.registry.util.RequestStatusChecker;
|
||||
import google.registry.util.Sleeper;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/** Shared setup for commands that validate the data replication between Datastore and Cloud SQL. */
|
||||
abstract class ValidateDatabaseMigrationCommand
|
||||
implements CommandWithConnection, CommandWithRemoteApi {
|
||||
|
||||
private static final String PIPELINE_NAME = "validate_database_pipeline";
|
||||
|
||||
private static final String MANUAL_PIPELINE_LAUNCH_COMMAND_TEMPLATE =
|
||||
"gcloud dataflow flex-template run "
|
||||
+ "\"%s-${USER}-$(date +%%Y%%m%%dt%%H%%M%%S)\" "
|
||||
+ "--template-file-gcs-location %s "
|
||||
+ "--project %s "
|
||||
+ "--region=%s "
|
||||
+ "--worker-machine-type=n2-standard-8 --num-workers=8 "
|
||||
+ "--parameters registryEnvironment=%s "
|
||||
+ "--parameters sqlSnapshotId=%s "
|
||||
+ "--parameters latestCommitLogTimestamp=%s "
|
||||
+ "--parameters diffOutputGcsBucket=%s ";
|
||||
|
||||
// States indicating a job is not finished yet.
|
||||
static final ImmutableSet<String> DATAFLOW_JOB_RUNNING_STATES =
|
||||
ImmutableSet.of(
|
||||
"JOB_STATE_UNKNOWN",
|
||||
"JOB_STATE_RUNNING",
|
||||
"JOB_STATE_STOPPED",
|
||||
"JOB_STATE_PENDING",
|
||||
"JOB_STATE_QUEUED");
|
||||
|
||||
static final Duration JOB_POLLING_INTERVAL = Duration.standardSeconds(60);
|
||||
|
||||
@Parameter(
|
||||
names = {"-m", "--manual"},
|
||||
description =
|
||||
"If true, let user launch the comparison pipeline manually out of band. "
|
||||
+ "Command will wait for user key-press to exit after syncing Datastore.")
|
||||
boolean manualLaunchPipeline;
|
||||
|
||||
@Parameter(
|
||||
names = {"-r", "--release"},
|
||||
description = "The release tag of the BEAM pipeline to run. It defaults to 'live'.")
|
||||
String release = "live";
|
||||
|
||||
@Parameter(
|
||||
names = {"-c", "--comparisonStartTimestamp"},
|
||||
description =
|
||||
"When comparing History and Epp Resource entities, ignore those that have not"
|
||||
+ " changed since this time.",
|
||||
converter = DateTimeParameter.class)
|
||||
DateTime comparisonStartTimestamp;
|
||||
|
||||
@Parameter(
|
||||
names = {"-o", "--outputBucket"},
|
||||
description =
|
||||
"The GCS bucket where data discrepancies are logged. "
|
||||
+ "It defaults to ${projectId}-beam")
|
||||
String outputBucket;
|
||||
|
||||
@Inject Clock clock;
|
||||
@Inject Dataflow dataflow;
|
||||
|
||||
@Inject
|
||||
@Config("defaultJobRegion")
|
||||
String jobRegion;
|
||||
|
||||
@Inject
|
||||
@Config("beamStagingBucketUrl")
|
||||
String stagingBucketUrl;
|
||||
|
||||
@Inject
|
||||
@Config("projectId")
|
||||
String projectId;
|
||||
|
||||
@Inject Sleeper sleeper;
|
||||
|
||||
AppEngineConnection connection;
|
||||
|
||||
@Override
|
||||
public void setConnection(AppEngineConnection connection) {
|
||||
this.connection = connection;
|
||||
}
|
||||
|
||||
String getDataflowJobStatus(String jobId) {
|
||||
try {
|
||||
return dataflow
|
||||
.projects()
|
||||
.locations()
|
||||
.jobs()
|
||||
.get(projectId, jobRegion, jobId)
|
||||
.execute()
|
||||
.getCurrentState();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
void launchPipelineAndWaitUntilFinish(
|
||||
String pipelineName, DatabaseSnapshot snapshot, String latestCommitTimestamp) {
|
||||
Job pipelineJob =
|
||||
launchComparisonPipeline(pipelineName, snapshot.getSnapshotId(), latestCommitTimestamp)
|
||||
.getJob();
|
||||
String jobId = pipelineJob.getId();
|
||||
|
||||
System.out.printf("Launched comparison pipeline %s (%s).\n", pipelineJob.getName(), jobId);
|
||||
|
||||
while (DATAFLOW_JOB_RUNNING_STATES.contains(getDataflowJobStatus(jobId))) {
|
||||
sleeper.sleepInterruptibly(JOB_POLLING_INTERVAL);
|
||||
}
|
||||
System.out.printf(
|
||||
"Pipeline ended with %s state. Please check counters for results.\n",
|
||||
getDataflowJobStatus(jobId));
|
||||
}
|
||||
|
||||
String getOutputBucket() {
|
||||
return Optional.ofNullable(outputBucket).orElse(projectId + "-beam");
|
||||
}
|
||||
|
||||
String getContainerSpecGcsPath() {
|
||||
return String.format(
|
||||
"%s/%s_metadata.json", stagingBucketUrl.replace("live", release), PIPELINE_NAME);
|
||||
}
|
||||
|
||||
String getManualLaunchCommand(
|
||||
String jobName, String snapshotId, String latestCommitLogTimestamp) {
|
||||
String baseCommand =
|
||||
String.format(
|
||||
MANUAL_PIPELINE_LAUNCH_COMMAND_TEMPLATE,
|
||||
jobName,
|
||||
getContainerSpecGcsPath(),
|
||||
projectId,
|
||||
jobRegion,
|
||||
RegistryToolEnvironment.get().name(),
|
||||
snapshotId,
|
||||
latestCommitLogTimestamp,
|
||||
getOutputBucket());
|
||||
if (comparisonStartTimestamp == null) {
|
||||
return baseCommand;
|
||||
}
|
||||
return baseCommand + "--parameters comparisonStartTimestamp=" + comparisonStartTimestamp;
|
||||
}
|
||||
|
||||
LaunchFlexTemplateResponse launchComparisonPipeline(
|
||||
String jobName, String sqlSnapshotId, String latestCommitLogTimestamp) {
|
||||
try {
|
||||
// Hardcode machine type and initial workers to force a quick start.
|
||||
ImmutableMap.Builder<String, String> paramsBuilder =
|
||||
new ImmutableMap.Builder()
|
||||
.put("workerMachineType", "n2-standard-8")
|
||||
.put("numWorkers", "8")
|
||||
.put("sqlSnapshotId", sqlSnapshotId)
|
||||
.put("latestCommitLogTimestamp", latestCommitLogTimestamp)
|
||||
.put("registryEnvironment", RegistryToolEnvironment.get().name())
|
||||
.put("diffOutputGcsBucket", getOutputBucket());
|
||||
if (comparisonStartTimestamp != null) {
|
||||
paramsBuilder.put("comparisonStartTimestamp", comparisonStartTimestamp.toString());
|
||||
}
|
||||
LaunchFlexTemplateParameter parameter =
|
||||
new LaunchFlexTemplateParameter()
|
||||
.setJobName(createJobName(Ascii.toLowerCase(jobName).replace('_', '-'), clock))
|
||||
.setContainerSpecGcsPath(getContainerSpecGcsPath())
|
||||
.setParameters(paramsBuilder.build());
|
||||
return dataflow
|
||||
.projects()
|
||||
.locations()
|
||||
.flexTemplates()
|
||||
.launch(
|
||||
projectId, jobRegion, new LaunchFlexTemplateRequest().setLaunchParameter(parameter))
|
||||
.execute();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A fake implementation of {@link RequestStatusChecker} for managing SQL-backed locks from
|
||||
* non-AppEngine platforms. This is only required until the Nomulus server is migrated off
|
||||
* AppEngine.
|
||||
*/
|
||||
static class FakeRequestStatusChecker implements RequestStatusChecker {
|
||||
|
||||
private final String logId =
|
||||
ValidateDatastoreCommand.class.getSimpleName() + "-" + UUID.randomUUID();
|
||||
|
||||
@Override
|
||||
public String getLogId() {
|
||||
return logId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRunning(String requestLogId) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,105 +0,0 @@
|
|||
// Copyright 2022 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.tools;
|
||||
|
||||
import static google.registry.model.replay.ReplicateToDatastoreAction.REPLICATE_TO_DATASTORE_LOCK_LEASE_LENGTH;
|
||||
import static google.registry.model.replay.ReplicateToDatastoreAction.REPLICATE_TO_DATASTORE_LOCK_NAME;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
|
||||
import com.beust.jcommander.Parameters;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.net.MediaType;
|
||||
import google.registry.backup.SyncDatastoreToSqlSnapshotAction;
|
||||
import google.registry.beam.common.DatabaseSnapshot;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.ReplayDirection;
|
||||
import google.registry.model.server.Lock;
|
||||
import google.registry.request.Action.Service;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Validates asynchronously replicated data from the primary Cloud SQL database to Datastore.
|
||||
*
|
||||
* <p>This command suspends the replication process (by acquiring the replication lock), take a
|
||||
* snapshot of the Cloud SQL database, invokes a Nomulus server action to sync Datastore to this
|
||||
* snapshot (See {@link SyncDatastoreToSqlSnapshotAction} for details), and finally launches a BEAM
|
||||
* pipeline to compare Datastore with the given SQL snapshot.
|
||||
*
|
||||
* <p>This command does not lock up the SQL database. Normal processing can proceed.
|
||||
*/
|
||||
@Parameters(commandDescription = "Validates Datastore with the primary Cloud SQL database.")
|
||||
public class ValidateDatastoreCommand extends ValidateDatabaseMigrationCommand {
|
||||
|
||||
private static final Service NOMULUS_SERVICE = Service.BACKEND;
|
||||
|
||||
@Override
|
||||
public void run() throws Exception {
|
||||
MigrationState state = DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc());
|
||||
if (!state.getReplayDirection().equals(ReplayDirection.SQL_TO_DATASTORE)) {
|
||||
throw new IllegalStateException("Cannot validate Datastore in migration step " + state);
|
||||
}
|
||||
Optional<Lock> lock =
|
||||
Lock.acquireSql(
|
||||
REPLICATE_TO_DATASTORE_LOCK_NAME,
|
||||
null,
|
||||
REPLICATE_TO_DATASTORE_LOCK_LEASE_LENGTH,
|
||||
new FakeRequestStatusChecker(),
|
||||
false);
|
||||
if (!lock.isPresent()) {
|
||||
throw new IllegalStateException("Cannot acquire the async propagation lock.");
|
||||
}
|
||||
|
||||
try {
|
||||
try (DatabaseSnapshot snapshot = DatabaseSnapshot.createSnapshot()) {
|
||||
System.out.printf("Obtained snapshot %s\n", snapshot.getSnapshotId());
|
||||
AppEngineConnection connectionToService = connection.withService(NOMULUS_SERVICE);
|
||||
String response =
|
||||
connectionToService.sendPostRequest(
|
||||
getNomulusEndpoint(snapshot.getSnapshotId()),
|
||||
ImmutableMap.<String, String>of(),
|
||||
MediaType.PLAIN_TEXT_UTF_8,
|
||||
"".getBytes(UTF_8));
|
||||
System.out.println(response);
|
||||
|
||||
lock.ifPresent(Lock::releaseSql);
|
||||
lock = Optional.empty();
|
||||
|
||||
// See SyncDatastoreToSqlSnapshotAction for response format.
|
||||
String latestCommitTimestamp =
|
||||
response.substring(response.lastIndexOf('(') + 1, response.lastIndexOf(')'));
|
||||
|
||||
if (manualLaunchPipeline) {
|
||||
System.out.printf(
|
||||
"To launch the pipeline manually, use the following command:\n%s\n",
|
||||
getManualLaunchCommand(
|
||||
"validate-datastore", snapshot.getSnapshotId(), latestCommitTimestamp));
|
||||
|
||||
System.out.print("\nEnter any key to continue when the pipeline ends:");
|
||||
System.in.read();
|
||||
} else {
|
||||
launchPipelineAndWaitUntilFinish("validate-datastore", snapshot, latestCommitTimestamp);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
lock.ifPresent(Lock::releaseSql);
|
||||
}
|
||||
}
|
||||
|
||||
private static String getNomulusEndpoint(String sqlSnapshotId) {
|
||||
return String.format(
|
||||
"%s?sqlSnapshotId=%s", SyncDatastoreToSqlSnapshotAction.PATH, sqlSnapshotId);
|
||||
}
|
||||
}
|
|
@ -1,91 +0,0 @@
|
|||
// Copyright 2022 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.tools;
|
||||
|
||||
import static google.registry.backup.ReplayCommitLogsToSqlAction.REPLAY_TO_SQL_LOCK_LEASE_LENGTH;
|
||||
import static google.registry.backup.ReplayCommitLogsToSqlAction.REPLAY_TO_SQL_LOCK_NAME;
|
||||
|
||||
import com.beust.jcommander.Parameters;
|
||||
import google.registry.beam.common.DatabaseSnapshot;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.ReplayDirection;
|
||||
import google.registry.model.replay.SqlReplayCheckpoint;
|
||||
import google.registry.model.server.Lock;
|
||||
import google.registry.persistence.transaction.TransactionManagerFactory;
|
||||
import java.util.Optional;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/**
|
||||
* Validates asynchronously replicated data from the primary Datastore to Cloud SQL.
|
||||
*
|
||||
* <p>This command suspends the replication process (by acquiring the replication lock), take a
|
||||
* snapshot of the Cloud SQL database, finds the corresponding Datastore snapshot, and finally
|
||||
* launches a BEAM pipeline to compare the two snapshots.
|
||||
*
|
||||
* <p>This command does not lock up either database. Normal processing can proceed.
|
||||
*/
|
||||
@Parameters(commandDescription = "Validates Cloud SQL with the primary Datastore.")
|
||||
public class ValidateSqlCommand extends ValidateDatabaseMigrationCommand {
|
||||
|
||||
@Override
|
||||
public void run() throws Exception {
|
||||
MigrationState state = DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc());
|
||||
if (!state.getReplayDirection().equals(ReplayDirection.DATASTORE_TO_SQL)) {
|
||||
throw new IllegalStateException("Cannot validate SQL in migration step " + state);
|
||||
}
|
||||
Optional<Lock> lock =
|
||||
Lock.acquireSql(
|
||||
REPLAY_TO_SQL_LOCK_NAME,
|
||||
null,
|
||||
REPLAY_TO_SQL_LOCK_LEASE_LENGTH,
|
||||
new FakeRequestStatusChecker(),
|
||||
false);
|
||||
if (!lock.isPresent()) {
|
||||
throw new IllegalStateException("Cannot acquire the async propagation lock.");
|
||||
}
|
||||
|
||||
try {
|
||||
DateTime latestCommitLogTime =
|
||||
TransactionManagerFactory.jpaTm().transact(() -> SqlReplayCheckpoint.get());
|
||||
try (DatabaseSnapshot databaseSnapshot = DatabaseSnapshot.createSnapshot()) {
|
||||
// Eagerly release the commitlog replay lock so that replay can resume.
|
||||
lock.ifPresent(Lock::releaseSql);
|
||||
lock = Optional.empty();
|
||||
|
||||
System.out.printf(
|
||||
"Start comparison with SQL snapshot (%s) and CommitLog timestamp (%s).\n",
|
||||
databaseSnapshot.getSnapshotId(), latestCommitLogTime);
|
||||
|
||||
if (manualLaunchPipeline) {
|
||||
System.out.printf(
|
||||
"To launch the pipeline manually, use the following command:\n%s\n",
|
||||
getManualLaunchCommand(
|
||||
"validate-sql",
|
||||
databaseSnapshot.getSnapshotId(),
|
||||
latestCommitLogTime.toString()));
|
||||
|
||||
System.out.print("\nEnter any key to continue when the pipeline ends:");
|
||||
System.in.read();
|
||||
} else {
|
||||
launchPipelineAndWaitUntilFinish(
|
||||
"validate-sql", databaseSnapshot, latestCommitLogTime.toString());
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
lock.ifPresent(Lock::releaseSql);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,677 +0,0 @@
|
|||
// 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.backup;
|
||||
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.backup.RestoreCommitLogsActionTest.createCheckpoint;
|
||||
import static google.registry.backup.RestoreCommitLogsActionTest.saveDiffFile;
|
||||
import static google.registry.backup.RestoreCommitLogsActionTest.saveDiffFileNotToRestore;
|
||||
import static google.registry.model.ImmutableObjectSubject.assertAboutImmutableObjects;
|
||||
import static google.registry.model.common.DatabaseMigrationStateSchedule.DEFAULT_TRANSITION_MAP;
|
||||
import static google.registry.model.common.EntityGroupRoot.getCrossTldKey;
|
||||
import static google.registry.model.ofy.CommitLogBucket.getBucketKey;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.testing.DatabaseHelper.createTld;
|
||||
import static google.registry.testing.DatabaseHelper.insertInDb;
|
||||
import static google.registry.testing.DatabaseHelper.newDomainBase;
|
||||
import static google.registry.testing.DatabaseHelper.persistActiveContact;
|
||||
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.lenient;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
||||
import com.google.cloud.storage.contrib.nio.testing.LocalStorageHelper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.ImmutableSortedMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.truth.Truth8;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.gcs.GcsUtils;
|
||||
import google.registry.model.common.ClassPathManager;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.domain.GracePeriod;
|
||||
import google.registry.model.domain.secdns.DelegationSignerData;
|
||||
import google.registry.model.index.ForeignKeyIndex;
|
||||
import google.registry.model.ofy.CommitLogBucket;
|
||||
import google.registry.model.ofy.CommitLogManifest;
|
||||
import google.registry.model.ofy.CommitLogMutation;
|
||||
import google.registry.model.ofy.Ofy;
|
||||
import google.registry.model.rde.RdeMode;
|
||||
import google.registry.model.rde.RdeNamingUtils;
|
||||
import google.registry.model.rde.RdeRevision;
|
||||
import google.registry.model.registrar.RegistrarContact;
|
||||
import google.registry.model.replay.SqlReplayCheckpoint;
|
||||
import google.registry.model.server.Lock;
|
||||
import google.registry.model.tld.label.PremiumList;
|
||||
import google.registry.model.tld.label.PremiumList.PremiumEntry;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.persistence.transaction.JpaTransactionManager;
|
||||
import google.registry.persistence.transaction.TransactionManagerFactory;
|
||||
import google.registry.testing.AppEngineExtension;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.FakeClock;
|
||||
import google.registry.testing.FakeResponse;
|
||||
import google.registry.testing.InjectExtension;
|
||||
import google.registry.testing.TestObject;
|
||||
import google.registry.util.RequestStatusChecker;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.Executors;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.InOrder;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
/** Tests for {@link ReplayCommitLogsToSqlAction}. */
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class ReplayCommitLogsToSqlActionTest {
|
||||
|
||||
private final FakeClock fakeClock = new FakeClock(DateTime.parse("2000-01-01TZ"));
|
||||
|
||||
@RegisterExtension
|
||||
public final AppEngineExtension appEngine =
|
||||
AppEngineExtension.builder()
|
||||
.withDatastoreAndCloudSql()
|
||||
.withClock(fakeClock)
|
||||
.withOfyTestEntities(TestObject.class)
|
||||
.withJpaUnitTestEntities(
|
||||
ContactResource.class,
|
||||
DelegationSignerData.class,
|
||||
DomainBase.class,
|
||||
GracePeriod.class,
|
||||
Lock.class,
|
||||
PremiumList.class,
|
||||
PremiumEntry.class,
|
||||
RegistrarContact.class,
|
||||
SqlReplayCheckpoint.class,
|
||||
TestObject.class)
|
||||
.build();
|
||||
|
||||
@RegisterExtension public final InjectExtension inject = new InjectExtension();
|
||||
|
||||
/** Local GCS service. */
|
||||
private final GcsUtils gcsUtils = new GcsUtils(LocalStorageHelper.getOptions());
|
||||
|
||||
private final ReplayCommitLogsToSqlAction action = new ReplayCommitLogsToSqlAction();
|
||||
private final FakeResponse response = new FakeResponse();
|
||||
@Mock private RequestStatusChecker requestStatusChecker;
|
||||
|
||||
@BeforeAll
|
||||
static void beforeAll() {
|
||||
ClassPathManager.addTestEntityClass(TestObject.class);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
inject.setStaticField(Ofy.class, "clock", fakeClock);
|
||||
lenient().when(requestStatusChecker.getLogId()).thenReturn("requestLogId");
|
||||
action.gcsUtils = gcsUtils;
|
||||
action.response = response;
|
||||
action.requestStatusChecker = requestStatusChecker;
|
||||
action.clock = fakeClock;
|
||||
action.gcsBucket = "gcs bucket";
|
||||
action.diffLister = new GcsDiffFileLister();
|
||||
action.diffLister.gcsUtils = gcsUtils;
|
||||
action.diffLister.executorProvider = MoreExecutors::newDirectExecutorService;
|
||||
action.diffLister.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
|
||||
jpaTm()
|
||||
.transact(
|
||||
() ->
|
||||
DatabaseMigrationStateSchedule.set(
|
||||
ImmutableSortedMap.of(
|
||||
START_OF_TIME,
|
||||
MigrationState.DATASTORE_ONLY,
|
||||
START_OF_TIME.plusMinutes(1),
|
||||
MigrationState.DATASTORE_PRIMARY)));
|
||||
TestObject.beforeSqlSaveCallCount = 0;
|
||||
TestObject.beforeSqlDeleteCallCount = 0;
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
void afterEach() {
|
||||
DatabaseHelper.removeDatabaseMigrationSchedule();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReplay_multipleDiffFiles() throws Exception {
|
||||
insertInDb(TestObject.create("previous to keep"), TestObject.create("previous to delete"));
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
// Create 3 transactions, across two diff files.
|
||||
// Before: {"previous to keep", "previous to delete"}
|
||||
// 1a: Add {"a", "b"}, Delete {"previous to delete"}
|
||||
// 1b: Add {"c", "d"}, Delete {"a"}
|
||||
// 2: Add {"e", "f"}, Delete {"c"}
|
||||
// After: {"previous to keep", "b", "d", "e", "f"}
|
||||
Key<CommitLogManifest> manifest1aKey =
|
||||
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(3));
|
||||
Key<CommitLogManifest> manifest1bKey =
|
||||
CommitLogManifest.createKey(getBucketKey(2), now.minusMinutes(2));
|
||||
Key<CommitLogManifest> manifest2Key =
|
||||
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
|
||||
saveDiffFileNotToRestore(gcsUtils, now.minusMinutes(2));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1),
|
||||
now.minusMinutes(3),
|
||||
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))),
|
||||
CommitLogMutation.create(manifest1aKey, TestObject.create("a")),
|
||||
CommitLogMutation.create(manifest1aKey, TestObject.create("b")),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(2),
|
||||
now.minusMinutes(2),
|
||||
ImmutableSet.of(Key.create(TestObject.create("a")))),
|
||||
CommitLogMutation.create(manifest1bKey, TestObject.create("c")),
|
||||
CommitLogMutation.create(manifest1bKey, TestObject.create("d")));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1),
|
||||
now.minusMinutes(1),
|
||||
ImmutableSet.of(Key.create(TestObject.create("c")))),
|
||||
CommitLogMutation.create(manifest2Key, TestObject.create("e")),
|
||||
CommitLogMutation.create(manifest2Key, TestObject.create("f")));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
fakeClock.advanceOneMilli();
|
||||
runAndAssertSuccess(now, 2, 3);
|
||||
assertExpectedIds("previous to keep", "b", "d", "e", "f");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReplay_noManifests() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
insertInDb(TestObject.create("previous to keep"));
|
||||
saveDiffFileNotToRestore(gcsUtils, now.minusMinutes(1));
|
||||
saveDiffFile(gcsUtils, createCheckpoint(now.minusMillis(2)));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMillis(1)));
|
||||
runAndAssertSuccess(now.minusMillis(1), 0, 0);
|
||||
assertExpectedIds("previous to keep");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReplay_dryRun() throws Exception {
|
||||
action.dryRun = true;
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
insertInDb(TestObject.create("previous to keep"));
|
||||
Key<CommitLogBucket> bucketKey = getBucketKey(1);
|
||||
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(bucketKey, now);
|
||||
saveDiffFileNotToRestore(gcsUtils, now.minusMinutes(2));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(bucketKey, now, null),
|
||||
CommitLogMutation.create(manifestKey, TestObject.create("a")),
|
||||
CommitLogMutation.create(manifestKey, TestObject.create("b")));
|
||||
|
||||
action.run();
|
||||
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
||||
assertThat(response.getPayload())
|
||||
.isEqualTo(
|
||||
"Running in dry-run mode, the first set of commit log files processed would be from "
|
||||
+ "searching from 1999-12-31T23:59:00.000Z to 1999-12-31T23:59:59.999Z and would "
|
||||
+ "contain 1 file(s). They are (limit 10): \n"
|
||||
+ "[commit_diff_until_1999-12-31T23:59:00.000Z]");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReplay_manifestWithNoDeletions() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
insertInDb(TestObject.create("previous to keep"));
|
||||
Key<CommitLogBucket> bucketKey = getBucketKey(1);
|
||||
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(bucketKey, now);
|
||||
saveDiffFileNotToRestore(gcsUtils, now.minusMinutes(2));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(bucketKey, now, null),
|
||||
CommitLogMutation.create(manifestKey, TestObject.create("a")),
|
||||
CommitLogMutation.create(manifestKey, TestObject.create("b")));
|
||||
runAndAssertSuccess(now.minusMinutes(1), 1, 1);
|
||||
assertExpectedIds("previous to keep", "a", "b");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReplay_manifestWithNoMutations() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
insertInDb(TestObject.create("previous to keep"), TestObject.create("previous to delete"));
|
||||
saveDiffFileNotToRestore(gcsUtils, now.minusMinutes(2));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1),
|
||||
now,
|
||||
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))));
|
||||
runAndAssertSuccess(now.minusMinutes(1), 1, 1);
|
||||
assertExpectedIds("previous to keep");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReplay_mutateExistingEntity() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
insertInDb(TestObject.create("existing", "a"));
|
||||
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(getBucketKey(1), now);
|
||||
saveDiffFileNotToRestore(gcsUtils, now.minusMinutes(1).minusMillis(1));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1)));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMillis(1)),
|
||||
CommitLogManifest.create(getBucketKey(1), now, null),
|
||||
CommitLogMutation.create(manifestKey, TestObject.create("existing", "b")));
|
||||
action.run();
|
||||
TestObject fromDatabase =
|
||||
jpaTm().transact(() -> jpaTm().loadByKey(VKey.createSql(TestObject.class, "existing")));
|
||||
assertThat(fromDatabase.getField()).isEqualTo("b");
|
||||
}
|
||||
|
||||
// This should be harmless
|
||||
@Test
|
||||
void testReplay_deleteMissingEntity() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
insertInDb(TestObject.create("previous to keep", "a"));
|
||||
saveDiffFileNotToRestore(gcsUtils, now.minusMinutes(1).minusMillis(1));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1)));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMillis(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1),
|
||||
now,
|
||||
ImmutableSet.of(Key.create(TestObject.create("previous to delete")))));
|
||||
action.run();
|
||||
assertExpectedIds("previous to keep");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReplay_doesNotChangeUpdateTime() throws Exception {
|
||||
// Save the contact with an earlier updateTimestamp
|
||||
ContactResource contactResource = persistActiveContact("contactfoobar");
|
||||
DateTime persistenceTime = fakeClock.nowUtc();
|
||||
Key<CommitLogBucket> bucketKey = getBucketKey(1);
|
||||
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(bucketKey, persistenceTime);
|
||||
CommitLogMutation mutation =
|
||||
tm().transact(() -> CommitLogMutation.create(manifestKey, contactResource));
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(persistenceTime.minusMinutes(1).minusMillis(1)));
|
||||
|
||||
// Replay the contact-save an hour later; the updateTimestamp should be unchanged
|
||||
fakeClock.advanceBy(Duration.standardHours(1));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(persistenceTime.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1), persistenceTime.minusMinutes(1), ImmutableSet.of()),
|
||||
mutation);
|
||||
runAndAssertSuccess(persistenceTime.minusMinutes(1), 1, 1);
|
||||
assertAboutImmutableObjects()
|
||||
.that(jpaTm().transact(() -> jpaTm().loadByEntity(contactResource)))
|
||||
.isEqualExceptFields(contactResource, "revisions");
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
void testReplay_properlyWeighted() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
Key<CommitLogManifest> manifestKey =
|
||||
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
|
||||
// Create (but don't save to SQL) a domain + contact
|
||||
createTld("tld");
|
||||
DomainBase domain = newDomainBase("example.tld");
|
||||
CommitLogMutation domainMutation =
|
||||
tm().transact(() -> CommitLogMutation.create(manifestKey, domain));
|
||||
ContactResource contact = tm().transact(() -> tm().loadByKey(domain.getRegistrant()));
|
||||
CommitLogMutation contactMutation =
|
||||
tm().transact(() -> CommitLogMutation.create(manifestKey, contact));
|
||||
|
||||
// Create and save to SQL a registrar contact that we will delete
|
||||
RegistrarContact toDelete = AppEngineExtension.makeRegistrarContact1();
|
||||
insertInDb(toDelete);
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
|
||||
// spy the txn manager so we can see what order things were inserted/removed
|
||||
JpaTransactionManager spy = spy(jpaTm());
|
||||
TransactionManagerFactory.setJpaTm(() -> spy);
|
||||
// Save in the commit logs the domain and contact (in that order) and the token deletion
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1), now.minusMinutes(1), ImmutableSet.of(Key.create(toDelete))),
|
||||
domainMutation,
|
||||
contactMutation);
|
||||
|
||||
runAndAssertSuccess(now.minusMinutes(1), 1, 1);
|
||||
// Verify two things:
|
||||
// 1. that the contact insert occurred before the domain insert (necessary for FK ordering)
|
||||
// even though the domain came first in the file
|
||||
// 2. that the allocation token delete occurred after the insertions
|
||||
InOrder inOrder = Mockito.inOrder(spy);
|
||||
inOrder.verify(spy).putIgnoringReadOnlyWithoutBackup(any(ContactResource.class));
|
||||
inOrder.verify(spy).putIgnoringReadOnlyWithoutBackup(any(DomainBase.class));
|
||||
inOrder.verify(spy).deleteIgnoringReadOnlyWithoutBackup(toDelete.createVKey());
|
||||
inOrder.verify(spy).putIgnoringReadOnlyWithoutBackup(any(SqlReplayCheckpoint.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
void testReplay_properlyWeighted_doesNotApplyCrossTransactions() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
Key<CommitLogManifest> manifestKey =
|
||||
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
|
||||
|
||||
// Create and save the standard contact
|
||||
ContactResource contact = persistActiveContact("contact1234");
|
||||
jpaTm().transact(() -> jpaTm().put(contact));
|
||||
|
||||
// Simulate a Datastore transaction with a new version of the contact
|
||||
ContactResource contactWithEdit =
|
||||
contact.asBuilder().setEmailAddress("replay@example.tld").build();
|
||||
CommitLogMutation contactMutation =
|
||||
ofyTm().transact(() -> CommitLogMutation.create(manifestKey, contactWithEdit));
|
||||
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
|
||||
// spy the txn manager so we can see what order things were inserted
|
||||
JpaTransactionManager spy = spy(jpaTm());
|
||||
TransactionManagerFactory.setJpaTm(() -> spy);
|
||||
// Save two commits -- the deletion, then the new version of the contact
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1).plusMillis(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1), now.minusMinutes(1), ImmutableSet.of(Key.create(contact))),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1), now.minusMinutes(1).plusMillis(1), ImmutableSet.of()),
|
||||
contactMutation);
|
||||
runAndAssertSuccess(now.minusMinutes(1).plusMillis(1), 1, 2);
|
||||
// Verify that the delete occurred first (because it was in the first transaction) even though
|
||||
// deletes have higher weight
|
||||
ArgumentCaptor<Object> putCaptor = ArgumentCaptor.forClass(Object.class);
|
||||
InOrder inOrder = Mockito.inOrder(spy);
|
||||
inOrder.verify(spy).deleteIgnoringReadOnlyWithoutBackup(contact.createVKey());
|
||||
inOrder.verify(spy).putIgnoringReadOnlyWithoutBackup(putCaptor.capture());
|
||||
assertThat(putCaptor.getValue().getClass()).isEqualTo(ContactResource.class);
|
||||
assertThat(jpaTm().transact(() -> jpaTm().loadByKey(contact.createVKey()).getEmailAddress()))
|
||||
.isEqualTo("replay@example.tld");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSuccess_nonReplicatedEntity_isNotReplayed() {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
|
||||
// spy the txn manager so we can verify it's never called
|
||||
JpaTransactionManager spy = spy(jpaTm());
|
||||
TransactionManagerFactory.setJpaTm(() -> spy);
|
||||
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
Key<CommitLogManifest> manifestKey =
|
||||
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
|
||||
|
||||
createTld("tld");
|
||||
// Have a commit log with a couple objects that shouldn't be replayed
|
||||
String triplet = RdeNamingUtils.makePartialName("tld", fakeClock.nowUtc(), RdeMode.FULL);
|
||||
RdeRevision rdeRevision =
|
||||
RdeRevision.create(triplet, "tld", fakeClock.nowUtc().toLocalDate(), RdeMode.FULL, 1);
|
||||
ForeignKeyIndex<DomainBase> fki = ForeignKeyIndex.create(newDomainBase("foo.tld"), now);
|
||||
tm().transact(
|
||||
() -> {
|
||||
try {
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1), now.minusMinutes(1), ImmutableSet.of()),
|
||||
// RDE Revisions are not replicated
|
||||
CommitLogMutation.create(manifestKey, rdeRevision),
|
||||
// FKIs aren't replayed to SQL at all
|
||||
CommitLogMutation.create(manifestKey, fki));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
runAndAssertSuccess(now.minusMinutes(1), 1, 1);
|
||||
// jpaTm()::putIgnoringReadOnly should only have been called with the checkpoint and the lock
|
||||
verify(spy, times(2)).putIgnoringReadOnlyWithoutBackup(any(SqlReplayCheckpoint.class));
|
||||
verify(spy).putIgnoringReadOnlyWithoutBackup(any(Lock.class));
|
||||
verify(spy, times(3)).putIgnoringReadOnlyWithoutBackup(any());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSuccess_nonReplicatedEntity_isNotDeleted() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
// spy the txn manager so we can verify it's never called
|
||||
JpaTransactionManager spy = spy(jpaTm());
|
||||
TransactionManagerFactory.setJpaTm(() -> spy);
|
||||
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
// Save a couple deletes that aren't propagated to SQL (the objects deleted are irrelevant)
|
||||
Key<CommitLogManifest> manifestKey =
|
||||
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(1));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1),
|
||||
now.minusMinutes(1),
|
||||
// one object only exists in Datastore, one is dually-written (so isn't replicated)
|
||||
ImmutableSet.of(getCrossTldKey(), manifestKey)));
|
||||
|
||||
runAndAssertSuccess(now.minusMinutes(1), 1, 1);
|
||||
verify(spy, times(0)).delete(any(VKey.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFailure_notEnabled() {
|
||||
jpaTm().transact(() -> DatabaseMigrationStateSchedule.set(DEFAULT_TRANSITION_MAP.toValueMap()));
|
||||
action.run();
|
||||
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
||||
assertThat(response.getPayload())
|
||||
.isEqualTo(
|
||||
"Skipping ReplayCommitLogsToSqlAction because we are in migration phase"
|
||||
+ " DATASTORE_ONLY.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFailure_cannotAcquireLock() {
|
||||
Truth8.assertThat(
|
||||
Lock.acquireSql(
|
||||
ReplayCommitLogsToSqlAction.class.getSimpleName(),
|
||||
null,
|
||||
Duration.standardHours(1),
|
||||
requestStatusChecker,
|
||||
false))
|
||||
.isPresent();
|
||||
fakeClock.advanceOneMilli();
|
||||
action.run();
|
||||
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
||||
assertThat(response.getPayload())
|
||||
.isEqualTo("Can't acquire SQL commit log replay lock, aborting.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSuccess_beforeSqlSaveCallback() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
Key<CommitLogBucket> bucketKey = getBucketKey(1);
|
||||
Key<CommitLogManifest> manifestKey = CommitLogManifest.createKey(bucketKey, now);
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(bucketKey, now, null),
|
||||
CommitLogMutation.create(manifestKey, TestObject.create("a")));
|
||||
runAndAssertSuccess(now.minusMinutes(1), 1, 1);
|
||||
assertThat(TestObject.beforeSqlSaveCallCount).isEqualTo(1);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSuccess_deleteSqlCallback() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1),
|
||||
now.minusMinutes(1),
|
||||
ImmutableSet.of(Key.create(TestObject.create("to delete")))));
|
||||
action.run();
|
||||
assertThat(TestObject.beforeSqlDeleteCallCount).isEqualTo(1);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSuccess_cascadingDelete() throws Exception {
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
createTld("tld");
|
||||
DomainBase domain =
|
||||
newDomainBase("example.tld")
|
||||
.asBuilder()
|
||||
.setDsData(ImmutableSet.of(DelegationSignerData.create(1, 2, 3, new byte[] {0, 1, 2})))
|
||||
.build();
|
||||
insertInDb(domain);
|
||||
assertThat(jpaTm().transact(() -> jpaTm().loadAllOf(DelegationSignerData.class))).isNotEmpty();
|
||||
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1), now.minusMinutes(3), ImmutableSet.of(Key.create(domain))));
|
||||
runAndAssertSuccess(now.minusMinutes(1), 1, 1);
|
||||
|
||||
assertThat(jpaTm().transact(() -> jpaTm().loadAllOf(DomainBase.class))).isEmpty();
|
||||
assertThat(jpaTm().transact(() -> jpaTm().loadAllOf(DelegationSignerData.class))).isEmpty();
|
||||
}
|
||||
|
||||
@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
|
||||
void testReplay_deleteAndResaveCascade_withOtherDeletion_noErrors() throws Exception {
|
||||
createTld("tld");
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
jpaTm().transact(() -> SqlReplayCheckpoint.set(now.minusMinutes(1).minusMillis(1)));
|
||||
|
||||
// Save a domain with a particular dsData in SQL as the base
|
||||
ImmutableSet<DelegationSignerData> dsData =
|
||||
ImmutableSet.of(DelegationSignerData.create(1, 2, 3, new byte[] {4, 5, 6}));
|
||||
DomainBase domainWithDsData =
|
||||
newDomainBase("example.tld").asBuilder().setDsData(dsData).build();
|
||||
insertInDb(domainWithDsData);
|
||||
|
||||
// Replay a version of that domain without the dsData
|
||||
Key<CommitLogManifest> manifestKeyOne =
|
||||
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(3));
|
||||
DomainBase domainWithoutDsData =
|
||||
domainWithDsData.asBuilder().setDsData(ImmutableSet.of()).build();
|
||||
CommitLogMutation domainWithoutDsDataMutation =
|
||||
ofyTm().transact(() -> CommitLogMutation.create(manifestKeyOne, domainWithoutDsData));
|
||||
|
||||
// Create an object (any object) to delete via replay to trigger Hibernate flush events
|
||||
TestObject testObject = TestObject.create("foo", "bar");
|
||||
insertInDb(testObject);
|
||||
|
||||
// Replay the original domain, with the original dsData
|
||||
Key<CommitLogManifest> manifestKeyTwo =
|
||||
CommitLogManifest.createKey(getBucketKey(1), now.minusMinutes(2));
|
||||
CommitLogMutation domainWithOriginalDsDataMutation =
|
||||
ofyTm().transact(() -> CommitLogMutation.create(manifestKeyTwo, domainWithDsData));
|
||||
|
||||
// If we try to perform all the events in one transaction (cascade-removal of the dsData,
|
||||
// cascade-adding the dsData back in, and deleting any other random object), Hibernate will
|
||||
// throw an exception
|
||||
saveDiffFile(
|
||||
gcsUtils,
|
||||
createCheckpoint(now.minusMinutes(1)),
|
||||
CommitLogManifest.create(
|
||||
getBucketKey(1), now.minusMinutes(3), ImmutableSet.of(Key.create(testObject))),
|
||||
domainWithoutDsDataMutation,
|
||||
CommitLogManifest.create(getBucketKey(1), now.minusMinutes(2), ImmutableSet.of()),
|
||||
domainWithOriginalDsDataMutation);
|
||||
runAndAssertSuccess(now.minusMinutes(1), 1, 2);
|
||||
}
|
||||
|
||||
private void runAndAssertSuccess(
|
||||
DateTime expectedCheckpointTime, int numFiles, int numTransactions) {
|
||||
action.run();
|
||||
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
||||
assertThat(response.getPayload())
|
||||
.startsWith(
|
||||
String.format(
|
||||
"Caught up to current time after replaying %d file(s) containing %d total"
|
||||
+ " transaction(s)",
|
||||
numFiles, numTransactions));
|
||||
assertThat(jpaTm().transact(SqlReplayCheckpoint::get)).isEqualTo(expectedCheckpointTime);
|
||||
}
|
||||
|
||||
private void assertExpectedIds(String... expectedIds) {
|
||||
ImmutableList<String> actualIds =
|
||||
jpaTm()
|
||||
.transact(
|
||||
() ->
|
||||
jpaTm().loadAllOf(TestObject.class).stream()
|
||||
.map(TestObject::getId)
|
||||
.collect(toImmutableList()));
|
||||
assertThat(actualIds).containsExactlyElementsIn(expectedIds);
|
||||
}
|
||||
}
|
|
@ -55,7 +55,6 @@ import google.registry.model.tld.Registry;
|
|||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.FakeClock;
|
||||
import google.registry.testing.FakeResponse;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import google.registry.testing.mapreduce.MapreduceTestCase;
|
||||
|
@ -65,8 +64,6 @@ import java.util.Optional;
|
|||
import org.joda.money.Money;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ExpandRecurringBillingEventsAction}. */
|
||||
@DualDatabaseTest
|
||||
|
@ -76,10 +73,6 @@ public class ExpandRecurringBillingEventsActionTest
|
|||
private DateTime currentTestTime = DateTime.parse("1999-01-05T00:00:00Z");
|
||||
private final FakeClock clock = new FakeClock(currentTestTime);
|
||||
|
||||
@Order(Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
public final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
private DomainBase domain;
|
||||
private DomainHistory historyEntry;
|
||||
private BillingEvent.Recurring recurring;
|
||||
|
|
|
@ -1,95 +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.comparedb;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.beam.TestPipelineExtension;
|
||||
import google.registry.beam.initsql.DatastoreSetupHelper;
|
||||
import google.registry.model.domain.DomainHistory;
|
||||
import google.registry.model.ofy.Ofy;
|
||||
import google.registry.model.registrar.Registrar;
|
||||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
||||
import google.registry.testing.DatastoreEntityExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import google.registry.testing.InjectExtension;
|
||||
import java.io.Serializable;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Optional;
|
||||
import org.apache.beam.sdk.coders.SerializableCoder;
|
||||
import org.apache.beam.sdk.testing.PAssert;
|
||||
import org.apache.beam.sdk.values.PCollectionTuple;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
/** Unit tests for {@link DatastoreSnapshots}. */
|
||||
class DatastoreSnapshotsTest {
|
||||
static final DateTime START_TIME = DateTime.parse("2000-01-01T00:00:00.0Z");
|
||||
|
||||
private FakeClock fakeClock = new FakeClock(START_TIME);
|
||||
|
||||
@RegisterExtension
|
||||
@Order(Order.DEFAULT - 1)
|
||||
final transient DatastoreEntityExtension datastore =
|
||||
new DatastoreEntityExtension().allThreads(true);
|
||||
|
||||
@RegisterExtension final transient InjectExtension injectExtension = new InjectExtension();
|
||||
|
||||
@SuppressWarnings("WeakerAccess")
|
||||
@TempDir
|
||||
transient Path tmpDir;
|
||||
|
||||
@RegisterExtension
|
||||
final transient TestPipelineExtension testPipeline =
|
||||
TestPipelineExtension.create().enableAbandonedNodeEnforcement(true);
|
||||
|
||||
@RegisterExtension
|
||||
final transient JpaIntegrationTestExtension database =
|
||||
new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationTestExtension();
|
||||
|
||||
DatastoreSetupHelper setupHelper;
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() throws Exception {
|
||||
injectExtension.setStaticField(Ofy.class, "clock", fakeClock);
|
||||
setupHelper = new DatastoreSetupHelper(tmpDir, fakeClock).initializeData();
|
||||
testPipeline
|
||||
.getCoderRegistry()
|
||||
.registerCoderForClass(SqlEntity.class, SerializableCoder.of(Serializable.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
void loadDatastoreSnapshotByKind() {
|
||||
PCollectionTuple tuple =
|
||||
DatastoreSnapshots.loadDatastoreSnapshotByKind(
|
||||
testPipeline,
|
||||
setupHelper.exportDir.getAbsolutePath(),
|
||||
setupHelper.commitLogDir.getAbsolutePath(),
|
||||
START_TIME,
|
||||
fakeClock.nowUtc().plusMillis(1),
|
||||
ImmutableSet.copyOf(DatastoreSetupHelper.ALL_KINDS),
|
||||
Optional.empty());
|
||||
PAssert.that(tuple.get(ValidateSqlUtils.createSqlEntityTupleTag(Registrar.class)))
|
||||
.containsInAnyOrder(setupHelper.registrar1, setupHelper.registrar2);
|
||||
PAssert.that(tuple.get(ValidateSqlUtils.createSqlEntityTupleTag(DomainHistory.class)))
|
||||
.containsInAnyOrder(setupHelper.historyEntry);
|
||||
testPipeline.run();
|
||||
}
|
||||
}
|
|
@ -1,101 +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.comparedb;
|
||||
|
||||
import static google.registry.beam.comparedb.ValidateSqlUtils.createSqlEntityTupleTag;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.beam.TestPipelineExtension;
|
||||
import google.registry.model.bulkquery.TestSetupHelper;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.domain.DomainHistory;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.model.registrar.Registrar;
|
||||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.model.tld.Registry;
|
||||
import google.registry.testing.AppEngineExtension;
|
||||
import google.registry.testing.DatastoreEntityExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import java.io.Serializable;
|
||||
import java.util.Optional;
|
||||
import org.apache.beam.sdk.coders.SerializableCoder;
|
||||
import org.apache.beam.sdk.testing.PAssert;
|
||||
import org.apache.beam.sdk.values.PCollectionTuple;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link SqlSnapshots}. */
|
||||
class SqlSnapshotsTest {
|
||||
|
||||
protected FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
|
||||
@RegisterExtension
|
||||
@Order(Order.DEFAULT - 1)
|
||||
final transient DatastoreEntityExtension datastore =
|
||||
new DatastoreEntityExtension().allThreads(true);
|
||||
|
||||
@RegisterExtension
|
||||
public final AppEngineExtension appEngine =
|
||||
AppEngineExtension.builder().withDatastoreAndCloudSql().withClock(fakeClock).build();
|
||||
|
||||
@RegisterExtension
|
||||
final transient TestPipelineExtension testPipeline =
|
||||
TestPipelineExtension.create().enableAbandonedNodeEnforcement(true);
|
||||
|
||||
private final TestSetupHelper setupHelper = new TestSetupHelper(fakeClock);
|
||||
|
||||
@BeforeEach
|
||||
void setUp() {
|
||||
testPipeline
|
||||
.getCoderRegistry()
|
||||
.registerCoderForClass(SqlEntity.class, SerializableCoder.of(Serializable.class));
|
||||
setupHelper.initializeAllEntities();
|
||||
setupHelper.setupBulkQueryJpaTm(appEngine);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
void afterEach() {
|
||||
setupHelper.tearDownBulkQueryJpaTm();
|
||||
}
|
||||
|
||||
@Test
|
||||
void loadCloudSqlSnapshotByType() {
|
||||
PCollectionTuple sqlSnapshot =
|
||||
SqlSnapshots.loadCloudSqlSnapshotByType(
|
||||
testPipeline,
|
||||
ImmutableSet.of(
|
||||
Registry.class,
|
||||
Registrar.class,
|
||||
DomainBase.class,
|
||||
DomainHistory.class,
|
||||
ContactResource.class,
|
||||
HostResource.class),
|
||||
Optional.empty(),
|
||||
Optional.empty());
|
||||
PAssert.that(sqlSnapshot.get(createSqlEntityTupleTag(Registry.class)))
|
||||
.containsInAnyOrder(setupHelper.registry);
|
||||
PAssert.that(sqlSnapshot.get(createSqlEntityTupleTag(DomainBase.class)))
|
||||
.containsInAnyOrder(setupHelper.domain);
|
||||
PAssert.that(sqlSnapshot.get(createSqlEntityTupleTag(DomainHistory.class)))
|
||||
.containsInAnyOrder(setupHelper.domainHistory);
|
||||
testPipeline.run();
|
||||
}
|
||||
}
|
|
@ -1,96 +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.comparedb;
|
||||
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.beam.comparedb.ValidateSqlUtils.getMedianIdForHistoryTable;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.truth.Truth;
|
||||
import google.registry.beam.comparedb.ValidateSqlUtils.DiffableFieldNormalizer;
|
||||
import google.registry.model.bulkquery.TestSetupHelper;
|
||||
import google.registry.model.contact.ContactAddress;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.model.contact.PostalInfo;
|
||||
import google.registry.model.domain.DomainHistory;
|
||||
import google.registry.testing.AppEngineExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import google.registry.util.DiffUtils;
|
||||
import java.util.Map;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ValidateSqlUtils}. */
|
||||
class ValidateSqlUtilsTest {
|
||||
|
||||
private final FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
|
||||
private final TestSetupHelper setupHelper = new TestSetupHelper(fakeClock);
|
||||
|
||||
@RegisterExtension
|
||||
public final AppEngineExtension appEngine =
|
||||
AppEngineExtension.builder().withDatastoreAndCloudSql().withClock(fakeClock).build();
|
||||
|
||||
@Test
|
||||
void getMedianIdForHistoryTable_emptyTable() {
|
||||
assertThat(getMedianIdForHistoryTable("DomainHistory")).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void getMedianIdForHistoryTable_oneRow() {
|
||||
setupHelper.initializeAllEntities();
|
||||
Truth.assertThat(jpaTm().transact(() -> jpaTm().loadAllOf(DomainHistory.class))).hasSize(1);
|
||||
assertThat(getMedianIdForHistoryTable("DomainHistory"))
|
||||
.hasValue(setupHelper.domainHistory.getId());
|
||||
}
|
||||
|
||||
@Test
|
||||
void getMedianIdForHistoryTable_twoRows() {
|
||||
setupHelper.initializeAllEntities();
|
||||
setupHelper.applyChangeToDomainAndHistory();
|
||||
Truth.assertThat(jpaTm().transact(() -> jpaTm().loadAllOf(DomainHistory.class))).hasSize(2);
|
||||
assertThat(getMedianIdForHistoryTable("DomainHistory"))
|
||||
.hasValue(setupHelper.domainHistory.getId());
|
||||
}
|
||||
|
||||
@Test
|
||||
void diffableFieldNormalizer() {
|
||||
ContactResource contactResource =
|
||||
new ContactResource.Builder()
|
||||
.setLocalizedPostalInfo(
|
||||
new PostalInfo.Builder()
|
||||
.setType(PostalInfo.Type.LOCALIZED)
|
||||
.setAddress(
|
||||
new ContactAddress.Builder()
|
||||
.setStreet(ImmutableList.of("111 8th Ave", ""))
|
||||
.setCity("New York")
|
||||
.setState("NY")
|
||||
.setZip("10011")
|
||||
.setCountryCode("US")
|
||||
.build())
|
||||
.build())
|
||||
.build();
|
||||
Map<String, Object> origMap = contactResource.toDiffableFieldMap();
|
||||
Map<String, Object> trimmedMap = Maps.transformEntries(origMap, new DiffableFieldNormalizer());
|
||||
// In the trimmed map, localizedPostalInfo.address.street only has one element in the list,
|
||||
// thus the output: 'null -> '
|
||||
Truth.assertThat(DiffUtils.prettyPrintEntityDeepDiff(trimmedMap, origMap))
|
||||
.isEqualTo("localizedPostalInfo.address.street.1: null -> \n");
|
||||
}
|
||||
}
|
|
@ -91,8 +91,7 @@ public abstract class FlowTestCase<F extends Flow> {
|
|||
private EppMetric.Builder eppMetricBuilder;
|
||||
|
||||
// Set the clock for transactional flows. We have to order this before the AppEngineExtension
|
||||
// which populates data (and may do so with clock-dependent commit logs if mixed with
|
||||
// ReplayExtension).
|
||||
// which populates data.
|
||||
@Order(value = Order.DEFAULT - 1)
|
||||
@RegisterExtension
|
||||
final InjectExtension inject =
|
||||
|
@ -108,6 +107,7 @@ public abstract class FlowTestCase<F extends Flow> {
|
|||
|
||||
@BeforeEach
|
||||
public void beforeEachFlowTestCase() {
|
||||
DatabaseHelper.setClock(clock);
|
||||
sessionMetadata = new HttpSessionMetadata(new FakeHttpSession());
|
||||
sessionMetadata.setRegistrarId("TheRegistrar");
|
||||
sessionMetadata.setServiceExtensionUris(ProtocolDefinition.getVisibleServiceExtensionUris());
|
||||
|
|
|
@ -26,19 +26,12 @@ import google.registry.flows.ResourceCheckFlowTestCase;
|
|||
import google.registry.flows.exceptions.TooManyResourceChecksException;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactCheckFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactCheckFlowTest extends ResourceCheckFlowTestCase<ContactCheckFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
ContactCheckFlowTest() {
|
||||
setEppInput("contact_check.xml");
|
||||
}
|
||||
|
|
|
@ -35,21 +35,14 @@ import google.registry.flows.exceptions.ResourceCreateContentionException;
|
|||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactCreateFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactCreateFlowTest extends ResourceFlowTestCase<ContactCreateFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
ContactCreateFlowTest() {
|
||||
setEppInput("contact_create.xml");
|
||||
clock.setTo(DateTime.parse("1999-04-03T22:00:00.0Z"));
|
||||
|
|
|
@ -54,23 +54,16 @@ import google.registry.model.transfer.TransferResponse;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import google.registry.testing.TestSqlOnly;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactDeleteFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactDeleteFlowTest extends ResourceFlowTestCase<ContactDeleteFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void initFlowTest() {
|
||||
setEppInput("contact_delete.xml");
|
||||
|
|
|
@ -41,20 +41,13 @@ import google.registry.model.eppcommon.AuthInfo.PasswordAuth;
|
|||
import google.registry.model.eppcommon.PresenceMarker;
|
||||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactInfoFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactInfoFlowTest extends ResourceFlowTestCase<ContactInfoFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
ContactInfoFlowTest() {
|
||||
setEppInput("contact_info.xml");
|
||||
}
|
||||
|
|
|
@ -44,22 +44,15 @@ import google.registry.model.transfer.TransferResponse;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactTransferApproveFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactTransferApproveFlowTest
|
||||
extends ContactTransferFlowTestCase<ContactTransferApproveFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void setUp() {
|
||||
setEppInput("contact_transfer_approve.xml");
|
||||
|
|
|
@ -41,22 +41,15 @@ import google.registry.model.transfer.TransferResponse;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactTransferCancelFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactTransferCancelFlowTest
|
||||
extends ContactTransferFlowTestCase<ContactTransferCancelFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void setUp() {
|
||||
this.setEppInput("contact_transfer_cancel.xml");
|
||||
|
|
|
@ -33,22 +33,15 @@ import google.registry.model.eppcommon.AuthInfo.PasswordAuth;
|
|||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactTransferQueryFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactTransferQueryFlowTest
|
||||
extends ContactTransferFlowTestCase<ContactTransferQueryFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void setUp() {
|
||||
setEppInput("contact_transfer_query.xml");
|
||||
|
|
|
@ -43,22 +43,15 @@ import google.registry.model.transfer.TransferResponse;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactTransferRejectFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactTransferRejectFlowTest
|
||||
extends ContactTransferFlowTestCase<ContactTransferRejectFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void setUp() {
|
||||
setEppInput("contact_transfer_reject.xml");
|
||||
|
|
|
@ -54,23 +54,16 @@ import google.registry.model.transfer.ContactTransferData;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactTransferRequestFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactTransferRequestFlowTest
|
||||
extends ContactTransferFlowTestCase<ContactTransferRequestFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
ContactTransferRequestFlowTest() {
|
||||
// We need the transfer to happen at exactly this time in order for the response to match up.
|
||||
clock.setTo(DateTime.parse("2000-06-08T22:00:00.0Z"));
|
||||
|
|
|
@ -45,20 +45,13 @@ import google.registry.model.contact.PostalInfo.Type;
|
|||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ContactUpdateFlow}. */
|
||||
@DualDatabaseTest
|
||||
class ContactUpdateFlowTest extends ResourceFlowTestCase<ContactUpdateFlow, ContactResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
ContactUpdateFlowTest() {
|
||||
setEppInput("contact_update.xml");
|
||||
}
|
||||
|
|
|
@ -75,7 +75,6 @@ import google.registry.model.tld.Registry;
|
|||
import google.registry.model.tld.Registry.TldState;
|
||||
import google.registry.model.tld.label.ReservedList;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.SetClockExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.joda.money.CurrencyUnit;
|
||||
|
@ -94,10 +93,6 @@ class DomainCheckFlowTest extends ResourceCheckFlowTestCase<DomainCheckFlow, Dom
|
|||
@RegisterExtension
|
||||
final SetClockExtension setClockExtension = new SetClockExtension(clock, "2009-01-01T10:00:00Z");
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
DomainCheckFlowTest() {
|
||||
setEppInput("domain_check_one_tld.xml");
|
||||
}
|
||||
|
|
|
@ -42,22 +42,15 @@ import google.registry.model.domain.DomainBase;
|
|||
import google.registry.model.tld.Registry;
|
||||
import google.registry.model.tld.Registry.TldState;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.joda.money.Money;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainClaimsCheckFlow}. */
|
||||
@DualDatabaseTest
|
||||
public class DomainClaimsCheckFlowTest
|
||||
extends ResourceFlowTestCase<DomainClaimsCheckFlow, DomainBase> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
DomainClaimsCheckFlowTest() {
|
||||
setEppInput("domain_check_claims.xml");
|
||||
}
|
||||
|
|
|
@ -49,7 +49,6 @@ import static google.registry.testing.DatabaseHelper.newHostResource;
|
|||
import static google.registry.testing.DatabaseHelper.persistActiveContact;
|
||||
import static google.registry.testing.DatabaseHelper.persistActiveDomain;
|
||||
import static google.registry.testing.DatabaseHelper.persistActiveHost;
|
||||
import static google.registry.testing.DatabaseHelper.persistDeletedDomain;
|
||||
import static google.registry.testing.DatabaseHelper.persistReservedList;
|
||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||
import static google.registry.testing.DomainBaseSubject.assertAboutDomains;
|
||||
|
@ -178,7 +177,6 @@ import google.registry.monitoring.whitebox.EppMetric;
|
|||
import google.registry.persistence.VKey;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TaskQueueHelper.TaskMatcher;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
|
@ -190,8 +188,6 @@ import org.joda.money.Money;
|
|||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainCreateFlow}. */
|
||||
@DualDatabaseTest
|
||||
|
@ -201,10 +197,6 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
|||
|
||||
private AllocationToken allocationToken;
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
DomainCreateFlowTest() {
|
||||
setEppInput("domain_create.xml", ImmutableMap.of("DOMAIN", "example.tld"));
|
||||
clock.setTo(DateTime.parse("1999-04-03T22:00:00.0Z").minus(Duration.millis(1)));
|
||||
|
@ -247,6 +239,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
|||
}
|
||||
persistActiveContact("jd1234");
|
||||
persistActiveContact("sh8013");
|
||||
clock.advanceOneMilli();
|
||||
}
|
||||
|
||||
private void persistContactsAndHosts() {
|
||||
|
@ -368,12 +361,6 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
|||
GracePeriodStatus.ADD, domain.getRepoId(), billingTime, "TheRegistrar", null),
|
||||
createBillingEvent));
|
||||
assertDnsTasksEnqueued(getUniqueIdFromCommand());
|
||||
|
||||
replayExtension.expectUpdateFor(domain);
|
||||
|
||||
// Verify that all timestamps are correct after SQL -> DS replay.
|
||||
// Added to confirm that timestamps get updated correctly.
|
||||
replayExtension.enableDomainTimestampChecks();
|
||||
}
|
||||
|
||||
private void assertNoLordn() throws Exception {
|
||||
|
@ -393,7 +380,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
|||
.hasLaunchNotice(null);
|
||||
String expectedPayload =
|
||||
String.format(
|
||||
"%s,%s,0000001761376042759136-65535,1,2014-09-09T09:09:09.016Z",
|
||||
"%s,%s,0000001761376042759136-65535,1,2014-09-09T09:09:09.017Z",
|
||||
reloadResourceByForeignKey().getRepoId(), domainName);
|
||||
assertTasksEnqueued(QUEUE_SUNRISE, new TaskMatcher().payload(expectedPayload));
|
||||
}
|
||||
|
@ -414,7 +401,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
|||
.payload(
|
||||
reloadResourceByForeignKey().getRepoId()
|
||||
+ ",example-one.tld,370d0b7c9223372036854775807,1,"
|
||||
+ "2009-08-16T09:00:00.016Z,2009-08-16T09:00:00.000Z");
|
||||
+ "2009-08-16T09:00:00.017Z,2009-08-16T09:00:00.000Z");
|
||||
assertTasksEnqueued(QUEUE_CLAIMS, task);
|
||||
}
|
||||
|
||||
|
@ -602,7 +589,6 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
|||
ImmutableMap.of("DOMAIN", "otherexample.tld", "YEARS", "2"));
|
||||
runFlowAssertResponse(
|
||||
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "otherexample.tld")));
|
||||
replayExtension.expectUpdateFor(reloadResourceByForeignKey());
|
||||
}
|
||||
|
||||
@TestOfyAndSql
|
||||
|
@ -855,8 +841,6 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
|||
@TestOfyAndSql
|
||||
void testSuccess_existedButWasDeleted() throws Exception {
|
||||
persistContactsAndHosts();
|
||||
replayExtension.expectUpdateFor(
|
||||
persistDeletedDomain(getUniqueIdFromCommand(), clock.nowUtc().minusDays(1)));
|
||||
clock.advanceOneMilli();
|
||||
doSuccessfulTest();
|
||||
}
|
||||
|
|
|
@ -104,7 +104,6 @@ import google.registry.model.transfer.TransferStatus;
|
|||
import google.registry.testing.CloudTasksHelper.TaskMatcher;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import java.util.Map;
|
||||
|
@ -112,17 +111,11 @@ import org.joda.money.Money;
|
|||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainDeleteFlow}. */
|
||||
@DualDatabaseTest
|
||||
class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, DomainBase> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
private DomainBase domain;
|
||||
private DomainHistory earlierHistoryEntry;
|
||||
|
||||
|
@ -855,7 +848,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
|
|||
.setEventTime(now)
|
||||
.setMsg(
|
||||
"Domain example.tld was deleted by registry administrator with final deletion"
|
||||
+ " effective: 2000-07-11T22:00:00.013Z")
|
||||
+ " effective: 2000-07-11T22:00:00.012Z")
|
||||
.setResponseData(
|
||||
ImmutableList.of(
|
||||
DomainPendingActionNotificationResponse.create(
|
||||
|
@ -864,7 +857,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
|
|||
new PollMessage.OneTime.Builder()
|
||||
.setRegistrarId("TheRegistrar")
|
||||
.setParent(deleteHistoryEntry)
|
||||
.setEventTime(DateTime.parse("2000-07-11T22:00:00.013Z"))
|
||||
.setEventTime(DateTime.parse("2000-07-11T22:00:00.012Z"))
|
||||
.setMsg("Deleted by registry administrator.")
|
||||
.setResponseData(
|
||||
ImmutableList.of(
|
||||
|
@ -872,7 +865,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
|
|||
"example.tld",
|
||||
true,
|
||||
deleteHistoryEntry.getTrid(),
|
||||
DateTime.parse("2000-07-11T22:00:00.013Z"))))
|
||||
DateTime.parse("2000-07-11T22:00:00.012Z"))))
|
||||
.build());
|
||||
}
|
||||
|
||||
|
|
|
@ -68,7 +68,6 @@ import google.registry.model.tld.Registry;
|
|||
import google.registry.persistence.VKey;
|
||||
import google.registry.testing.AppEngineExtension;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.SetClockExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
|
@ -86,10 +85,6 @@ class DomainInfoFlowTest extends ResourceFlowTestCase<DomainInfoFlow, DomainBase
|
|||
final SetClockExtension setClockExtension =
|
||||
new SetClockExtension(clock, "2005-03-03T22:00:00.000Z");
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
/**
|
||||
* The domain_info_fee.xml default substitutions common to most tests.
|
||||
*
|
||||
|
|
|
@ -15,7 +15,6 @@
|
|||
package google.registry.flows.domain;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.model.billing.BillingEvent.Flag.AUTO_RENEW;
|
||||
import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.DEFAULT;
|
||||
import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.NONPREMIUM;
|
||||
|
|
|
@ -79,7 +79,6 @@ import google.registry.model.reporting.HistoryEntry;
|
|||
import google.registry.model.tld.Registry;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.SetClockExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
|
@ -117,10 +116,6 @@ class DomainRenewFlowTest extends ResourceFlowTestCase<DomainRenewFlow, DomainBa
|
|||
final SetClockExtension setClockExtension =
|
||||
new SetClockExtension(clock, expirationTime.minusMillis(20));
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void initDomainTest() {
|
||||
createTld("tld");
|
||||
|
|
|
@ -78,7 +78,6 @@ import google.registry.model.reporting.HistoryEntry;
|
|||
import google.registry.model.tld.Registry;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import java.util.Map;
|
||||
|
@ -86,18 +85,12 @@ import java.util.Optional;
|
|||
import org.joda.money.Money;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainRestoreRequestFlow}. */
|
||||
@DualDatabaseTest
|
||||
class DomainRestoreRequestFlowTest
|
||||
extends ResourceFlowTestCase<DomainRestoreRequestFlow, DomainBase> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
private static final ImmutableMap<String, String> FEE_06_MAP =
|
||||
ImmutableMap.of("FEE_VERSION", "0.6", "FEE_NS", "fee", "CURRENCY", "USD");
|
||||
private static final ImmutableMap<String, String> FEE_11_MAP =
|
||||
|
|
|
@ -76,7 +76,6 @@ import google.registry.model.transfer.TransferStatus;
|
|||
import google.registry.persistence.VKey;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import java.util.Arrays;
|
||||
|
@ -85,18 +84,12 @@ import org.joda.money.Money;
|
|||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainTransferApproveFlow}. */
|
||||
@DualDatabaseTest
|
||||
class DomainTransferApproveFlowTest
|
||||
extends DomainTransferFlowTestCase<DomainTransferApproveFlow, DomainBase> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
setEppInput("domain_transfer_approve.xml");
|
||||
|
|
|
@ -59,24 +59,17 @@ import google.registry.model.transfer.TransferResponse.DomainTransferResponse;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainTransferCancelFlow}. */
|
||||
@DualDatabaseTest
|
||||
class DomainTransferCancelFlowTest
|
||||
extends DomainTransferFlowTestCase<DomainTransferCancelFlow, DomainBase> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
setEppInput("domain_transfer_cancel.xml");
|
||||
|
|
|
@ -36,21 +36,14 @@ import google.registry.model.eppcommon.AuthInfo.PasswordAuth;
|
|||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainTransferQueryFlow}. */
|
||||
@DualDatabaseTest
|
||||
class DomainTransferQueryFlowTest
|
||||
extends DomainTransferFlowTestCase<DomainTransferQueryFlow, DomainBase> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
setEppInput("domain_transfer_query.xml");
|
||||
|
|
|
@ -61,24 +61,17 @@ import google.registry.model.transfer.TransferResponse;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainTransferRejectFlow}. */
|
||||
@DualDatabaseTest
|
||||
class DomainTransferRejectFlowTest
|
||||
extends DomainTransferFlowTestCase<DomainTransferRejectFlow, DomainBase> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
setEppInput("domain_transfer_reject.xml");
|
||||
|
|
|
@ -110,7 +110,6 @@ import google.registry.persistence.VKey;
|
|||
import google.registry.testing.CloudTasksHelper.TaskMatcher;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import java.util.Map;
|
||||
|
@ -120,18 +119,12 @@ import org.joda.money.Money;
|
|||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainTransferRequestFlow}. */
|
||||
@DualDatabaseTest
|
||||
class DomainTransferRequestFlowTest
|
||||
extends DomainTransferFlowTestCase<DomainTransferRequestFlow, DomainBase> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
private static final ImmutableMap<String, String> BASE_FEE_MAP =
|
||||
new ImmutableMap.Builder<String, String>()
|
||||
.put("DOMAIN", "example.tld")
|
||||
|
|
|
@ -108,15 +108,12 @@ import google.registry.model.tld.Registry;
|
|||
import google.registry.persistence.VKey;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import java.util.Optional;
|
||||
import org.joda.money.Money;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainUpdateFlow}. */
|
||||
@DualDatabaseTest
|
||||
|
@ -139,10 +136,6 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
|||
private ContactResource mak21Contact;
|
||||
private ContactResource unusedContact;
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
createTld("tld");
|
||||
|
@ -206,7 +199,6 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
|||
.setDomain(domain)
|
||||
.build());
|
||||
clock.advanceOneMilli();
|
||||
replayExtension.expectUpdateFor(domain);
|
||||
return domain;
|
||||
}
|
||||
|
||||
|
@ -218,7 +210,6 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
|||
assertTransactionalFlow(true);
|
||||
runFlowAssertResponse(loadFile(expectedXmlFilename));
|
||||
DomainBase domain = reloadResourceByForeignKey();
|
||||
replayExtension.expectUpdateFor(domain);
|
||||
// Check that the domain was updated. These values came from the xml.
|
||||
assertAboutDomains()
|
||||
.that(domain)
|
||||
|
@ -236,10 +227,6 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
|||
assertNoBillingEvents();
|
||||
assertDnsTasksEnqueued("example.tld");
|
||||
assertLastHistoryContainsResource(reloadResourceByForeignKey());
|
||||
|
||||
// Verify that all timestamps are correct after SQL -> DS replay.
|
||||
// Added to confirm that timestamps get updated correctly.
|
||||
replayExtension.enableDomainTimestampChecks();
|
||||
}
|
||||
|
||||
@TestOfyAndSql
|
||||
|
@ -319,9 +306,6 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
|||
}
|
||||
persistResource(
|
||||
reloadResourceByForeignKey().asBuilder().setNameservers(nameservers.build()).build());
|
||||
// Add a null update here so we don't compare.
|
||||
replayExtension.expectUpdateFor(null);
|
||||
clock.advanceOneMilli();
|
||||
}
|
||||
|
||||
@TestOfyAndSql
|
||||
|
|
|
@ -26,19 +26,12 @@ import google.registry.flows.ResourceCheckFlowTestCase;
|
|||
import google.registry.flows.exceptions.TooManyResourceChecksException;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link HostCheckFlow}. */
|
||||
@DualDatabaseTest
|
||||
class HostCheckFlowTest extends ResourceCheckFlowTestCase<HostCheckFlow, HostResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
HostCheckFlowTest() {
|
||||
setEppInput("host_check.xml");
|
||||
}
|
||||
|
|
|
@ -57,21 +57,14 @@ import google.registry.model.host.HostResource;
|
|||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link HostCreateFlow}. */
|
||||
@DualDatabaseTest
|
||||
class HostCreateFlowTest extends ResourceFlowTestCase<HostCreateFlow, HostResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
private void setEppHostCreateInput(String hostName, String hostAddrs) {
|
||||
setEppInput(
|
||||
"host_create.xml",
|
||||
|
|
|
@ -53,22 +53,15 @@ import google.registry.model.transfer.DomainTransferData;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link HostDeleteFlow}. */
|
||||
@DualDatabaseTest
|
||||
class HostDeleteFlowTest extends ResourceFlowTestCase<HostDeleteFlow, HostResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void initFlowTest() {
|
||||
setEppInput("host_delete.xml", ImmutableMap.of("HOSTNAME", "ns1.example.tld"));
|
||||
|
|
|
@ -38,22 +38,15 @@ import google.registry.model.domain.DomainBase;
|
|||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import javax.annotation.Nullable;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link HostInfoFlow}. */
|
||||
@DualDatabaseTest
|
||||
class HostInfoFlowTest extends ResourceFlowTestCase<HostInfoFlow, HostResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
@BeforeEach
|
||||
void initHostTest() {
|
||||
createTld("foobar");
|
||||
|
|
|
@ -81,23 +81,16 @@ import google.registry.model.transfer.DomainTransferData;
|
|||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TaskQueueHelper.TaskMatcher;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import javax.annotation.Nullable;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link HostUpdateFlow}. */
|
||||
@DualDatabaseTest
|
||||
class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, HostResource> {
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
private void setEppHostUpdateInput(
|
||||
String oldHostName, String newHostName, String ipOrStatusToAdd, String ipOrStatusToRem) {
|
||||
setEppInput(
|
||||
|
|
|
@ -33,7 +33,6 @@ import google.registry.model.contact.ContactResource;
|
|||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.poll.PollMessage;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.SetClockExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -49,10 +48,6 @@ class PollAckFlowTest extends FlowTestCase<PollAckFlow> {
|
|||
@RegisterExtension
|
||||
final SetClockExtension setClockExtension = new SetClockExtension(clock, "2011-01-02T01:01:01Z");
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
/** This is the message id being sent in the ACK request. */
|
||||
private static final long MESSAGE_ID = 3;
|
||||
|
||||
|
|
|
@ -41,7 +41,6 @@ import google.registry.model.transfer.TransferResponse.ContactTransferResponse;
|
|||
import google.registry.model.transfer.TransferResponse.DomainTransferResponse;
|
||||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.SetClockExtension;
|
||||
import google.registry.testing.TestOfyAndSql;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
|
@ -56,10 +55,6 @@ class PollRequestFlowTest extends FlowTestCase<PollRequestFlow> {
|
|||
@RegisterExtension
|
||||
final SetClockExtension setClockExtension = new SetClockExtension(clock, "2011-01-02T01:01:01Z");
|
||||
|
||||
@Order(value = Order.DEFAULT - 2)
|
||||
@RegisterExtension
|
||||
final ReplayExtension replayExtension = ReplayExtension.createWithDoubleReplay(clock);
|
||||
|
||||
private DomainBase domain;
|
||||
private ContactResource contact;
|
||||
private HostResource host;
|
||||
|
|
|
@ -18,7 +18,6 @@ import static com.google.common.truth.Truth.assertThat;
|
|||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||
import static google.registry.model.ImmutableObjectSubject.assertAboutImmutableObjects;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||
import static google.registry.testing.ContactResourceSubject.assertAboutContacts;
|
||||
import static google.registry.testing.DatabaseHelper.cloneAndSetAutoTimestamps;
|
||||
import static google.registry.testing.DatabaseHelper.createTld;
|
||||
|
@ -31,7 +30,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.model.EntityTestCase;
|
||||
import google.registry.model.contact.Disclose.PostalInfoChoice;
|
||||
import google.registry.model.contact.PostalInfo.Type;
|
||||
|
@ -39,9 +37,6 @@ import google.registry.model.eppcommon.AuthInfo.PasswordAuth;
|
|||
import google.registry.model.eppcommon.PresenceMarker;
|
||||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.model.eppcommon.Trid;
|
||||
import google.registry.model.index.EppResourceIndex;
|
||||
import google.registry.model.index.ForeignKeyIndex;
|
||||
import google.registry.model.index.ForeignKeyIndex.ForeignKeyContactIndex;
|
||||
import google.registry.model.transfer.ContactTransferData;
|
||||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
|
@ -287,28 +282,4 @@ public class ContactResourceTest extends EntityTestCase {
|
|||
// If there are circular references, this will overflow the stack.
|
||||
contactResource.toHydratedString();
|
||||
}
|
||||
|
||||
@TestOfyAndSql
|
||||
void testBeforeDatastoreSaveOnReplay_indexes() {
|
||||
ImmutableList<ForeignKeyContactIndex> foreignKeyIndexes =
|
||||
ofyTm().loadAllOf(ForeignKeyContactIndex.class);
|
||||
ImmutableList<EppResourceIndex> eppResourceIndexes = ofyTm().loadAllOf(EppResourceIndex.class);
|
||||
fakeClock.advanceOneMilli();
|
||||
ofyTm()
|
||||
.transact(
|
||||
() -> {
|
||||
foreignKeyIndexes.forEach(ofyTm()::delete);
|
||||
eppResourceIndexes.forEach(ofyTm()::delete);
|
||||
});
|
||||
assertThat(ofyTm().loadAllOf(ForeignKeyContactIndex.class)).isEmpty();
|
||||
assertThat(ofyTm().loadAllOf(EppResourceIndex.class)).isEmpty();
|
||||
|
||||
ofyTm().transact(() -> contactResource.beforeDatastoreSaveOnReplay());
|
||||
|
||||
assertThat(ofyTm().loadAllOf(ForeignKeyContactIndex.class))
|
||||
.containsExactly(
|
||||
ForeignKeyIndex.create(contactResource, contactResource.getDeletionTime()));
|
||||
assertThat(ofyTm().loadAllOf(EppResourceIndex.class))
|
||||
.containsExactly(EppResourceIndex.create(Key.create(contactResource)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@ import static com.google.common.collect.Iterables.getOnlyElement;
|
|||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||
import static google.registry.testing.DatabaseHelper.cloneAndSetAutoTimestamps;
|
||||
import static google.registry.testing.DatabaseHelper.createTld;
|
||||
import static google.registry.testing.DatabaseHelper.newDomainBase;
|
||||
|
@ -52,9 +51,6 @@ import google.registry.model.eppcommon.AuthInfo.PasswordAuth;
|
|||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.model.eppcommon.Trid;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.model.index.EppResourceIndex;
|
||||
import google.registry.model.index.ForeignKeyIndex;
|
||||
import google.registry.model.index.ForeignKeyIndex.ForeignKeyDomainIndex;
|
||||
import google.registry.model.poll.PollMessage;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.model.tld.Registry;
|
||||
|
@ -899,42 +895,6 @@ public class DomainBaseTest extends EntityTestCase {
|
|||
new BillEventInfo(recurringBillKey, null)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testBeforeDatastoreSaveOnReplay_indexes() {
|
||||
ImmutableList<ForeignKeyDomainIndex> foreignKeyIndexes =
|
||||
ofyTm().loadAllOf(ForeignKeyDomainIndex.class);
|
||||
ImmutableList<EppResourceIndex> eppResourceIndexes = ofyTm().loadAllOf(EppResourceIndex.class);
|
||||
fakeClock.advanceOneMilli();
|
||||
ofyTm()
|
||||
.transact(
|
||||
() -> {
|
||||
foreignKeyIndexes.forEach(ofyTm()::delete);
|
||||
eppResourceIndexes.forEach(ofyTm()::delete);
|
||||
});
|
||||
assertThat(ofyTm().loadAllOf(ForeignKeyDomainIndex.class)).isEmpty();
|
||||
assertThat(ofyTm().loadAllOf(EppResourceIndex.class)).isEmpty();
|
||||
|
||||
ofyTm().transact(() -> domain.beforeDatastoreSaveOnReplay());
|
||||
|
||||
assertThat(ofyTm().loadAllOf(ForeignKeyDomainIndex.class))
|
||||
.containsExactly(ForeignKeyIndex.create(domain, domain.getDeletionTime()));
|
||||
assertThat(ofyTm().loadAllOf(EppResourceIndex.class))
|
||||
.containsExactly(EppResourceIndex.create(Key.create(domain)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testBeforeSqlSaveOnReplay_canonicalName() {
|
||||
domain.fullyQualifiedDomainName = "EXAMPLE.COM";
|
||||
assertThat(domain.getDomainName()).isEqualTo("EXAMPLE.COM");
|
||||
domain.beforeSqlSaveOnReplay();
|
||||
assertThat(domain.getDomainName()).isEqualTo("example.com");
|
||||
|
||||
domain.fullyQualifiedDomainName = "kittyçat.com";
|
||||
assertThat(domain.getDomainName()).isEqualTo("kittyçat.com");
|
||||
domain.beforeSqlSaveOnReplay();
|
||||
assertThat(domain.getDomainName()).isEqualTo("xn--kittyat-yxa.com");
|
||||
}
|
||||
|
||||
static class BillEventInfo extends ImmutableObject {
|
||||
VKey<BillingEvent.Recurring> billingEventRecurring;
|
||||
Long billingEventRecurringHistoryId;
|
||||
|
|
|
@ -20,7 +20,6 @@ import static google.registry.persistence.transaction.TransactionManagerFactory.
|
|||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.testing.DatabaseHelper.insertInDb;
|
||||
import static google.registry.testing.DatabaseHelper.loadByEntity;
|
||||
import static google.registry.testing.DatabaseHelper.newContactResource;
|
||||
import static google.registry.testing.DatabaseHelper.newContactResourceWithRoid;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
|
||||
|
@ -117,36 +116,6 @@ public class ContactHistoryTest extends EntityTestCase {
|
|||
assertThat(hostHistoryFromDb).isEqualTo(historyEntryFromDb);
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testBeforeSqlSave_afterContactPersisted() {
|
||||
ContactResource contactResource = newContactResource("contactId");
|
||||
ContactHistory contactHistory =
|
||||
new ContactHistory.Builder()
|
||||
.setType(HistoryEntry.Type.HOST_CREATE)
|
||||
.setXmlBytes("<xml></xml>".getBytes(UTF_8))
|
||||
.setModificationTime(fakeClock.nowUtc())
|
||||
.setRegistrarId("TheRegistrar")
|
||||
.setTrid(Trid.create("ABC-123", "server-trid"))
|
||||
.setBySuperuser(false)
|
||||
.setReason("reason")
|
||||
.setRequestedByRegistrar(true)
|
||||
.setContactRepoId(contactResource.getRepoId())
|
||||
.build();
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
jpaTm().put(contactResource);
|
||||
contactHistory.beforeSqlSaveOnReplay();
|
||||
jpaTm().put(contactHistory);
|
||||
});
|
||||
jpaTm()
|
||||
.transact(
|
||||
() ->
|
||||
assertAboutImmutableObjects()
|
||||
.that(jpaTm().loadByEntity(contactResource))
|
||||
.hasFieldsEqualTo(jpaTm().loadByEntity(contactHistory).getContactBase().get()));
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testWipeOutPii_assertsAllPiiFieldsAreNull() {
|
||||
ContactHistory originalEntity =
|
||||
|
|
|
@ -55,7 +55,6 @@ import google.registry.testing.DualDatabaseTest;
|
|||
import google.registry.testing.TestOfyOnly;
|
||||
import google.registry.testing.TestSqlOnly;
|
||||
import google.registry.util.SerializeUtils;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.Optional;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
|
@ -193,135 +192,6 @@ public class DomainHistoryTest extends EntityTestCase {
|
|||
putInDb(domainHistoryFromDb2);
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testBeforeSqlSave_afterDomainPersisted() {
|
||||
DomainBase domain = createDomainWithContactsAndHosts();
|
||||
DomainHistory historyWithoutResource =
|
||||
new DomainHistory.Builder()
|
||||
.setType(HistoryEntry.Type.DOMAIN_CREATE)
|
||||
.setXmlBytes("<xml></xml>".getBytes(UTF_8))
|
||||
.setModificationTime(fakeClock.nowUtc())
|
||||
.setRegistrarId("TheRegistrar")
|
||||
.setTrid(Trid.create("ABC-123", "server-trid"))
|
||||
.setBySuperuser(false)
|
||||
.setReason("reason")
|
||||
.setRequestedByRegistrar(true)
|
||||
.setDomainRepoId(domain.getRepoId())
|
||||
.setOtherRegistrarId("otherClient")
|
||||
.setPeriod(Period.create(1, Period.Unit.YEARS))
|
||||
.build();
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
jpaTm()
|
||||
.put(
|
||||
domain
|
||||
.asBuilder()
|
||||
.setPersistedCurrentSponsorRegistrarId("NewRegistrar")
|
||||
.build());
|
||||
historyWithoutResource.beforeSqlSaveOnReplay();
|
||||
jpaTm().put(historyWithoutResource);
|
||||
});
|
||||
jpaTm()
|
||||
.transact(
|
||||
() ->
|
||||
assertAboutImmutableObjects()
|
||||
.that(jpaTm().loadByEntity(domain))
|
||||
.hasFieldsEqualTo(
|
||||
jpaTm().loadByEntity(historyWithoutResource).getDomainContent().get()));
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testBeforeSqlSave_canonicalNameUncapitalized() throws Exception {
|
||||
Field domainNameField = DomainContent.class.getDeclaredField("fullyQualifiedDomainName");
|
||||
// reflection hacks to get around visibility issues
|
||||
domainNameField.setAccessible(true);
|
||||
DomainBase domain = createDomainWithContactsAndHosts();
|
||||
domainNameField.set(domain, "EXAMPLE.TLD");
|
||||
|
||||
DomainHistory historyWithoutResource =
|
||||
new DomainHistory.Builder()
|
||||
.setType(HistoryEntry.Type.DOMAIN_CREATE)
|
||||
.setXmlBytes("<xml></xml>".getBytes(UTF_8))
|
||||
.setModificationTime(fakeClock.nowUtc())
|
||||
.setRegistrarId("TheRegistrar")
|
||||
.setTrid(Trid.create("ABC-123", "server-trid"))
|
||||
.setBySuperuser(false)
|
||||
.setReason("reason")
|
||||
.setRequestedByRegistrar(true)
|
||||
.setDomainRepoId(domain.getRepoId())
|
||||
.setOtherRegistrarId("otherClient")
|
||||
.setPeriod(Period.create(1, Period.Unit.YEARS))
|
||||
.build();
|
||||
|
||||
DatabaseHelper.putInDb(domain, historyWithoutResource);
|
||||
jpaTm().transact(historyWithoutResource::beforeSqlSaveOnReplay);
|
||||
|
||||
assertThat(historyWithoutResource.getDomainContent().get().getDomainName())
|
||||
.isEqualTo("example.tld");
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testBeforeSqlSave_canonicalNameUtf8() throws Exception {
|
||||
Field domainNameField = DomainContent.class.getDeclaredField("fullyQualifiedDomainName");
|
||||
// reflection hacks to get around visibility issues
|
||||
domainNameField.setAccessible(true);
|
||||
DomainBase domain = createDomainWithContactsAndHosts();
|
||||
domainNameField.set(domain, "kittyçat.tld");
|
||||
|
||||
DomainHistory historyWithoutResource =
|
||||
new DomainHistory.Builder()
|
||||
.setType(HistoryEntry.Type.DOMAIN_CREATE)
|
||||
.setXmlBytes("<xml></xml>".getBytes(UTF_8))
|
||||
.setModificationTime(fakeClock.nowUtc())
|
||||
.setRegistrarId("TheRegistrar")
|
||||
.setTrid(Trid.create("ABC-123", "server-trid"))
|
||||
.setBySuperuser(false)
|
||||
.setReason("reason")
|
||||
.setRequestedByRegistrar(true)
|
||||
.setDomainRepoId(domain.getRepoId())
|
||||
.setOtherRegistrarId("otherClient")
|
||||
.setPeriod(Period.create(1, Period.Unit.YEARS))
|
||||
.build();
|
||||
|
||||
DatabaseHelper.putInDb(domain, historyWithoutResource);
|
||||
jpaTm().transact(historyWithoutResource::beforeSqlSaveOnReplay);
|
||||
|
||||
assertThat(historyWithoutResource.getDomainContent().get().getDomainName())
|
||||
.isEqualTo("xn--kittyat-yxa.tld");
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testFillingHistory_missingDigest() {
|
||||
createTld("tld");
|
||||
DomainBase baseDomain = createDomainWithContactsAndHosts();
|
||||
DomainBase domain =
|
||||
baseDomain
|
||||
.asBuilder()
|
||||
.setDsData(
|
||||
ImmutableSet.of(
|
||||
DelegationSignerData.create(0, 1, 2, new byte[] {}, baseDomain.getRepoId()),
|
||||
DelegationSignerData.create(3, 4, 5, null, baseDomain.getRepoId())))
|
||||
.build();
|
||||
DomainHistory domainHistory =
|
||||
new DomainHistory.Builder()
|
||||
.setDomainRepoId(domain.getRepoId())
|
||||
.setRegistrarId(domain.getCurrentSponsorRegistrarId())
|
||||
.setModificationTime(fakeClock.nowUtc())
|
||||
.setType(HistoryEntry.Type.DOMAIN_CREATE)
|
||||
.build();
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
domain.beforeSqlSaveOnReplay();
|
||||
jpaTm().put(domain);
|
||||
domainHistory.beforeSqlSaveOnReplay();
|
||||
jpaTm().put(domainHistory);
|
||||
});
|
||||
assertThat(DatabaseHelper.loadByEntity(domain).getDsData()).isEmpty();
|
||||
assertThat(DatabaseHelper.loadByEntity(domainHistory).getDsDataHistories()).isEmpty();
|
||||
}
|
||||
|
||||
static DomainBase createDomainWithContactsAndHosts() {
|
||||
createTld("tld");
|
||||
HostResource host = newHostResourceWithRoid("ns1.example.com", "host1");
|
||||
|
|
|
@ -20,7 +20,6 @@ import static google.registry.persistence.transaction.TransactionManagerFactory.
|
|||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.testing.DatabaseHelper.insertInDb;
|
||||
import static google.registry.testing.DatabaseHelper.loadByEntity;
|
||||
import static google.registry.testing.DatabaseHelper.newHostResource;
|
||||
import static google.registry.testing.DatabaseHelper.newHostResourceWithRoid;
|
||||
import static google.registry.testing.SqlHelper.saveRegistrar;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
|
@ -33,12 +32,10 @@ import google.registry.model.host.HostHistory;
|
|||
import google.registry.model.host.HostResource;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
import google.registry.testing.TestOfyOnly;
|
||||
import google.registry.testing.TestSqlOnly;
|
||||
import google.registry.util.SerializeUtils;
|
||||
import java.lang.reflect.Field;
|
||||
|
||||
/** Tests for {@link HostHistory}. */
|
||||
@DualDatabaseTest
|
||||
|
@ -118,88 +115,6 @@ public class HostHistoryTest extends EntityTestCase {
|
|||
assertThat(hostHistoryFromDb).isEqualTo(historyEntryFromDb);
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testBeforeSqlSave_afterHostPersisted() {
|
||||
HostResource hostResource = newHostResource("ns1.example.tld");
|
||||
HostHistory hostHistory =
|
||||
new HostHistory.Builder()
|
||||
.setType(HistoryEntry.Type.HOST_CREATE)
|
||||
.setXmlBytes("<xml></xml>".getBytes(UTF_8))
|
||||
.setModificationTime(fakeClock.nowUtc())
|
||||
.setRegistrarId("TheRegistrar")
|
||||
.setTrid(Trid.create("ABC-123", "server-trid"))
|
||||
.setBySuperuser(false)
|
||||
.setReason("reason")
|
||||
.setRequestedByRegistrar(true)
|
||||
.setHostRepoId(hostResource.getRepoId())
|
||||
.build();
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
jpaTm().put(hostResource);
|
||||
hostHistory.beforeSqlSaveOnReplay();
|
||||
jpaTm().put(hostHistory);
|
||||
});
|
||||
jpaTm()
|
||||
.transact(
|
||||
() ->
|
||||
assertAboutImmutableObjects()
|
||||
.that(jpaTm().loadByEntity(hostResource))
|
||||
.hasFieldsEqualTo(jpaTm().loadByEntity(hostHistory).getHostBase().get()));
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testBeforeSqlSave_canonicalNameUncapitalized() throws Exception {
|
||||
Field hostNameField = HostBase.class.getDeclaredField("fullyQualifiedHostName");
|
||||
// reflection hacks to get around visibility issues
|
||||
hostNameField.setAccessible(true);
|
||||
HostResource hostResource = newHostResource("ns1.example.tld");
|
||||
hostNameField.set(hostResource, "NS1.EXAMPLE.TLD");
|
||||
HostHistory hostHistory =
|
||||
new HostHistory.Builder()
|
||||
.setType(HistoryEntry.Type.HOST_CREATE)
|
||||
.setXmlBytes("<xml></xml>".getBytes(UTF_8))
|
||||
.setModificationTime(fakeClock.nowUtc())
|
||||
.setRegistrarId("TheRegistrar")
|
||||
.setTrid(Trid.create("ABC-123", "server-trid"))
|
||||
.setBySuperuser(false)
|
||||
.setReason("reason")
|
||||
.setRequestedByRegistrar(true)
|
||||
.setHostRepoId(hostResource.getRepoId())
|
||||
.build();
|
||||
|
||||
DatabaseHelper.putInDb(hostResource, hostHistory);
|
||||
jpaTm().transact(hostHistory::beforeSqlSaveOnReplay);
|
||||
|
||||
assertThat(hostHistory.getHostBase().get().getHostName()).isEqualTo("ns1.example.tld");
|
||||
}
|
||||
|
||||
@TestSqlOnly
|
||||
void testBeforeSqlSave_canonicalNameUtf8() throws Exception {
|
||||
Field hostNameField = HostBase.class.getDeclaredField("fullyQualifiedHostName");
|
||||
// reflection hacks to get around visibility issues
|
||||
hostNameField.setAccessible(true);
|
||||
HostResource hostResource = newHostResource("ns1.example.tld");
|
||||
hostNameField.set(hostResource, "ns1.kittyçat.tld");
|
||||
HostHistory hostHistory =
|
||||
new HostHistory.Builder()
|
||||
.setType(HistoryEntry.Type.HOST_CREATE)
|
||||
.setXmlBytes("<xml></xml>".getBytes(UTF_8))
|
||||
.setModificationTime(fakeClock.nowUtc())
|
||||
.setRegistrarId("TheRegistrar")
|
||||
.setTrid(Trid.create("ABC-123", "server-trid"))
|
||||
.setBySuperuser(false)
|
||||
.setReason("reason")
|
||||
.setRequestedByRegistrar(true)
|
||||
.setHostRepoId(hostResource.getRepoId())
|
||||
.build();
|
||||
|
||||
DatabaseHelper.putInDb(hostResource, hostHistory);
|
||||
jpaTm().transact(hostHistory::beforeSqlSaveOnReplay);
|
||||
|
||||
assertThat(hostHistory.getHostBase().get().getHostName()).isEqualTo("ns1.xn--kittyat-yxa.tld");
|
||||
}
|
||||
|
||||
private void assertHostHistoriesEqual(HostHistory one, HostHistory two) {
|
||||
assertAboutImmutableObjects().that(one).isEqualExceptFields(two, "hostBase");
|
||||
assertAboutImmutableObjects()
|
||||
|
|
|
@ -18,7 +18,6 @@ import static com.google.common.truth.Truth.assertThat;
|
|||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||
import static google.registry.model.ImmutableObjectSubject.immutableObjectCorrespondence;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.testing.DatabaseHelper.cloneAndSetAutoTimestamps;
|
||||
import static google.registry.testing.DatabaseHelper.createTld;
|
||||
|
@ -31,15 +30,11 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.net.InetAddresses;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.model.EntityTestCase;
|
||||
import google.registry.model.ImmutableObjectSubject;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.model.eppcommon.Trid;
|
||||
import google.registry.model.index.EppResourceIndex;
|
||||
import google.registry.model.index.ForeignKeyIndex;
|
||||
import google.registry.model.index.ForeignKeyIndex.ForeignKeyHostIndex;
|
||||
import google.registry.model.transfer.DomainTransferData;
|
||||
import google.registry.model.transfer.TransferStatus;
|
||||
import google.registry.testing.DualDatabaseTest;
|
||||
|
@ -296,40 +291,4 @@ class HostResourceTest extends EntityTestCase {
|
|||
host = host.asBuilder().setLastTransferTime(day1).setLastSuperordinateChange(day2).build();
|
||||
assertThat(host.computeLastTransferTime(domain)).isEqualTo(day3);
|
||||
}
|
||||
|
||||
@TestOfyOnly
|
||||
void testBeforeDatastoreSaveOnReplay_indexes() {
|
||||
ImmutableList<ForeignKeyHostIndex> foreignKeyIndexes =
|
||||
ofyTm().loadAllOf(ForeignKeyHostIndex.class);
|
||||
ImmutableList<EppResourceIndex> eppResourceIndexes = ofyTm().loadAllOf(EppResourceIndex.class);
|
||||
fakeClock.advanceOneMilli();
|
||||
ofyTm()
|
||||
.transact(
|
||||
() -> {
|
||||
foreignKeyIndexes.forEach(ofyTm()::delete);
|
||||
eppResourceIndexes.forEach(ofyTm()::delete);
|
||||
});
|
||||
assertThat(ofyTm().loadAllOf(ForeignKeyHostIndex.class)).isEmpty();
|
||||
assertThat(ofyTm().loadAllOf(EppResourceIndex.class)).isEmpty();
|
||||
|
||||
ofyTm().transact(() -> host.beforeDatastoreSaveOnReplay());
|
||||
|
||||
assertThat(ofyTm().loadAllOf(ForeignKeyHostIndex.class))
|
||||
.containsExactly(ForeignKeyIndex.create(host, host.getDeletionTime()));
|
||||
assertThat(ofyTm().loadAllOf(EppResourceIndex.class))
|
||||
.containsExactly(EppResourceIndex.create(Key.create(host)));
|
||||
}
|
||||
|
||||
@TestOfyOnly
|
||||
void testBeforeSqlSaveOnReplay_canonicalName() {
|
||||
host.fullyQualifiedHostName = "NS1.EXAMPLE.COM";
|
||||
assertThat(host.getHostName()).isEqualTo("NS1.EXAMPLE.COM");
|
||||
host.beforeSqlSaveOnReplay();
|
||||
assertThat(host.getHostName()).isEqualTo("ns1.example.com");
|
||||
|
||||
host.fullyQualifiedHostName = "ns1.kittyçat.com";
|
||||
assertThat(host.getHostName()).isEqualTo("ns1.kittyçat.com");
|
||||
host.beforeSqlSaveOnReplay();
|
||||
assertThat(host.getHostName()).isEqualTo("ns1.xn--kittyat-yxa.com");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,538 +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.model.replay;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.model.replay.ReplicateToDatastoreAction.applyTransaction;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||
import static google.registry.testing.DatabaseHelper.insertInDb;
|
||||
import static google.registry.testing.LogsSubject.assertAboutLogs;
|
||||
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
import static org.junit.jupiter.api.Assumptions.assumeTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableSortedMap;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.google.common.testing.TestLogHandler;
|
||||
import com.google.common.truth.Truth8;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
||||
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.model.ofy.CommitLogBucket;
|
||||
import google.registry.model.ofy.Ofy;
|
||||
import google.registry.model.server.Lock;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.persistence.transaction.JpaTransactionManagerImpl;
|
||||
import google.registry.persistence.transaction.TransactionEntity;
|
||||
import google.registry.persistence.transaction.TransactionManagerFactory;
|
||||
import google.registry.testing.AppEngineExtension;
|
||||
import google.registry.testing.DatabaseHelper;
|
||||
import google.registry.testing.FakeClock;
|
||||
import google.registry.testing.FakeResponse;
|
||||
import google.registry.testing.InjectExtension;
|
||||
import google.registry.testing.ReplayExtension;
|
||||
import google.registry.testing.TestObject;
|
||||
import google.registry.util.RequestStatusChecker;
|
||||
import google.registry.util.SystemClock;
|
||||
import java.lang.reflect.Proxy;
|
||||
import java.util.List;
|
||||
import java.util.logging.Level;
|
||||
import java.util.logging.Logger;
|
||||
import javax.persistence.EntityManager;
|
||||
import javax.persistence.EntityManagerFactory;
|
||||
import javax.persistence.EntityTransaction;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
import org.junitpioneer.jupiter.RetryingTest;
|
||||
|
||||
public class ReplicateToDatastoreActionTest {
|
||||
|
||||
private final FakeClock fakeClock = new FakeClock(DateTime.parse("2000-01-01TZ"));
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
@RegisterExtension
|
||||
public final AppEngineExtension appEngine =
|
||||
AppEngineExtension.builder()
|
||||
.withDatastoreAndCloudSql()
|
||||
.withOfyTestEntities(AllocationToken.class, Lock.class, TestObject.class)
|
||||
.withJpaUnitTestEntities(AllocationToken.class, Lock.class, TestObject.class)
|
||||
.withClock(fakeClock)
|
||||
.build();
|
||||
|
||||
@RegisterExtension final InjectExtension injectExtension = new InjectExtension();
|
||||
|
||||
private final TestLogHandler logHandler = new TestLogHandler();
|
||||
private ReplicateToDatastoreAction action;
|
||||
private FakeResponse response;
|
||||
|
||||
@BeforeEach
|
||||
void setUp() {
|
||||
resetAction();
|
||||
injectExtension.setStaticField(Ofy.class, "clock", fakeClock);
|
||||
// Use a single bucket to expose timestamp inversion problems.
|
||||
injectExtension.setStaticField(
|
||||
CommitLogBucket.class, "bucketIdSupplier", Suppliers.ofInstance(1));
|
||||
fakeClock.setAutoIncrementByOneMilli();
|
||||
DatabaseHelper.setMigrationScheduleToSqlPrimary(fakeClock);
|
||||
Logger.getLogger(ReplicateToDatastoreAction.class.getCanonicalName()).addHandler(logHandler);
|
||||
fakeClock.advanceBy(Duration.standardDays(1));
|
||||
TestObject.beforeDatastoreSaveCallCount = 0;
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
void tearDown() {
|
||||
DatabaseHelper.removeDatabaseMigrationSchedule();
|
||||
fakeClock.disableAutoIncrement();
|
||||
TransactionManagerFactory.setClockForTesting(new SystemClock());
|
||||
}
|
||||
|
||||
@RetryingTest(4)
|
||||
void testReplication() {
|
||||
assumeTrue(ReplayExtension.replayTestsEnabled());
|
||||
|
||||
TestObject foo = TestObject.create("foo");
|
||||
TestObject bar = TestObject.create("bar");
|
||||
TestObject baz = TestObject.create("baz");
|
||||
|
||||
insertInDb(foo, bar);
|
||||
runAndVerifySuccess();
|
||||
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(foo.key()))).isEqualTo(foo);
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(bar.key()))).isEqualTo(bar);
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(baz.key())).isPresent()).isFalse();
|
||||
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
jpaTm().delete(bar.key());
|
||||
jpaTm().insert(baz);
|
||||
});
|
||||
runAndVerifySuccess();
|
||||
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(bar.key()).isPresent())).isFalse();
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(baz.key()))).isEqualTo(baz);
|
||||
}
|
||||
|
||||
@RetryingTest(4)
|
||||
void testReplayFromLastTxn() {
|
||||
assumeTrue(ReplayExtension.replayTestsEnabled());
|
||||
|
||||
TestObject foo = TestObject.create("foo");
|
||||
TestObject bar = TestObject.create("bar");
|
||||
|
||||
// Write a transaction containing "foo".
|
||||
insertInDb(foo);
|
||||
runAndVerifySuccess();
|
||||
|
||||
// Verify that it propagated to datastore, then remove "foo" directly from datastore.
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(foo.key()))).isEqualTo(foo);
|
||||
ofyTm().transact(() -> ofyTm().delete(foo.key()));
|
||||
|
||||
// Write "bar"
|
||||
insertInDb(bar);
|
||||
runAndVerifySuccess();
|
||||
|
||||
// If we replayed only the most recent transaction, we should have "bar" but not "foo".
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(bar.key()))).isEqualTo(bar);
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(foo.key()).isPresent())).isFalse();
|
||||
}
|
||||
|
||||
@RetryingTest(4)
|
||||
void testUnintentionalConcurrency() {
|
||||
assumeTrue(ReplayExtension.replayTestsEnabled());
|
||||
|
||||
TestObject foo = TestObject.create("foo");
|
||||
TestObject bar = TestObject.create("bar");
|
||||
|
||||
// Write a transaction and run just the batch fetch.
|
||||
insertInDb(foo);
|
||||
List<TransactionEntity> txns1 = action.getTransactionBatchAtSnapshot();
|
||||
assertThat(txns1).hasSize(1);
|
||||
|
||||
// Write a second transaction and do another batch fetch.
|
||||
insertInDb(bar);
|
||||
List<TransactionEntity> txns2 = action.getTransactionBatchAtSnapshot();
|
||||
assertThat(txns2).hasSize(2);
|
||||
|
||||
// Apply the first batch.
|
||||
applyTransaction(txns1.get(0));
|
||||
|
||||
// Remove the foo record so we can ensure that this transaction doesn't get doublle-played.
|
||||
ofyTm().transact(() -> ofyTm().delete(foo.key()));
|
||||
|
||||
// Apply the second batch.
|
||||
for (TransactionEntity txn : txns2) {
|
||||
applyTransaction(txn);
|
||||
}
|
||||
|
||||
// Verify that the first transaction didn't get replayed but the second one did.
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(foo.key()).isPresent())).isFalse();
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(bar.key()))).isEqualTo(bar);
|
||||
assertAboutLogs()
|
||||
.that(logHandler)
|
||||
.hasLogAtLevelWithMessage(
|
||||
Level.WARNING, "Ignoring transaction 1, which appears to have already been applied.");
|
||||
}
|
||||
|
||||
@RetryingTest(4)
|
||||
void testCreateAutoTimestamp() {
|
||||
// Verify that fields populated by the DB (e.g. CreateAutoTimestamp) correctly get populated in
|
||||
// both databases.
|
||||
assumeTrue(ReplayExtension.replayTestsEnabled());
|
||||
|
||||
AllocationToken allocationToken =
|
||||
new AllocationToken.Builder().setToken("abc123").setTokenType(TokenType.SINGLE_USE).build();
|
||||
insertInDb(allocationToken);
|
||||
runAndVerifySuccess();
|
||||
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByEntity(allocationToken)))
|
||||
.isEqualTo(jpaTm().transact(() -> jpaTm().loadByEntity(allocationToken)));
|
||||
}
|
||||
|
||||
@RetryingTest(4)
|
||||
void testNoTransactionIdUpdate() {
|
||||
// Create an object.
|
||||
TestObject foo = TestObject.create("foo");
|
||||
insertInDb(foo);
|
||||
|
||||
// Fail two transactions.
|
||||
for (int i = 0; i < 2; ++i) {
|
||||
try {
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
jpaTm().delete(foo.key());
|
||||
// Explicitly save the transaction entity to force the id update.
|
||||
jpaTm().insert(new TransactionEntity(new byte[] {1, 2, 3}));
|
||||
throw new RuntimeException("fail!!!");
|
||||
});
|
||||
} catch (Exception e) {
|
||||
logger.atInfo().log("Got expected exception.");
|
||||
}
|
||||
}
|
||||
|
||||
TestObject bar = TestObject.create("bar");
|
||||
insertInDb(bar);
|
||||
|
||||
// Make sure we have only the expected transaction ids.
|
||||
List<TransactionEntity> txns = action.getTransactionBatchAtSnapshot();
|
||||
assertThat(txns).hasSize(2);
|
||||
for (TransactionEntity txn : txns) {
|
||||
assertThat(txn.getId()).isNotEqualTo(2);
|
||||
assertThat(txn.getId()).isNotEqualTo(3);
|
||||
applyTransaction(txn);
|
||||
}
|
||||
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(foo.key()))).isEqualTo(foo);
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(bar.key()))).isEqualTo(bar);
|
||||
assertThat(ofyTm().transact(() -> LastSqlTransaction.load()).getTransactionId()).isEqualTo(4);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testTransactionGapReplay() {
|
||||
insertInDb(TestObject.create("foo"));
|
||||
DeferredCommit deferred = new DeferredCommit(fakeClock);
|
||||
TestObject bar = TestObject.create("bar");
|
||||
deferred.transact(() -> jpaTm().insert(bar));
|
||||
TestObject baz = TestObject.create("baz");
|
||||
insertInDb(baz);
|
||||
|
||||
action.run();
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(bar.key())).isPresent()).isFalse();
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(baz.key()))).isEqualTo(baz);
|
||||
VKey<ReplayGap> gapKey = VKey.createOfy(ReplayGap.class, Key.create(ReplayGap.class, 2));
|
||||
Truth8.assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(gapKey))).isPresent();
|
||||
|
||||
deferred.commit();
|
||||
resetAction();
|
||||
action.run();
|
||||
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(bar.key()))).isEqualTo(bar);
|
||||
// Verify that the gap record has been cleaned up.
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(gapKey).isPresent())).isFalse();
|
||||
}
|
||||
|
||||
/** Verify that we can handle creation and deletion of > 25 gap records. */
|
||||
@Test
|
||||
void testLargeNumberOfGaps() {
|
||||
// Fail thirty transactions.
|
||||
for (int i = 0; i < 30; ++i) {
|
||||
try {
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
insertInDb(TestObject.create("foo"));
|
||||
// Explicitly save the transaction entity to force the id update.
|
||||
jpaTm().insert(new TransactionEntity(new byte[] {1, 2, 3}));
|
||||
throw new RuntimeException("fail!!!");
|
||||
});
|
||||
} catch (Exception e) {
|
||||
;
|
||||
}
|
||||
}
|
||||
|
||||
TestObject bar = TestObject.create("bar");
|
||||
insertInDb(bar);
|
||||
|
||||
// Verify that the transaction was successfully applied and that we have generated 30 gap
|
||||
// records.
|
||||
action.run();
|
||||
Truth8.assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(bar.key()))).isPresent();
|
||||
assertThat(ofyTm().loadAllOf(ReplayGap.class)).hasSize(30);
|
||||
|
||||
// Verify that we can clean up this many gap records after expiration.
|
||||
fakeClock.advanceBy(Duration.millis(ReplicateToDatastoreAction.MAX_GAP_RETENTION_MILLIS + 1));
|
||||
resetAction();
|
||||
action.run();
|
||||
assertThat(ofyTm().loadAllOf(ReplayGap.class)).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGapRecordExpiration() {
|
||||
insertInDb(TestObject.create("foo"));
|
||||
|
||||
// Fail a transaction, create a gap.
|
||||
try {
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
jpaTm().insert(TestObject.create("other"));
|
||||
// Explicitly save the transaction entity to force the id update.
|
||||
jpaTm().insert(new TransactionEntity(new byte[] {1, 2, 3}));
|
||||
throw new RuntimeException("fail!!!");
|
||||
});
|
||||
} catch (Exception e) {
|
||||
logger.atInfo().log("Got expected exception.");
|
||||
}
|
||||
|
||||
insertInDb(TestObject.create("bar"));
|
||||
|
||||
action.run();
|
||||
|
||||
// Verify that the gap record has been created
|
||||
VKey<ReplayGap> gapKey = VKey.createOfy(ReplayGap.class, Key.create(ReplayGap.class, 2));
|
||||
Truth8.assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(gapKey))).isPresent();
|
||||
|
||||
fakeClock.advanceBy(Duration.millis(ReplicateToDatastoreAction.MAX_GAP_RETENTION_MILLIS + 1));
|
||||
resetAction();
|
||||
action.run();
|
||||
|
||||
// Verify that the gap record has been destroyed.
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(gapKey)).isPresent()).isFalse();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testBeforeDatastoreSaveCallback() {
|
||||
assumeTrue(ReplayExtension.replayTestsEnabled());
|
||||
|
||||
TestObject testObject = TestObject.create("foo");
|
||||
insertInDb(testObject);
|
||||
action.run();
|
||||
assertThat(ofyTm().loadAllOf(TestObject.class)).containsExactly(testObject);
|
||||
assertThat(TestObject.beforeDatastoreSaveCallCount).isEqualTo(1);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNotInMigrationState_doesNothing() {
|
||||
assumeTrue(ReplayExtension.replayTestsEnabled());
|
||||
|
||||
// set a schedule that backtracks the current status to DATASTORE_PRIMARY
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
jpaTm()
|
||||
.transact(
|
||||
() ->
|
||||
DatabaseMigrationStateSchedule.set(
|
||||
ImmutableSortedMap.<DateTime, MigrationState>naturalOrder()
|
||||
.put(START_OF_TIME, MigrationState.DATASTORE_ONLY)
|
||||
.put(START_OF_TIME.plusHours(1), MigrationState.DATASTORE_PRIMARY)
|
||||
.put(START_OF_TIME.plusHours(2), MigrationState.DATASTORE_PRIMARY_NO_ASYNC)
|
||||
.put(START_OF_TIME.plusHours(3), MigrationState.DATASTORE_PRIMARY_READ_ONLY)
|
||||
.put(START_OF_TIME.plusHours(4), MigrationState.SQL_PRIMARY)
|
||||
.put(now.plusHours(1), MigrationState.SQL_PRIMARY_READ_ONLY)
|
||||
.put(now.plusHours(2), MigrationState.DATASTORE_PRIMARY_READ_ONLY)
|
||||
.put(now.plusHours(3), MigrationState.DATASTORE_PRIMARY)
|
||||
.build()));
|
||||
fakeClock.advanceBy(Duration.standardDays(1));
|
||||
|
||||
insertInDb(TestObject.create("foo"));
|
||||
action.run();
|
||||
// Replication shouldn't have happened
|
||||
assertThat(ofyTm().loadAllOf(TestObject.class)).isEmpty();
|
||||
assertAboutLogs()
|
||||
.that(logHandler)
|
||||
.hasLogAtLevelWithMessage(
|
||||
Level.INFO,
|
||||
"Skipping ReplicateToDatastoreAction because we are in migration phase "
|
||||
+ "DATASTORE_PRIMARY.");
|
||||
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
||||
assertThat(response.getPayload())
|
||||
.isEqualTo(
|
||||
"Skipping ReplicateToDatastoreAction because we are in migration phase"
|
||||
+ " DATASTORE_PRIMARY.");
|
||||
}
|
||||
|
||||
@Test
|
||||
void replicationWorksInReadOnly() {
|
||||
|
||||
// Put us in SQL primary now, readonly in an hour, then in datastore primary after 25 hours.
|
||||
// And we'll need the TransactionManagerFactory to use the fake clock.
|
||||
DateTime now = fakeClock.nowUtc();
|
||||
TransactionManagerFactory.setClockForTesting(fakeClock);
|
||||
jpaTm()
|
||||
.transact(
|
||||
() ->
|
||||
DatabaseMigrationStateSchedule.set(
|
||||
ImmutableSortedMap.<DateTime, MigrationState>naturalOrder()
|
||||
.put(START_OF_TIME, MigrationState.DATASTORE_ONLY)
|
||||
.put(START_OF_TIME.plusHours(1), MigrationState.DATASTORE_PRIMARY)
|
||||
.put(START_OF_TIME.plusHours(2), MigrationState.DATASTORE_PRIMARY_NO_ASYNC)
|
||||
.put(START_OF_TIME.plusHours(3), MigrationState.DATASTORE_PRIMARY_READ_ONLY)
|
||||
.put(START_OF_TIME.plusHours(4), MigrationState.SQL_PRIMARY_READ_ONLY)
|
||||
.put(START_OF_TIME.plusHours(5), MigrationState.SQL_PRIMARY)
|
||||
.put(now.plusHours(1), MigrationState.SQL_PRIMARY_READ_ONLY)
|
||||
.put(now.plusHours(25), MigrationState.DATASTORE_PRIMARY_READ_ONLY)
|
||||
.put(now.plusHours(26), MigrationState.DATASTORE_PRIMARY_READ_ONLY)
|
||||
.build()));
|
||||
|
||||
TestObject foo = TestObject.create("foo");
|
||||
insertInDb(foo);
|
||||
TestObject bar = TestObject.create("bar");
|
||||
insertInDb(bar);
|
||||
TestObject baz = TestObject.create("baz");
|
||||
insertInDb(baz);
|
||||
jpaTm().transact(() -> jpaTm().delete(baz.key()));
|
||||
|
||||
// get to read-only
|
||||
fakeClock.advanceBy(Duration.standardDays(1));
|
||||
|
||||
// process the transaction in readonly.
|
||||
action.run();
|
||||
|
||||
// Forward the next day (datastore primary). Verify that datastore has all of the changes.
|
||||
fakeClock.advanceBy(Duration.standardDays(1));
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(foo.key()))).isEqualTo(foo);
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKey(bar.key()))).isEqualTo(bar);
|
||||
assertThat(ofyTm().transact(() -> ofyTm().loadByKeyIfPresent(baz.key()).isPresent())).isFalse();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFailure_cannotAcquireLock() {
|
||||
assumeTrue(ReplayExtension.replayTestsEnabled());
|
||||
|
||||
RequestStatusChecker requestStatusChecker = mock(RequestStatusChecker.class);
|
||||
when(requestStatusChecker.getLogId()).thenReturn("logId");
|
||||
Truth8.assertThat(
|
||||
Lock.acquireSql(
|
||||
ReplicateToDatastoreAction.class.getSimpleName(),
|
||||
null,
|
||||
Duration.standardHours(1),
|
||||
requestStatusChecker,
|
||||
false))
|
||||
.isPresent();
|
||||
fakeClock.advanceOneMilli();
|
||||
|
||||
resetAction();
|
||||
action.run();
|
||||
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
||||
assertThat(response.getPayload())
|
||||
.isEqualTo("Can't acquire ReplicateToDatastoreAction lock, aborting.");
|
||||
}
|
||||
|
||||
private void runAndVerifySuccess() {
|
||||
resetAction();
|
||||
action.run();
|
||||
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
||||
assertThat(response.getPayload())
|
||||
.isEqualTo("Replayed 1 transaction(s) from Cloud SQL -> Datastore.");
|
||||
}
|
||||
|
||||
private void resetAction() {
|
||||
response = new FakeResponse();
|
||||
RequestStatusChecker requestStatusChecker = mock(RequestStatusChecker.class);
|
||||
when(requestStatusChecker.getLogId()).thenReturn("logId");
|
||||
action = new ReplicateToDatastoreAction(fakeClock, requestStatusChecker, response);
|
||||
}
|
||||
|
||||
/**
|
||||
* Deep fake of EntityManagerFactory -> EntityManager -> EntityTransaction that allows us to defer
|
||||
* the actual commit until after the other transactions are replayed.
|
||||
*/
|
||||
static class DeferredCommit {
|
||||
|
||||
FakeClock clock;
|
||||
EntityTransaction realTransaction;
|
||||
|
||||
DeferredCommit(FakeClock clock) {
|
||||
this.clock = clock;
|
||||
}
|
||||
|
||||
private static <T> T makeProxy(
|
||||
Class<T> iface, T delegate, String method, Supplier<?> supplier) {
|
||||
return (T)
|
||||
Proxy.newProxyInstance(
|
||||
delegate.getClass().getClassLoader(),
|
||||
new Class[] {iface},
|
||||
(proxy, meth, args) -> {
|
||||
if (meth.getName().equals(method)) {
|
||||
return supplier.get();
|
||||
} else {
|
||||
return meth.invoke(delegate, args);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
EntityManager createEntityManagerProxy(EntityManager orgEntityManager) {
|
||||
return makeProxy(
|
||||
EntityManager.class,
|
||||
orgEntityManager,
|
||||
"getTransaction",
|
||||
() ->
|
||||
makeProxy(
|
||||
EntityTransaction.class,
|
||||
realTransaction = orgEntityManager.getTransaction(),
|
||||
"commit",
|
||||
() -> null));
|
||||
}
|
||||
|
||||
void commit() {
|
||||
realTransaction.commit();
|
||||
}
|
||||
|
||||
void transact(Runnable runnable) {
|
||||
EntityManagerFactory orgEmf =
|
||||
jpaTm().transact(() -> jpaTm().getEntityManager().getEntityManagerFactory());
|
||||
EntityManagerFactory emfProxy =
|
||||
makeProxy(
|
||||
EntityManagerFactory.class,
|
||||
orgEmf,
|
||||
"createEntityManager",
|
||||
() -> createEntityManagerProxy(orgEmf.createEntityManager()));
|
||||
|
||||
new JpaTransactionManagerImpl(emfProxy, clock).transact(runnable);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -225,8 +225,6 @@ abstract class JpaTransactionManagerExtension implements BeforeEachCallback, Aft
|
|||
public void afterEach(ExtensionContext context) {
|
||||
TransactionManagerFactory.setJpaTm(Suppliers.ofInstance(cachedTm));
|
||||
TransactionManagerFactory.setReplicaJpaTm(Suppliers.ofInstance(cachedTm));
|
||||
// Even though we didn't set this, reset it to make sure no other tests are affected
|
||||
JpaTransactionManagerImpl.removeReplaySqlToDsOverrideForTest();
|
||||
cachedTm = null;
|
||||
}
|
||||
|
||||
|
|
|
@ -476,39 +476,19 @@ public final class AppEngineExtension implements BeforeEachCallback, AfterEachCa
|
|||
public void afterEach(ExtensionContext context) throws Exception {
|
||||
checkArgumentNotNull(context, "The ExtensionContext must not be null");
|
||||
try {
|
||||
// If there is a replay extension, we'll want to call its replay() method.
|
||||
//
|
||||
// We have to provide this hook here for ReplayExtension instead of relying on
|
||||
// ReplayExtension's afterEach() method because of ordering and the conflation of environment
|
||||
// initialization and basic entity initialization.
|
||||
//
|
||||
// ReplayExtension's beforeEach() has to be called before this so that the entities that we
|
||||
// initialize (e.g. "TheRegistrar") also get replayed. But that means that ReplayExtension's
|
||||
// afterEach() won't be called until after ours. Since we tear down the datastore and SQL
|
||||
// database in our own afterEach(), ReplayExtension's afterEach() would fail if we let the
|
||||
// replay happen there.
|
||||
ReplayExtension replayer =
|
||||
(ReplayExtension)
|
||||
context.getStore(ExtensionContext.Namespace.GLOBAL).get(ReplayExtension.class);
|
||||
if (replayer != null) {
|
||||
replayer.replay();
|
||||
if (withCloudSql) {
|
||||
if (enableJpaEntityCoverageCheck) {
|
||||
jpaIntegrationWithCoverageExtension.afterEach(context);
|
||||
} else if (withJpaUnitTest) {
|
||||
jpaUnitTestExtension.afterEach(context);
|
||||
} else {
|
||||
jpaIntegrationTestExtension.afterEach(context);
|
||||
}
|
||||
}
|
||||
tearDown();
|
||||
} finally {
|
||||
try {
|
||||
if (withCloudSql) {
|
||||
if (enableJpaEntityCoverageCheck) {
|
||||
jpaIntegrationWithCoverageExtension.afterEach(context);
|
||||
} else if (withJpaUnitTest) {
|
||||
jpaUnitTestExtension.afterEach(context);
|
||||
} else {
|
||||
jpaIntegrationTestExtension.afterEach(context);
|
||||
}
|
||||
}
|
||||
tearDown();
|
||||
} finally {
|
||||
if (isWithDatastoreAndCloudSql()) {
|
||||
restoreTmAfterDualDatabaseTest(context);
|
||||
}
|
||||
if (isWithDatastoreAndCloudSql()) {
|
||||
restoreTmAfterDualDatabaseTest(context);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -131,11 +131,6 @@ import org.joda.time.Duration;
|
|||
/** Static utils for setting up test resources. */
|
||||
public class DatabaseHelper {
|
||||
|
||||
// The following two fields are injected by ReplayExtension.
|
||||
|
||||
// If this is true, all of the methods that save to the datastore do so with backup.
|
||||
private static boolean alwaysSaveWithBackup;
|
||||
|
||||
// If the clock is defined, it will always be advanced by one millsecond after a transaction.
|
||||
private static FakeClock clock;
|
||||
|
||||
|
@ -149,10 +144,6 @@ public class DatabaseHelper {
|
|||
DatabaseHelper.class, "default_premium_list_testdata.csv")),
|
||||
String.class));
|
||||
|
||||
public static void setAlwaysSaveWithBackup(boolean enable) {
|
||||
alwaysSaveWithBackup = enable;
|
||||
}
|
||||
|
||||
public static void setClock(FakeClock fakeClock) {
|
||||
clock = fakeClock;
|
||||
}
|
||||
|
@ -1001,8 +992,7 @@ public class DatabaseHelper {
|
|||
|
||||
private static <R extends ImmutableObject> void saveResource(R resource, boolean wantBackup) {
|
||||
if (tm().isOfy()) {
|
||||
Consumer<ImmutableObject> saver =
|
||||
wantBackup || alwaysSaveWithBackup ? tm()::put : tm()::putWithoutBackup;
|
||||
Consumer<ImmutableObject> saver = wantBackup ? tm()::put : tm()::putWithoutBackup;
|
||||
saver.accept(resource);
|
||||
if (resource instanceof EppResource) {
|
||||
EppResource eppResource = (EppResource) resource;
|
||||
|
@ -1220,14 +1210,7 @@ public class DatabaseHelper {
|
|||
* entities.
|
||||
*/
|
||||
public static <R> void insertSimpleResources(final Iterable<R> resources) {
|
||||
tm().transact(
|
||||
() -> {
|
||||
if (alwaysSaveWithBackup) {
|
||||
tm().insertAll(ImmutableList.copyOf(resources));
|
||||
} else {
|
||||
tm().insertAllWithoutBackup(ImmutableList.copyOf(resources));
|
||||
}
|
||||
});
|
||||
tm().transact(() -> tm().insertAllWithoutBackup(ImmutableList.copyOf(resources)));
|
||||
maybeAdvanceClock();
|
||||
// Force the session to be cleared so that when we read it back, we read from Datastore
|
||||
// and not from the transaction's session cache.
|
||||
|
@ -1235,12 +1218,7 @@ public class DatabaseHelper {
|
|||
}
|
||||
|
||||
public static void deleteResource(final Object resource) {
|
||||
if (alwaysSaveWithBackup) {
|
||||
tm().transact(() -> tm().delete(resource));
|
||||
maybeAdvanceClock();
|
||||
} else {
|
||||
transactIfJpaTm(() -> tm().deleteWithoutBackup(resource));
|
||||
}
|
||||
transactIfJpaTm(() -> tm().deleteWithoutBackup(resource));
|
||||
// Force the session to be cleared so that when we read it back, we read from Datastore and
|
||||
// not from the transaction's session cache.
|
||||
tm().clearSessionCache();
|
||||
|
|
|
@ -52,16 +52,6 @@ class DualDatabaseTestInvocationContextProvider implements TestTemplateInvocatio
|
|||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if "context" is an objectify unit test.
|
||||
*
|
||||
* <p>Provided to allow ReplayExtension to make this determination.
|
||||
*/
|
||||
static boolean inOfyContext(ExtensionContext context) {
|
||||
return (DatabaseType) context.getStore(NAMESPACE).get(INJECTED_TM_SUPPLIER_KEY)
|
||||
== DatabaseType.OFY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<TestTemplateInvocationContext> provideTestTemplateInvocationContexts(
|
||||
ExtensionContext context) {
|
||||
|
|
|
@ -1,429 +0,0 @@
|
|||
// 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.testing;
|
||||
|
||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth.assertWithMessage;
|
||||
import static google.registry.model.ImmutableObjectSubject.assertAboutImmutableObjects;
|
||||
import static google.registry.model.ofy.ObjectifyService.auditedOfy;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||
import static java.lang.annotation.ElementType.METHOD;
|
||||
import static java.lang.annotation.RetentionPolicy.RUNTIME;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import com.google.common.base.Ascii;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Streams;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.model.EntityClasses;
|
||||
import google.registry.model.ImmutableObject;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.ofy.CommitLogBucket;
|
||||
import google.registry.model.ofy.ReplayQueue;
|
||||
import google.registry.model.ofy.TransactionInfo;
|
||||
import google.registry.model.replay.DatastoreEntity;
|
||||
import google.registry.model.replay.ReplicateToDatastoreAction;
|
||||
import google.registry.model.replay.SqlEntity;
|
||||
import google.registry.persistence.VKey;
|
||||
import google.registry.persistence.transaction.JpaEntityCoverageExtension;
|
||||
import google.registry.persistence.transaction.JpaTransactionManagerImpl;
|
||||
import google.registry.persistence.transaction.Transaction;
|
||||
import google.registry.persistence.transaction.Transaction.Delete;
|
||||
import google.registry.persistence.transaction.Transaction.Mutation;
|
||||
import google.registry.persistence.transaction.Transaction.Update;
|
||||
import google.registry.persistence.transaction.TransactionEntity;
|
||||
import google.registry.util.RequestStatusChecker;
|
||||
import java.io.IOException;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.Target;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import javax.annotation.Nullable;
|
||||
import org.junit.jupiter.api.TestTemplate;
|
||||
import org.junit.jupiter.api.extension.AfterEachCallback;
|
||||
import org.junit.jupiter.api.extension.BeforeEachCallback;
|
||||
import org.junit.jupiter.api.extension.ExtensionContext;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
/**
|
||||
* A JUnit extension that replays datastore transactions against postgresql.
|
||||
*
|
||||
* <p>This extension must be ordered before AppEngineExtension so that the test entities saved in
|
||||
* that extension are also replayed. If AppEngineExtension is not used,
|
||||
* JpaTransactionManagerExtension must be, and this extension should be ordered _after_
|
||||
* JpaTransactionManagerExtension so that writes to SQL work.
|
||||
*/
|
||||
public class ReplayExtension implements BeforeEachCallback, AfterEachCallback {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
private static ImmutableSet<String> NON_REPLICATED_TYPES =
|
||||
ImmutableSet.of(
|
||||
"PremiumList",
|
||||
"PremiumListRevision",
|
||||
"PremiumListEntry",
|
||||
"ReservedList",
|
||||
"RdeRevision",
|
||||
"ServerSecret",
|
||||
"SignedMarkRevocationList",
|
||||
"ClaimsListShard",
|
||||
"TmchCrl",
|
||||
"EppResourceIndex",
|
||||
"ForeignKeyIndex",
|
||||
"ForeignKeyHostIndex",
|
||||
"ForeignKeyContactIndex",
|
||||
"ForeignKeyDomainIndex");
|
||||
|
||||
// Entity classes to be ignored during the final database comparison. Note that this is just a
|
||||
// mash-up of Datastore and SQL classes, and used for filtering both sets. We could split them
|
||||
// out, but there is plenty of overlap and no name collisions so it doesn't matter very much.
|
||||
private static ImmutableSet<String> IGNORED_ENTITIES =
|
||||
Streams.concat(
|
||||
ImmutableSet.of(
|
||||
// These entities are @Embed-ded in Datastore
|
||||
"DelegationSignerData",
|
||||
"DomainDsDataHistory",
|
||||
"DomainTransactionRecord",
|
||||
"GracePeriod",
|
||||
"GracePeriodHistory",
|
||||
|
||||
// These entities are legitimately not comparable.
|
||||
"ClaimsEntry",
|
||||
"ClaimsList",
|
||||
"CommitLogBucket",
|
||||
"CommitLogManifest",
|
||||
"CommitLogMutation",
|
||||
"PremiumEntry",
|
||||
"ReservedListEntry")
|
||||
.stream(),
|
||||
NON_REPLICATED_TYPES.stream())
|
||||
.collect(toImmutableSet());
|
||||
|
||||
FakeClock clock;
|
||||
boolean replayed = false;
|
||||
boolean inOfyContext;
|
||||
InjectExtension injectExtension = new InjectExtension();
|
||||
@Nullable ReplicateToDatastoreAction sqlToDsReplicator;
|
||||
List<DomainBase> expectedUpdates = new ArrayList<>();
|
||||
boolean enableDomainTimestampChecks;
|
||||
boolean enableDatabaseCompare = true;
|
||||
|
||||
private ReplayExtension(FakeClock clock, @Nullable ReplicateToDatastoreAction sqlToDsReplicator) {
|
||||
this.clock = clock;
|
||||
this.sqlToDsReplicator = sqlToDsReplicator;
|
||||
}
|
||||
|
||||
public static ReplayExtension createWithCompare(FakeClock clock) {
|
||||
return new ReplayExtension(clock, null);
|
||||
}
|
||||
|
||||
// This allows us to disable the replay tests from an environment variable in specific
|
||||
// environments (namely kokoro) where we see flakiness of unknown origin.
|
||||
//
|
||||
// TODO(b/197534789): Remove this once we get to the bottom of test flakiness
|
||||
public static boolean replayTestsEnabled() {
|
||||
String disableReplayTests = System.getenv("NOMULUS_DISABLE_REPLAY_TESTS");
|
||||
if (disableReplayTests == null) {
|
||||
return true;
|
||||
}
|
||||
return !Ascii.toLowerCase(disableReplayTests).equals("true");
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a replay extension that replays from SQL to cloud datastore when running in SQL mode.
|
||||
*/
|
||||
public static ReplayExtension createWithDoubleReplay(FakeClock clock) {
|
||||
// TODO: use the proper double-replay extension when the tests are not flaky
|
||||
if (replayTestsEnabled()) {
|
||||
return new ReplayExtension(
|
||||
clock,
|
||||
new ReplicateToDatastoreAction(
|
||||
clock, Mockito.mock(RequestStatusChecker.class), new FakeResponse()));
|
||||
} else {
|
||||
return createWithCompare(clock);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Enable checking of domain timestamps during replay.
|
||||
*
|
||||
* <p>This was added to facilitate testing of a very specific bug wherein create/update
|
||||
* auto-timestamps serialized to the SQL -> DS Transaction table had different values from those
|
||||
* actually stored in SQL.
|
||||
*
|
||||
* <p>In order to use this, you also need to use expectUpdateFor() to store the states of a
|
||||
* DomainBase object at a given point in time.
|
||||
*/
|
||||
public void enableDomainTimestampChecks() {
|
||||
enableDomainTimestampChecks = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* If we're doing domain time checks, add the current state of a domain to check against.
|
||||
*
|
||||
* <p>A null argument is a placeholder to deal with b/217952766. Basically it allows us to ignore
|
||||
* one particular state in the sequence (where the timestamp is not what we expect it to be).
|
||||
*/
|
||||
public void expectUpdateFor(@Nullable DomainBase domain) {
|
||||
expectedUpdates.add(domain);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeEach(ExtensionContext context) {
|
||||
Optional<Method> elem = context.getTestMethod();
|
||||
if (elem.isPresent() && elem.get().isAnnotationPresent(NoDatabaseCompare.class)) {
|
||||
enableDatabaseCompare = false;
|
||||
}
|
||||
|
||||
// Use a single bucket to expose timestamp inversion problems. This typically happens when
|
||||
// a test with this extension rolls back the fake clock in the setup method, creating inverted
|
||||
// timestamp with the canned data preloaded by AppengineExtension. The solution is to move
|
||||
// the clock change to the test's constructor.
|
||||
injectExtension.setStaticField(
|
||||
CommitLogBucket.class, "bucketIdSupplier", Suppliers.ofInstance(1));
|
||||
DatabaseHelper.setClock(clock);
|
||||
DatabaseHelper.setAlwaysSaveWithBackup(true);
|
||||
ReplayQueue.clear();
|
||||
|
||||
// When running in JPA mode with double replay enabled, enable JPA transaction replication.
|
||||
// Note that we can't just use isOfy() here because this extension gets run before the dual-test
|
||||
// transaction manager gets injected.
|
||||
inOfyContext = DualDatabaseTestInvocationContextProvider.inOfyContext(context);
|
||||
if (sqlToDsReplicator != null && !inOfyContext) {
|
||||
JpaTransactionManagerImpl.setReplaySqlToDatastoreOverrideForTest(true);
|
||||
}
|
||||
|
||||
context.getStore(ExtensionContext.Namespace.GLOBAL).put(ReplayExtension.class, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterEach(ExtensionContext context) {
|
||||
// This ensures that we do the replay even if we're not called from AppEngineExtension. It
|
||||
// is safe to call replay() twice, as the method ensures idempotence.
|
||||
replay();
|
||||
injectExtension.afterEach(context);
|
||||
if (sqlToDsReplicator != null) {
|
||||
JpaTransactionManagerImpl.removeReplaySqlToDsOverrideForTest();
|
||||
}
|
||||
}
|
||||
|
||||
public void replay() {
|
||||
if (!replayed) {
|
||||
if (inOfyContext) {
|
||||
replayToSql();
|
||||
} else {
|
||||
// Disable database backups. For unknown reason, if we don't do this we get residual commit
|
||||
// log entries that cause timestamp inversions in other tests.
|
||||
DatabaseHelper.setAlwaysSaveWithBackup(false);
|
||||
|
||||
// Do the ofy replay.
|
||||
replayToOfy();
|
||||
|
||||
// Clean out anything that ends up in the replay queue.
|
||||
ReplayQueue.clear();
|
||||
}
|
||||
replayed = true;
|
||||
}
|
||||
}
|
||||
|
||||
private void replayToSql() {
|
||||
DatabaseHelper.setAlwaysSaveWithBackup(false);
|
||||
ImmutableMap<Key<?>, Object> changes = ReplayQueue.replay();
|
||||
|
||||
// Compare JPA to OFY, if requested.
|
||||
for (ImmutableMap.Entry<Key<?>, Object> entry : changes.entrySet()) {
|
||||
// Don't verify non-replicated types.
|
||||
if (NON_REPLICATED_TYPES.contains(entry.getKey().getKind())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// Since the object may have changed in datastore by the time we're doing the replay, we
|
||||
// have to compare the current value in SQL (which we just mutated) against the value that
|
||||
// we originally would have persisted (that being the object in the entry).
|
||||
VKey<?> vkey = VKey.from(entry.getKey());
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> {
|
||||
Optional<?> jpaValue = jpaTm().loadByKeyIfPresent(vkey);
|
||||
if (entry.getValue().equals(TransactionInfo.Delete.SENTINEL)) {
|
||||
assertThat(jpaValue.isPresent()).isFalse();
|
||||
} else {
|
||||
ImmutableObject immutJpaObject = (ImmutableObject) jpaValue.get();
|
||||
assertAboutImmutableObjects().that(immutJpaObject).hasCorrectHashValue();
|
||||
assertAboutImmutableObjects()
|
||||
.that(immutJpaObject)
|
||||
.isEqualAcrossDatabases(
|
||||
(ImmutableObject)
|
||||
((DatastoreEntity) entry.getValue()).toSqlEntity().get());
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private void replayToOfy() {
|
||||
if (sqlToDsReplicator == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
List<TransactionEntity> transactionBatch;
|
||||
do {
|
||||
transactionBatch = sqlToDsReplicator.getTransactionBatchAtSnapshot();
|
||||
for (TransactionEntity txn : transactionBatch) {
|
||||
ReplicateToDatastoreAction.applyTransaction(txn);
|
||||
ofyTm().transact(() -> compareSqlTransaction(txn));
|
||||
clock.advanceOneMilli();
|
||||
}
|
||||
} while (!transactionBatch.isEmpty());
|
||||
|
||||
// Now that everything has been replayed, compare the databases.
|
||||
if (enableDatabaseCompare) {
|
||||
compareDatabases();
|
||||
}
|
||||
}
|
||||
|
||||
/** Verifies that the replaying the SQL transaction created the same entities in Datastore. */
|
||||
private void compareSqlTransaction(TransactionEntity transactionEntity) {
|
||||
Transaction transaction;
|
||||
try {
|
||||
transaction = Transaction.deserialize(transactionEntity.getContents());
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Error during transaction deserialization.", e);
|
||||
}
|
||||
for (Mutation mutation : transaction.getMutations()) {
|
||||
if (mutation instanceof Update) {
|
||||
Update update = (Update) mutation;
|
||||
ImmutableObject fromTransactionEntity = (ImmutableObject) update.getEntity();
|
||||
ImmutableObject fromDatastore = ofyTm().loadByEntity(fromTransactionEntity);
|
||||
if (fromDatastore instanceof SqlEntity) {
|
||||
// We store the Datastore entity in the transaction, so use that if necessary
|
||||
fromDatastore = (ImmutableObject) ((SqlEntity) fromDatastore).toDatastoreEntity().get();
|
||||
}
|
||||
assertAboutImmutableObjects().that(fromDatastore).hasCorrectHashValue();
|
||||
assertAboutImmutableObjects()
|
||||
.that(fromDatastore)
|
||||
.isEqualAcrossDatabases(fromTransactionEntity);
|
||||
|
||||
// Check DomainBase timestamps if appropriate.
|
||||
if (enableDomainTimestampChecks && fromTransactionEntity instanceof DomainBase) {
|
||||
DomainBase expectedDomain = expectedUpdates.remove(0);
|
||||
|
||||
// Just skip it if the expectedDomain is null.
|
||||
if (expectedDomain == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
DomainBase domainEntity = (DomainBase) fromTransactionEntity;
|
||||
assertThat(domainEntity.getCreationTime()).isEqualTo(expectedDomain.getCreationTime());
|
||||
assertThat(domainEntity.getUpdateTimestamp())
|
||||
.isEqualTo(expectedDomain.getUpdateTimestamp());
|
||||
}
|
||||
} else {
|
||||
Delete delete = (Delete) mutation;
|
||||
VKey<?> key = delete.getKey();
|
||||
assertWithMessage(String.format("Expected key %s to not exist in Datastore", key))
|
||||
.that(ofyTm().exists(key))
|
||||
.isFalse();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Compares the final state of both databases after replay is complete. */
|
||||
private void compareDatabases() {
|
||||
boolean gotDiffs = false;
|
||||
|
||||
// Build a map containing all of the SQL entities indexed by their key.
|
||||
HashMap<Object, Object> sqlEntities = new HashMap<>();
|
||||
for (Class<?> cls : JpaEntityCoverageExtension.ALL_JPA_ENTITIES) {
|
||||
if (IGNORED_ENTITIES.contains(cls.getSimpleName())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
jpaTm()
|
||||
.transact(
|
||||
() -> jpaTm().loadAllOfStream(cls).forEach(e -> sqlEntities.put(getSqlKey(e), e)));
|
||||
}
|
||||
|
||||
for (Class<? extends ImmutableObject> cls : EntityClasses.ALL_CLASSES) {
|
||||
if (IGNORED_ENTITIES.contains(cls.getSimpleName())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
for (ImmutableObject entity : auditedOfy().load().type(cls).list()) {
|
||||
// Find the entity in SQL and verify that it's the same.
|
||||
Key<?> ofyKey = Key.create(entity);
|
||||
Object sqlKey = VKey.from(ofyKey).getSqlKey();
|
||||
ImmutableObject sqlEntity = (ImmutableObject) sqlEntities.get(sqlKey);
|
||||
Optional<SqlEntity> expectedSqlEntity = ((DatastoreEntity) entity).toSqlEntity();
|
||||
if (expectedSqlEntity.isPresent()) {
|
||||
// Check for null just so we get a better error message.
|
||||
if (sqlEntity == null) {
|
||||
logger.atSevere().log("Entity %s is in Datastore but not in SQL.", ofyKey);
|
||||
gotDiffs = true;
|
||||
} else {
|
||||
try {
|
||||
assertAboutImmutableObjects()
|
||||
.that((ImmutableObject) expectedSqlEntity.get())
|
||||
.isEqualAcrossDatabases(sqlEntity);
|
||||
} catch (AssertionError e) {
|
||||
// Show the message but swallow the stack trace (we'll get that from the fail() at
|
||||
// the end of the comparison).
|
||||
logger.atSevere().log("For entity %s: %s", ofyKey, e.getMessage());
|
||||
gotDiffs = true;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
logger.atInfo().log("Datastore entity has no sql representation for %s", ofyKey);
|
||||
}
|
||||
sqlEntities.remove(sqlKey);
|
||||
}
|
||||
}
|
||||
|
||||
// Report any objects in the SQL set that we didn't remove while iterating over the Datastore
|
||||
// objects.
|
||||
if (!sqlEntities.isEmpty()) {
|
||||
for (Object item : sqlEntities.values()) {
|
||||
logger.atSevere().log(
|
||||
"Entity of %s found in SQL but not in datastore: %s", item.getClass().getName(), item);
|
||||
}
|
||||
gotDiffs = true;
|
||||
}
|
||||
|
||||
if (gotDiffs) {
|
||||
fail("There were differences between the final SQL and Datastore contents.");
|
||||
}
|
||||
}
|
||||
|
||||
private static Object getSqlKey(Object entity) {
|
||||
return jpaTm()
|
||||
.getEntityManager()
|
||||
.getEntityManagerFactory()
|
||||
.getPersistenceUnitUtil()
|
||||
.getIdentifier(entity);
|
||||
}
|
||||
|
||||
/** Annotation to use for test methods where we don't want to do a database comparison yet. */
|
||||
@Target({METHOD})
|
||||
@Retention(RUNTIME)
|
||||
@TestTemplate
|
||||
public @interface NoDatabaseCompare {}
|
||||
}
|
|
@ -34,10 +34,6 @@ import javax.persistence.Transient;
|
|||
@EntityForTesting
|
||||
public class TestObject extends ImmutableObject implements DatastoreAndSqlEntity {
|
||||
|
||||
public static int beforeSqlSaveCallCount;
|
||||
public static int beforeSqlDeleteCallCount;
|
||||
public static int beforeDatastoreSaveCallCount;
|
||||
|
||||
@Parent @Transient Key<EntityGroupRoot> parent;
|
||||
|
||||
@Id @javax.persistence.Id String id;
|
||||
|
@ -76,20 +72,6 @@ public class TestObject extends ImmutableObject implements DatastoreAndSqlEntity
|
|||
return instance;
|
||||
}
|
||||
|
||||
public static void beforeSqlDelete(VKey<TestObject> key) {
|
||||
beforeSqlDeleteCallCount++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeSqlSaveOnReplay() {
|
||||
beforeSqlSaveCallCount++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeDatastoreSaveOnReplay() {
|
||||
beforeDatastoreSaveCallCount++;
|
||||
}
|
||||
|
||||
/** A test @VirtualEntity model object, which should not be persisted. */
|
||||
@Entity
|
||||
@VirtualEntity
|
||||
|
|
|
@ -5,7 +5,7 @@
|
|||
</result>
|
||||
<msgQ count="1" id="1-3-EXAMPLE-5-6-2011">
|
||||
<qDate>2011-01-02T01:01:01Z</qDate>
|
||||
<msg>Domain test.example was deleted by registry administrator with final deletion effective: 2011-01-02T00:56:01.009Z</msg>
|
||||
<msg>Domain test.example was deleted by registry administrator with final deletion effective: 2011-01-02T00:56:01.008Z</msg>
|
||||
</msgQ>
|
||||
<resData>
|
||||
<domain:panData>
|
||||
|
|
|
@ -3,7 +3,6 @@ PATH CLASS
|
|||
/_dr/cron/commitLogFanout CommitLogFanoutAction GET y INTERNAL,API APP ADMIN
|
||||
/_dr/cron/fanout TldFanoutAction GET y INTERNAL,API APP ADMIN
|
||||
/_dr/cron/readDnsQueue ReadDnsQueueAction GET y INTERNAL,API APP ADMIN
|
||||
/_dr/cron/replicateToDatastore ReplicateToDatastoreAction GET y INTERNAL,API APP ADMIN
|
||||
/_dr/dnsRefresh RefreshDnsAction GET y INTERNAL,API APP ADMIN
|
||||
/_dr/task/backupDatastore BackupDatastoreAction POST y INTERNAL,API APP ADMIN
|
||||
/_dr/task/brdaCopy BrdaCopyAction POST y INTERNAL,API APP ADMIN
|
||||
|
@ -33,12 +32,10 @@ PATH CLASS
|
|||
/_dr/task/rdeUpload RdeUploadAction POST n INTERNAL,API APP ADMIN
|
||||
/_dr/task/refreshDnsOnHostRename RefreshDnsOnHostRenameAction GET n INTERNAL,API APP ADMIN
|
||||
/_dr/task/relockDomain RelockDomainAction POST y INTERNAL,API APP ADMIN
|
||||
/_dr/task/replayCommitLogsToSql ReplayCommitLogsToSqlAction POST y INTERNAL,API APP ADMIN
|
||||
/_dr/task/resaveAllEppResources ResaveAllEppResourcesAction GET n INTERNAL,API APP ADMIN
|
||||
/_dr/task/resaveAllEppResourcesPipeline ResaveAllEppResourcesPipelineAction GET n INTERNAL,API APP ADMIN
|
||||
/_dr/task/resaveEntity ResaveEntityAction POST n INTERNAL,API APP ADMIN
|
||||
/_dr/task/sendExpiringCertificateNotificationEmail SendExpiringCertificateNotificationEmailAction GET n INTERNAL,API APP ADMIN
|
||||
/_dr/task/syncDatastoreToSqlSnapshot SyncDatastoreToSqlSnapshotAction POST n INTERNAL,API APP ADMIN
|
||||
/_dr/task/syncGroupMembers SyncGroupMembersAction POST n INTERNAL,API APP ADMIN
|
||||
/_dr/task/syncRegistrarsSheet SyncRegistrarsSheetAction POST n INTERNAL,API APP ADMIN
|
||||
/_dr/task/tmchCrl TmchCrlAction POST y INTERNAL,API APP ADMIN
|
||||
|
|
|
@ -98,8 +98,6 @@ steps:
|
|||
google/registry/beam/invoicing_pipeline_metadata.json \
|
||||
google.registry.beam.rde.RdePipeline \
|
||||
google/registry/beam/rde_pipeline_metadata.json \
|
||||
google.registry.beam.comparedb.ValidateDatabasePipeline \
|
||||
google/registry/beam/validate_database_pipeline_metadata.json \
|
||||
google.registry.beam.resave.ResaveAllEppResourcesPipeline \
|
||||
google/registry/beam/resave_all_epp_resources_pipeline_metadata.json
|
||||
# Tentatively build and publish Cloud SQL schema jar here, before schema release
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue