mirror of
https://github.com/google/nomulus.git
synced 2025-07-31 06:56:28 +02:00
Delete code relating to SQL init and scheduling (#1661)
One of the more significant changes introduced in this PR is that we use SQL as the backing database in all tests unless otherwise specified, e.g. by using the TmOverrideExtension. We change various ofy-related tests to use this. This includes various changes: - Deletion of SqlEntity/DatastoreEntity and related classes. Includes any necessary changes because of that (e.g. getting a nice SQL key on error in RegistryJpaIO). - Deletion of classes that used libraries from the init-sql code (RefreshDnsOnHostRenameAction) - Removal of the JpaTransactionManager's backup implementation - Modification of RegistryJpaWriteTest to not use init-sql code - Removal of the Transaction class and related classes, however it does not remove the TransactionEntity class as that would require DB changes - Removal of anything related to the actual usage of the database migration schedule or read-only phases - Various test changes and fixes to account for the differences in SQL (like how foreign keys need to exist) This deliberately doesn't do anything to alter the objects actually stored in the DB yet, just how we use them
This commit is contained in:
parent
dcc11379c8
commit
2f8be045c7
232 changed files with 457 additions and 8245 deletions
|
@ -765,11 +765,6 @@ createUberJar(
|
||||||
// User should install gcloud and login to GCP before invoking this tasks.
|
// User should install gcloud and login to GCP before invoking this tasks.
|
||||||
if (environment == 'alpha') {
|
if (environment == 'alpha') {
|
||||||
def pipelines = [
|
def pipelines = [
|
||||||
initSql :
|
|
||||||
[
|
|
||||||
mainClass: 'google.registry.beam.initsql.InitSqlPipeline',
|
|
||||||
metaData : 'google/registry/beam/init_sql_pipeline_metadata.json'
|
|
||||||
],
|
|
||||||
bulkDeleteDatastore:
|
bulkDeleteDatastore:
|
||||||
[
|
[
|
||||||
mainClass: 'google.registry.beam.datastore.BulkDeleteDatastorePipeline',
|
mainClass: 'google.registry.beam.datastore.BulkDeleteDatastorePipeline',
|
||||||
|
@ -888,48 +883,6 @@ task buildToolImage(dependsOn: nomulus, type: Exec) {
|
||||||
commandLine 'docker', 'build', '-t', 'nomulus-tool', '.'
|
commandLine 'docker', 'build', '-t', 'nomulus-tool', '.'
|
||||||
}
|
}
|
||||||
|
|
||||||
task generateInitSqlPipelineGraph(type: FilteringTest) {
|
|
||||||
tests = ['InitSqlPipelineGraphTest.createPipeline_compareGraph']
|
|
||||||
ignoreFailures = true
|
|
||||||
}
|
|
||||||
|
|
||||||
task updateInitSqlPipelineGraph(type: Copy) {
|
|
||||||
def graphRelativePath = 'google/registry/beam/initsql/'
|
|
||||||
from ("${projectDir}/build/resources/test/${graphRelativePath}") {
|
|
||||||
include 'pipeline_curr.dot'
|
|
||||||
rename 'curr', 'golden'
|
|
||||||
}
|
|
||||||
into "src/test/resources/${graphRelativePath}"
|
|
||||||
|
|
||||||
dependsOn generateInitSqlPipelineGraph
|
|
||||||
|
|
||||||
doLast {
|
|
||||||
if (com.google.common.base.Strings.isNullOrEmpty(project.dot_path)) {
|
|
||||||
getLogger().info('Property dot_path is null. Not creating image for pipeline graph.')
|
|
||||||
}
|
|
||||||
def dotPath = project.dot_path
|
|
||||||
if (!new File(dotPath).exists()) {
|
|
||||||
throw new RuntimeException(
|
|
||||||
"""\
|
|
||||||
${dotPath} not found. Make sure graphviz is installed
|
|
||||||
and the dot_path property is set correctly."""
|
|
||||||
.stripIndent())
|
|
||||||
}
|
|
||||||
def goldenGraph = "src/test/resources/${graphRelativePath}/pipeline_golden.dot"
|
|
||||||
def goldenImage = "src/test/resources/${graphRelativePath}/pipeline_golden.png"
|
|
||||||
def cmd = "${dotPath} -Tpng -o \"${goldenImage}\" \"${goldenGraph}\""
|
|
||||||
try {
|
|
||||||
rootProject.ext.execInBash(cmd, projectDir)
|
|
||||||
} catch (Throwable throwable) {
|
|
||||||
throw new RuntimeException(
|
|
||||||
"""\
|
|
||||||
Failed to generate golden image with command ${cmd}
|
|
||||||
Error: ${throwable.getMessage()}
|
|
||||||
""")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Build the devtool jar.
|
// Build the devtool jar.
|
||||||
createUberJar(
|
createUberJar(
|
||||||
'devtool',
|
'devtool',
|
||||||
|
|
|
@ -1,140 +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.base.Preconditions.checkState;
|
|
||||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
|
||||||
import static google.registry.backup.BackupUtils.createDeserializingIterator;
|
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
|
||||||
import google.registry.model.ImmutableObject;
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import google.registry.model.ofy.CommitLogCheckpoint;
|
|
||||||
import google.registry.model.ofy.CommitLogManifest;
|
|
||||||
import google.registry.model.ofy.CommitLogMutation;
|
|
||||||
import java.io.BufferedInputStream;
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.FileInputStream;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.InputStream;
|
|
||||||
import java.nio.channels.Channels;
|
|
||||||
import java.nio.channels.ReadableByteChannel;
|
|
||||||
import java.util.Iterator;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Helpers for reading CommitLog records from a file.
|
|
||||||
*
|
|
||||||
* <p>This class is adapted from {@link RestoreCommitLogsAction}, and will be used in the initial
|
|
||||||
* population of the Cloud SQL database.
|
|
||||||
*/
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public final class CommitLogImports {
|
|
||||||
|
|
||||||
private CommitLogImports() {}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns entities in an {@code inputStream} (from a single CommitLog file) as an {@link
|
|
||||||
* ImmutableList} of {@link ImmutableList}s of {@link VersionedEntity} records where the inner
|
|
||||||
* lists each consist of one transaction. Upon completion the {@code inputStream} is closed.
|
|
||||||
*
|
|
||||||
* <p>The returned list may be empty, since CommitLogs are written at fixed intervals regardless
|
|
||||||
* if actual changes exist. Each sublist, however, will not be empty.
|
|
||||||
*
|
|
||||||
* <p>A CommitLog file starts with a {@link CommitLogCheckpoint}, followed by (repeated)
|
|
||||||
* subsequences of [{@link CommitLogManifest}, [{@link CommitLogMutation}] ...]. Each subsequence
|
|
||||||
* represents the changes in one transaction. The {@code CommitLogManifest} contains deleted
|
|
||||||
* entity keys, whereas each {@code CommitLogMutation} contains one whole entity.
|
|
||||||
*/
|
|
||||||
static ImmutableList<ImmutableList<VersionedEntity>> loadEntitiesByTransaction(
|
|
||||||
InputStream inputStream) {
|
|
||||||
try (InputStream input = new BufferedInputStream(inputStream)) {
|
|
||||||
Iterator<ImmutableObject> commitLogs = createDeserializingIterator(input, false);
|
|
||||||
checkState(commitLogs.hasNext());
|
|
||||||
checkState(commitLogs.next() instanceof CommitLogCheckpoint);
|
|
||||||
|
|
||||||
ImmutableList.Builder<ImmutableList<VersionedEntity>> resultBuilder =
|
|
||||||
new ImmutableList.Builder<>();
|
|
||||||
ImmutableList.Builder<VersionedEntity> currentTransactionBuilder =
|
|
||||||
new ImmutableList.Builder<>();
|
|
||||||
|
|
||||||
while (commitLogs.hasNext()) {
|
|
||||||
ImmutableObject currentObject = commitLogs.next();
|
|
||||||
if (currentObject instanceof CommitLogManifest) {
|
|
||||||
// CommitLogManifest means we are starting a new transaction
|
|
||||||
addIfNonempty(resultBuilder, currentTransactionBuilder);
|
|
||||||
currentTransactionBuilder = new ImmutableList.Builder<>();
|
|
||||||
VersionedEntity.fromManifest((CommitLogManifest) currentObject)
|
|
||||||
.forEach(currentTransactionBuilder::add);
|
|
||||||
} else if (currentObject instanceof CommitLogMutation) {
|
|
||||||
currentTransactionBuilder.add(
|
|
||||||
VersionedEntity.fromMutation((CommitLogMutation) currentObject));
|
|
||||||
} else {
|
|
||||||
throw new IllegalStateException(
|
|
||||||
String.format("Unknown entity type %s in commit logs", currentObject.getClass()));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Add the last transaction in (if it's not empty)
|
|
||||||
addIfNonempty(resultBuilder, currentTransactionBuilder);
|
|
||||||
return resultBuilder.build();
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns entities in an {@code inputStream} (from a single CommitLog file) as an {@link
|
|
||||||
* ImmutableList} of {@link VersionedEntity} records. Upon completion the {@code inputStream} is
|
|
||||||
* closed.
|
|
||||||
*
|
|
||||||
* <p>The returned list may be empty, since CommitLogs are written at fixed intervals regardless
|
|
||||||
* if actual changes exist.
|
|
||||||
*
|
|
||||||
* <p>A CommitLog file starts with a {@link CommitLogCheckpoint}, followed by (repeated)
|
|
||||||
* subsequences of [{@link CommitLogManifest}, [{@link CommitLogMutation}] ...]. Each subsequence
|
|
||||||
* represents the changes in one transaction. The {@code CommitLogManifest} contains deleted
|
|
||||||
* entity keys, whereas each {@code CommitLogMutation} contains one whole entity.
|
|
||||||
*/
|
|
||||||
static ImmutableList<VersionedEntity> loadEntities(InputStream inputStream) {
|
|
||||||
return loadEntitiesByTransaction(inputStream).stream()
|
|
||||||
.flatMap(ImmutableList::stream)
|
|
||||||
.collect(toImmutableList());
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Covenience method that adapts {@link #loadEntities(InputStream)} to a {@link File}. */
|
|
||||||
public static ImmutableList<VersionedEntity> loadEntities(File commitLogFile) {
|
|
||||||
try {
|
|
||||||
return loadEntities(new FileInputStream(commitLogFile));
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Covenience method that adapts {@link #loadEntities(InputStream)} to a {@link
|
|
||||||
* ReadableByteChannel}.
|
|
||||||
*/
|
|
||||||
public static ImmutableList<VersionedEntity> loadEntities(ReadableByteChannel channel) {
|
|
||||||
return loadEntities(Channels.newInputStream(channel));
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void addIfNonempty(
|
|
||||||
ImmutableList.Builder<ImmutableList<VersionedEntity>> resultBuilder,
|
|
||||||
ImmutableList.Builder<VersionedEntity> currentTransactionBuilder) {
|
|
||||||
ImmutableList<VersionedEntity> currentTransaction = currentTransactionBuilder.build();
|
|
||||||
if (!currentTransaction.isEmpty()) {
|
|
||||||
resultBuilder.add(currentTransaction);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,379 +0,0 @@
|
||||||
// Copyright 2017 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.batch;
|
|
||||||
|
|
||||||
import static com.google.appengine.api.taskqueue.QueueConstants.maxLeaseCount;
|
|
||||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
|
||||||
import static com.google.common.base.Preconditions.checkNotNull;
|
|
||||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
|
||||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_HOST_KEY;
|
|
||||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTED_TIME;
|
|
||||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
|
||||||
import static google.registry.batch.AsyncTaskMetrics.OperationType.DNS_REFRESH;
|
|
||||||
import static google.registry.mapreduce.inputs.EppResourceInputs.createEntityInput;
|
|
||||||
import static google.registry.model.EppResourceUtils.getLinkedDomainKeys;
|
|
||||||
import static google.registry.model.EppResourceUtils.isActive;
|
|
||||||
import static google.registry.model.EppResourceUtils.isDeleted;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
|
||||||
import static google.registry.util.DateTimeUtils.latestOf;
|
|
||||||
import static java.util.concurrent.TimeUnit.DAYS;
|
|
||||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
|
||||||
import static java.util.logging.Level.INFO;
|
|
||||||
import static java.util.logging.Level.SEVERE;
|
|
||||||
import static org.joda.time.Duration.standardHours;
|
|
||||||
|
|
||||||
import com.google.appengine.api.taskqueue.LeaseOptions;
|
|
||||||
import com.google.appengine.api.taskqueue.Queue;
|
|
||||||
import com.google.appengine.api.taskqueue.TaskHandle;
|
|
||||||
import com.google.appengine.api.taskqueue.TransientFailureException;
|
|
||||||
import com.google.appengine.tools.mapreduce.Mapper;
|
|
||||||
import com.google.appengine.tools.mapreduce.Reducer;
|
|
||||||
import com.google.appengine.tools.mapreduce.ReducerInput;
|
|
||||||
import com.google.auto.value.AutoValue;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
|
||||||
import com.google.common.flogger.FluentLogger;
|
|
||||||
import google.registry.batch.AsyncTaskMetrics.OperationResult;
|
|
||||||
import google.registry.dns.DnsQueue;
|
|
||||||
import google.registry.mapreduce.MapreduceRunner;
|
|
||||||
import google.registry.mapreduce.inputs.NullInput;
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import google.registry.model.domain.DomainBase;
|
|
||||||
import google.registry.model.host.HostResource;
|
|
||||||
import google.registry.model.server.Lock;
|
|
||||||
import google.registry.persistence.VKey;
|
|
||||||
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.NonFinalForTesting;
|
|
||||||
import google.registry.util.RequestStatusChecker;
|
|
||||||
import google.registry.util.Retrier;
|
|
||||||
import google.registry.util.SystemClock;
|
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.NoSuchElementException;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.logging.Level;
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import javax.inject.Inject;
|
|
||||||
import javax.inject.Named;
|
|
||||||
import org.apache.http.HttpStatus;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
import org.joda.time.Duration;
|
|
||||||
|
|
||||||
/** Performs batched DNS refreshes for applicable domains following a host rename. */
|
|
||||||
@Action(
|
|
||||||
service = Action.Service.BACKEND,
|
|
||||||
path = "/_dr/task/refreshDnsOnHostRename",
|
|
||||||
auth = Auth.AUTH_INTERNAL_OR_ADMIN)
|
|
||||||
@DeleteAfterMigration
|
|
||||||
@Deprecated
|
|
||||||
public class RefreshDnsOnHostRenameAction implements Runnable {
|
|
||||||
|
|
||||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
|
||||||
private static final Duration LEASE_LENGTH = standardHours(4);
|
|
||||||
|
|
||||||
@Inject AsyncTaskMetrics asyncTaskMetrics;
|
|
||||||
@Inject Clock clock;
|
|
||||||
@Inject MapreduceRunner mrRunner;
|
|
||||||
@Inject @Named(QUEUE_ASYNC_HOST_RENAME) Queue pullQueue;
|
|
||||||
|
|
||||||
@Inject DnsQueue dnsQueue;
|
|
||||||
@Inject RequestStatusChecker requestStatusChecker;
|
|
||||||
@Inject Response response;
|
|
||||||
@Inject Retrier retrier;
|
|
||||||
@Inject RefreshDnsOnHostRenameAction() {}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
// Check if the lock can be acquired, and if not, a previous run of this mapreduce is still
|
|
||||||
// executing, so return early.
|
|
||||||
Optional<Lock> lock =
|
|
||||||
Lock.acquire(
|
|
||||||
RefreshDnsOnHostRenameAction.class.getSimpleName(),
|
|
||||||
null,
|
|
||||||
LEASE_LENGTH,
|
|
||||||
requestStatusChecker,
|
|
||||||
false);
|
|
||||||
|
|
||||||
if (!lock.isPresent()) {
|
|
||||||
logRespondAndUnlock(INFO, "Can't acquire lock; aborting.", lock);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Lease the async tasks to process.
|
|
||||||
LeaseOptions options =
|
|
||||||
LeaseOptions.Builder.withCountLimit(maxLeaseCount())
|
|
||||||
.leasePeriod(LEASE_LENGTH.getStandardSeconds(), SECONDS);
|
|
||||||
List<TaskHandle> tasks = pullQueue.leaseTasks(options);
|
|
||||||
asyncTaskMetrics.recordDnsRefreshBatchSize(tasks.size());
|
|
||||||
|
|
||||||
// Check if there are no tasks to process, and if so, return early.
|
|
||||||
if (tasks.isEmpty()) {
|
|
||||||
logRespondAndUnlock(
|
|
||||||
INFO, "No DNS refresh on host rename tasks to process in pull queue; finishing.", lock);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
ImmutableList.Builder<DnsRefreshRequest> requestsBuilder = new ImmutableList.Builder<>();
|
|
||||||
ImmutableList.Builder<VKey<HostResource>> hostKeys = new ImmutableList.Builder<>();
|
|
||||||
final List<DnsRefreshRequest> requestsToDelete = new ArrayList<>();
|
|
||||||
|
|
||||||
for (TaskHandle task : tasks) {
|
|
||||||
try {
|
|
||||||
DnsRefreshRequest request = DnsRefreshRequest.createFromTask(task, clock.nowUtc());
|
|
||||||
if (request.isRefreshNeeded()) {
|
|
||||||
requestsBuilder.add(request);
|
|
||||||
hostKeys.add(request.hostKey());
|
|
||||||
} else {
|
|
||||||
// Skip hosts that are deleted.
|
|
||||||
requestsToDelete.add(request);
|
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
logger.atSevere().withCause(e).log(
|
|
||||||
"Could not parse DNS refresh for host request, delaying task for a day: %s", task);
|
|
||||||
// Grab the lease for a whole day, so it won't continue throwing errors every five minutes.
|
|
||||||
pullQueue.modifyTaskLease(task, 1L, DAYS);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
deleteTasksWithRetry(
|
|
||||||
requestsToDelete, pullQueue, asyncTaskMetrics, retrier, OperationResult.STALE);
|
|
||||||
ImmutableList<DnsRefreshRequest> refreshRequests = requestsBuilder.build();
|
|
||||||
if (refreshRequests.isEmpty()) {
|
|
||||||
logRespondAndUnlock(
|
|
||||||
INFO, "No async DNS refreshes to process because all renamed hosts are deleted.", lock);
|
|
||||||
} else {
|
|
||||||
logger.atInfo().log(
|
|
||||||
"Processing asynchronous DNS refresh for renamed hosts: %s", hostKeys.build());
|
|
||||||
if (tm().isOfy()) {
|
|
||||||
runMapreduce(refreshRequests, lock);
|
|
||||||
} else {
|
|
||||||
try {
|
|
||||||
refreshRequests.stream()
|
|
||||||
.flatMap(
|
|
||||||
request ->
|
|
||||||
getLinkedDomainKeys(request.hostKey(), request.lastUpdateTime(), null)
|
|
||||||
.stream())
|
|
||||||
.distinct()
|
|
||||||
.map(domainKey -> tm().transact(() -> tm().loadByKey(domainKey).getDomainName()))
|
|
||||||
.forEach(
|
|
||||||
domainName -> {
|
|
||||||
retrier.callWithRetry(
|
|
||||||
() -> dnsQueue.addDomainRefreshTask(domainName),
|
|
||||||
TransientFailureException.class);
|
|
||||||
logger.atInfo().log("Enqueued DNS refresh for domain '%s'.", domainName);
|
|
||||||
});
|
|
||||||
deleteTasksWithRetry(
|
|
||||||
refreshRequests,
|
|
||||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
|
||||||
asyncTaskMetrics,
|
|
||||||
retrier,
|
|
||||||
OperationResult.SUCCESS);
|
|
||||||
} catch (Throwable t) {
|
|
||||||
String message = "Error refreshing DNS on host rename.";
|
|
||||||
logger.atSevere().withCause(t).log(message);
|
|
||||||
response.setPayload(message);
|
|
||||||
response.setStatus(HttpStatus.SC_INTERNAL_SERVER_ERROR);
|
|
||||||
} finally {
|
|
||||||
lock.get().release();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void runMapreduce(ImmutableList<DnsRefreshRequest> refreshRequests, Optional<Lock> lock) {
|
|
||||||
try {
|
|
||||||
mrRunner
|
|
||||||
.setJobName("Enqueue DNS refreshes for domains referencing renamed hosts")
|
|
||||||
.setModuleName("backend")
|
|
||||||
.setDefaultReduceShards(1)
|
|
||||||
.runMapreduce(
|
|
||||||
new RefreshDnsOnHostRenameMapper(refreshRequests, retrier),
|
|
||||||
new RefreshDnsOnHostRenameReducer(refreshRequests, lock.get(), retrier),
|
|
||||||
// Add an extra NullInput so that the reducer always fires exactly once.
|
|
||||||
ImmutableList.of(new NullInput<>(), createEntityInput(DomainBase.class)))
|
|
||||||
.sendLinkToMapreduceConsole(response);
|
|
||||||
} catch (Throwable t) {
|
|
||||||
logRespondAndUnlock(
|
|
||||||
SEVERE, "Error starting mapreduce to refresh DNS for renamed hosts.", lock);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void logRespondAndUnlock(Level level, String message, Optional<Lock> lock) {
|
|
||||||
logger.at(level).log(message);
|
|
||||||
response.setPayload(message);
|
|
||||||
lock.ifPresent(Lock::release);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Map over domains and refresh the DNS of those that reference the renamed hosts. */
|
|
||||||
public static class RefreshDnsOnHostRenameMapper
|
|
||||||
extends Mapper<DomainBase, Boolean, Boolean> {
|
|
||||||
|
|
||||||
private static final long serialVersionUID = -5261698524424335531L;
|
|
||||||
private static final DnsQueue dnsQueue = DnsQueue.create();
|
|
||||||
|
|
||||||
private final ImmutableList<DnsRefreshRequest> refreshRequests;
|
|
||||||
private final Retrier retrier;
|
|
||||||
|
|
||||||
RefreshDnsOnHostRenameMapper(
|
|
||||||
ImmutableList<DnsRefreshRequest> refreshRequests, Retrier retrier) {
|
|
||||||
this.refreshRequests = refreshRequests;
|
|
||||||
this.retrier = retrier;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public final void map(@Nullable final DomainBase domain) {
|
|
||||||
if (domain == null) {
|
|
||||||
// Emit a single value so that the reducer always runs. The key and value don't matter.
|
|
||||||
emit(true, true);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
VKey<HostResource> referencingHostKey = null;
|
|
||||||
for (DnsRefreshRequest request : refreshRequests) {
|
|
||||||
if (isActive(domain, request.lastUpdateTime())
|
|
||||||
&& domain.getNameservers().contains(request.hostKey())) {
|
|
||||||
referencingHostKey = request.hostKey();
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (referencingHostKey != null) {
|
|
||||||
retrier.callWithRetry(
|
|
||||||
() -> dnsQueue.addDomainRefreshTask(domain.getDomainName()),
|
|
||||||
TransientFailureException.class);
|
|
||||||
logger.atInfo().log(
|
|
||||||
"Enqueued DNS refresh for domain %s referenced by host %s.",
|
|
||||||
domain.getDomainName(), referencingHostKey);
|
|
||||||
getContext().incrementCounter("domains refreshed");
|
|
||||||
} else {
|
|
||||||
getContext().incrementCounter("domains not refreshed");
|
|
||||||
}
|
|
||||||
|
|
||||||
// Don't catch errors -- we allow the mapreduce to terminate on any errors that can't be
|
|
||||||
// resolved by retrying the transaction. The reducer only fires if the mapper completes
|
|
||||||
// without errors, meaning that it is acceptable to delete all tasks.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A reducer that always fires exactly once.
|
|
||||||
*
|
|
||||||
* <p>This is really a reducer in name only; what it's really doing is waiting for all of the
|
|
||||||
* mapper tasks to finish, and then delete the pull queue tasks. Note that this only happens if
|
|
||||||
* the mapper completes execution without errors.
|
|
||||||
*/
|
|
||||||
public static class RefreshDnsOnHostRenameReducer extends Reducer<Boolean, Boolean, Void> {
|
|
||||||
|
|
||||||
private static final long serialVersionUID = 9077366205249562118L;
|
|
||||||
|
|
||||||
@NonFinalForTesting
|
|
||||||
private static AsyncTaskMetrics asyncTaskMetrics = new AsyncTaskMetrics(new SystemClock());
|
|
||||||
|
|
||||||
private final Lock lock;
|
|
||||||
private final Retrier retrier;
|
|
||||||
private final List<DnsRefreshRequest> refreshRequests;
|
|
||||||
|
|
||||||
RefreshDnsOnHostRenameReducer(
|
|
||||||
List<DnsRefreshRequest> refreshRequests, Lock lock, Retrier retrier) {
|
|
||||||
this.refreshRequests = refreshRequests;
|
|
||||||
this.lock = lock;
|
|
||||||
this.retrier = retrier;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void reduce(Boolean key, ReducerInput<Boolean> values) {
|
|
||||||
// The reduce() method is run precisely once, because the NullInput caused the mapper to emit
|
|
||||||
// a dummy value once.
|
|
||||||
deleteTasksWithRetry(
|
|
||||||
refreshRequests,
|
|
||||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
|
||||||
asyncTaskMetrics,
|
|
||||||
retrier,
|
|
||||||
OperationResult.SUCCESS);
|
|
||||||
|
|
||||||
lock.release();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Deletes a list of tasks from the given queue using a retrier. */
|
|
||||||
private static void deleteTasksWithRetry(
|
|
||||||
final List<DnsRefreshRequest> refreshRequests,
|
|
||||||
final Queue queue,
|
|
||||||
AsyncTaskMetrics asyncTaskMetrics,
|
|
||||||
Retrier retrier,
|
|
||||||
OperationResult result) {
|
|
||||||
if (refreshRequests.isEmpty()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
final List<TaskHandle> tasks =
|
|
||||||
refreshRequests.stream().map(DnsRefreshRequest::task).collect(toImmutableList());
|
|
||||||
retrier.callWithRetry(() -> queue.deleteTask(tasks), TransientFailureException.class);
|
|
||||||
refreshRequests.forEach(
|
|
||||||
r -> asyncTaskMetrics.recordAsyncFlowResult(DNS_REFRESH, result, r.requestedTime()));
|
|
||||||
}
|
|
||||||
|
|
||||||
/** A class that encapsulates the values of a request to refresh DNS for a renamed host. */
|
|
||||||
@AutoValue
|
|
||||||
abstract static class DnsRefreshRequest implements Serializable {
|
|
||||||
|
|
||||||
private static final long serialVersionUID = 1772812852271288622L;
|
|
||||||
|
|
||||||
abstract VKey<HostResource> hostKey();
|
|
||||||
|
|
||||||
abstract DateTime lastUpdateTime();
|
|
||||||
abstract DateTime requestedTime();
|
|
||||||
abstract boolean isRefreshNeeded();
|
|
||||||
abstract TaskHandle task();
|
|
||||||
|
|
||||||
@AutoValue.Builder
|
|
||||||
abstract static class Builder {
|
|
||||||
abstract Builder setHostKey(VKey<HostResource> hostKey);
|
|
||||||
|
|
||||||
abstract Builder setLastUpdateTime(DateTime lastUpdateTime);
|
|
||||||
abstract Builder setRequestedTime(DateTime requestedTime);
|
|
||||||
abstract Builder setIsRefreshNeeded(boolean isRefreshNeeded);
|
|
||||||
abstract Builder setTask(TaskHandle task);
|
|
||||||
abstract DnsRefreshRequest build();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a packaged-up {@link DnsRefreshRequest} parsed from a task queue task.
|
|
||||||
*/
|
|
||||||
static DnsRefreshRequest createFromTask(TaskHandle task, DateTime now) throws Exception {
|
|
||||||
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
|
||||||
VKey<HostResource> hostKey =
|
|
||||||
VKey.create(checkNotNull(params.get(PARAM_HOST_KEY), "Host to refresh not specified"));
|
|
||||||
HostResource host =
|
|
||||||
tm().transact(() -> tm().loadByKeyIfPresent(hostKey))
|
|
||||||
.orElseThrow(() -> new NoSuchElementException("Host to refresh doesn't exist"));
|
|
||||||
boolean isHostDeleted =
|
|
||||||
isDeleted(host, latestOf(now, host.getUpdateTimestamp().getTimestamp()));
|
|
||||||
if (isHostDeleted) {
|
|
||||||
logger.atInfo().log("Host %s is already deleted, not refreshing DNS.", hostKey);
|
|
||||||
}
|
|
||||||
return new AutoValue_RefreshDnsOnHostRenameAction_DnsRefreshRequest.Builder()
|
|
||||||
.setHostKey(hostKey)
|
|
||||||
.setLastUpdateTime(host.getUpdateTimestamp().getTimestamp())
|
|
||||||
.setRequestedTime(
|
|
||||||
DateTime.parse(
|
|
||||||
checkNotNull(params.get(PARAM_REQUESTED_TIME), "Requested time not specified")))
|
|
||||||
.setIsRefreshNeeded(!isHostDeleted)
|
|
||||||
.setTask(task)
|
|
||||||
.build();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -23,8 +23,6 @@ import com.google.common.collect.Streams;
|
||||||
import google.registry.beam.common.RegistryQuery.CriteriaQuerySupplier;
|
import google.registry.beam.common.RegistryQuery.CriteriaQuerySupplier;
|
||||||
import google.registry.model.UpdateAutoTimestamp;
|
import google.registry.model.UpdateAutoTimestamp;
|
||||||
import google.registry.model.UpdateAutoTimestamp.DisableAutoUpdateResource;
|
import google.registry.model.UpdateAutoTimestamp.DisableAutoUpdateResource;
|
||||||
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
|
||||||
import google.registry.model.replay.SqlEntity;
|
|
||||||
import google.registry.persistence.transaction.JpaTransactionManager;
|
import google.registry.persistence.transaction.JpaTransactionManager;
|
||||||
import google.registry.persistence.transaction.TransactionManagerFactory;
|
import google.registry.persistence.transaction.TransactionManagerFactory;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
@ -235,10 +233,6 @@ public final class RegistryJpaIO {
|
||||||
|
|
||||||
@ProcessElement
|
@ProcessElement
|
||||||
public void processElement(OutputReceiver<T> outputReceiver) {
|
public void processElement(OutputReceiver<T> outputReceiver) {
|
||||||
// Preload the migration schedule into cache, otherwise the cache loading query may happen
|
|
||||||
// before the setDatabaseSnapshot call in the transaction below, causing it to fail.
|
|
||||||
DatabaseMigrationStateSchedule.get();
|
|
||||||
|
|
||||||
jpaTm()
|
jpaTm()
|
||||||
.transactNoRetry(
|
.transactNoRetry(
|
||||||
() -> {
|
() -> {
|
||||||
|
@ -433,11 +427,23 @@ public final class RegistryJpaIO {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Returns this entity's primary key field(s) in a string. */
|
||||||
private String toEntityKeyString(Object entity) {
|
private String toEntityKeyString(Object entity) {
|
||||||
if (entity instanceof SqlEntity) {
|
try {
|
||||||
return ((SqlEntity) entity).getPrimaryKeyString();
|
return jpaTm()
|
||||||
|
.transact(
|
||||||
|
() ->
|
||||||
|
String.format(
|
||||||
|
"%s_%s",
|
||||||
|
entity.getClass().getSimpleName(),
|
||||||
|
jpaTm()
|
||||||
|
.getEntityManager()
|
||||||
|
.getEntityManagerFactory()
|
||||||
|
.getPersistenceUnitUtil()
|
||||||
|
.getIdentifier(entity)));
|
||||||
|
} catch (IllegalArgumentException e) {
|
||||||
|
return "Non-SqlEntity: " + entity;
|
||||||
}
|
}
|
||||||
return "Non-SqlEntity: " + String.valueOf(entity);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,119 +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.beam.initsql;
|
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkArgument;
|
|
||||||
import static com.google.common.base.Preconditions.checkNotNull;
|
|
||||||
import static com.google.common.base.Strings.isNullOrEmpty;
|
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
|
||||||
import com.google.common.collect.Streams;
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Helpers for determining the fully qualified paths to Nomulus backup files. A backup consists of a
|
|
||||||
* Datastore export and Nomulus CommitLogs that overlap with the export.
|
|
||||||
*/
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public final class BackupPaths {
|
|
||||||
|
|
||||||
private BackupPaths() {}
|
|
||||||
|
|
||||||
private static final String WILDCARD_CHAR = "*";
|
|
||||||
private static final String EXPORT_PATTERN_TEMPLATE = "%s/all_namespaces/kind_%s/output-%s";
|
|
||||||
|
|
||||||
public static final String COMMIT_LOG_NAME_PREFIX = "commit_diff_until_";
|
|
||||||
private static final String COMMIT_LOG_PATTERN_TEMPLATE = "%s/" + COMMIT_LOG_NAME_PREFIX + "*";
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Pattern of the per-project file with Cloud SQL connection information. To get a concrete path,
|
|
||||||
* user needs to provide the name of the environment, alpha, crash, sandbox, or production. This
|
|
||||||
* file is meant for applications without access to secrets stored in Datastore.
|
|
||||||
*
|
|
||||||
* <p>In production, this is an base-64 encoded encrypted file with one line, which contains
|
|
||||||
* space-separated values of Cloud SQL instance name, login, and password.
|
|
||||||
*
|
|
||||||
* <p>A plain text may be used for tests to a local database. Replace Cloud SQL instance name with
|
|
||||||
* JDBC URL.
|
|
||||||
*/
|
|
||||||
private static final String SQL_CONN_INFO_FILE_PATTERN =
|
|
||||||
"gs://domain-registry-dev-deploy/cloudsql-credentials/%s/admin_credential.enc";
|
|
||||||
|
|
||||||
private static final ImmutableSet<String> ALLOWED_ENV =
|
|
||||||
ImmutableSet.of("alpha", "crash", "sandbox", "production");
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a regex pattern that matches all Datastore export files of a given {@code kind}.
|
|
||||||
*
|
|
||||||
* @param exportDir path to the top directory of a Datastore export
|
|
||||||
* @param kind the 'kind' of the Datastore entity
|
|
||||||
*/
|
|
||||||
public static String getExportFileNamePattern(String exportDir, String kind) {
|
|
||||||
checkArgument(!isNullOrEmpty(exportDir), "Null or empty exportDir.");
|
|
||||||
checkArgument(!isNullOrEmpty(kind), "Null or empty kind.");
|
|
||||||
return String.format(EXPORT_PATTERN_TEMPLATE, exportDir, kind, WILDCARD_CHAR);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns an {@link ImmutableList} of regex patterns that match all Datastore export files of the
|
|
||||||
* given {@code kinds}.
|
|
||||||
*
|
|
||||||
* @param exportDir path to the top directory of a Datastore export
|
|
||||||
* @param kinds all entity 'kinds' to be matched
|
|
||||||
*/
|
|
||||||
public static ImmutableList<String> getExportFilePatterns(
|
|
||||||
String exportDir, Iterable<String> kinds) {
|
|
||||||
checkNotNull(kinds, "kinds");
|
|
||||||
return Streams.stream(kinds)
|
|
||||||
.map(kind -> getExportFileNamePattern(exportDir, kind))
|
|
||||||
.collect(ImmutableList.toImmutableList());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the fully qualified path of a Datastore export file with the given {@code kind} and
|
|
||||||
* {@code shard}.
|
|
||||||
*
|
|
||||||
* @param exportDir path to the top directory of a Datastore export
|
|
||||||
* @param kind the 'kind' of the Datastore entity
|
|
||||||
* @param shard an integer suffix of the file name
|
|
||||||
*/
|
|
||||||
public static String getExportFileNameByShard(String exportDir, String kind, int shard) {
|
|
||||||
checkArgument(!isNullOrEmpty(exportDir), "Null or empty exportDir.");
|
|
||||||
checkArgument(!isNullOrEmpty(kind), "Null or empty kind.");
|
|
||||||
checkArgument(shard >= 0, "Negative shard %s not allowed.", shard);
|
|
||||||
return String.format(EXPORT_PATTERN_TEMPLATE, exportDir, kind, shard);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns an {@link ImmutableList} of regex patterns that match all CommitLog files. */
|
|
||||||
public static ImmutableList<String> getCommitLogFilePatterns(String commitLogDir) {
|
|
||||||
return ImmutableList.of(String.format(COMMIT_LOG_PATTERN_TEMPLATE, commitLogDir));
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Gets the Commit timestamp from a CommitLog file name. */
|
|
||||||
public static DateTime getCommitLogTimestamp(String fileName) {
|
|
||||||
checkArgument(!isNullOrEmpty(fileName), "Null or empty fileName.");
|
|
||||||
int start = fileName.lastIndexOf(COMMIT_LOG_NAME_PREFIX);
|
|
||||||
checkArgument(start >= 0, "Illegal file name %s.", fileName);
|
|
||||||
return DateTime.parse(fileName.substring(start + COMMIT_LOG_NAME_PREFIX.length()));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static ImmutableList<String> getCloudSQLCredentialFilePatterns(String environmentName) {
|
|
||||||
checkArgument(
|
|
||||||
ALLOWED_ENV.contains(environmentName), "Invalid environment name %s", environmentName);
|
|
||||||
return ImmutableList.of(String.format(SQL_CONN_INFO_FILE_PATTERN, environmentName));
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,78 +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.beam.initsql;
|
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkArgument;
|
|
||||||
import static com.google.common.base.Preconditions.checkNotNull;
|
|
||||||
|
|
||||||
import com.google.appengine.api.datastore.Entity;
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.Objects;
|
|
||||||
|
|
||||||
/** Helper for manipulating {@code DomainBase} when migrating from Datastore to SQL database */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
final class DomainBaseUtil {
|
|
||||||
|
|
||||||
private DomainBaseUtil() {}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Removes properties that contain foreign keys from a Datastore {@link Entity} that represents an
|
|
||||||
* Ofy {@link google.registry.model.domain.DomainBase}. This breaks the cycle of foreign key
|
|
||||||
* constraints between entity kinds, allowing {@code DomainBases} to be inserted into the SQL
|
|
||||||
* database. See {@link InitSqlPipeline} for a use case, where the full {@code DomainBases} are
|
|
||||||
* written again during the last stage of the pipeline.
|
|
||||||
*
|
|
||||||
* <p>The returned object may be in bad state. Specifically, {@link
|
|
||||||
* google.registry.model.eppcommon.StatusValue#INACTIVE} is not added after name servers are
|
|
||||||
* removed. This only impacts tests that manipulate Datastore entities directly.
|
|
||||||
*
|
|
||||||
* <p>This operation is performed on an Datastore {@link Entity} instead of Ofy Java object
|
|
||||||
* because Objectify requires access to a Datastore service when converting an Ofy object to a
|
|
||||||
* Datastore {@code Entity}. If we insist on working with Objectify objects, we face a few
|
|
||||||
* unsatisfactory options:
|
|
||||||
*
|
|
||||||
* <ul>
|
|
||||||
* <li>Connect to our production Datastore, which incurs unnecessary security and code health
|
|
||||||
* risk.
|
|
||||||
* <li>Connect to a separate real Datastore instance, which is a waster and overkill.
|
|
||||||
* <li>Use an in-memory test Datastore, which is a project health risk in that the test
|
|
||||||
* Datastore would be added to Nomulus' production binary unless we create a separate
|
|
||||||
* project for this pipeline.
|
|
||||||
* </ul>
|
|
||||||
*
|
|
||||||
* <p>Given our use case, operating on Datastore entities is the best option.
|
|
||||||
*
|
|
||||||
* @throws IllegalArgumentException if input does not represent a DomainBase
|
|
||||||
*/
|
|
||||||
static Entity removeBillingAndPollAndHosts(Entity domainBase) {
|
|
||||||
checkNotNull(domainBase, "domainBase");
|
|
||||||
checkArgument(
|
|
||||||
Objects.equals(domainBase.getKind(), "DomainBase"),
|
|
||||||
"Expecting DomainBase, got %s",
|
|
||||||
domainBase.getKind());
|
|
||||||
Entity clone = domainBase.clone();
|
|
||||||
clone.removeProperty("autorenewBillingEvent");
|
|
||||||
clone.removeProperty("autorenewPollMessage");
|
|
||||||
clone.removeProperty("deletePollMessage");
|
|
||||||
clone.removeProperty("nsHosts");
|
|
||||||
domainBase.getProperties().keySet().stream()
|
|
||||||
.filter(s -> s.startsWith("transferData."))
|
|
||||||
.forEach(s -> clone.removeProperty(s));
|
|
||||||
domainBase.getProperties().keySet().stream()
|
|
||||||
.filter(s -> s.startsWith("gracePeriods."))
|
|
||||||
.forEach(s -> clone.removeProperty(s));
|
|
||||||
return clone;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,242 +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.beam.initsql;
|
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkArgument;
|
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
|
||||||
import com.googlecode.objectify.Key;
|
|
||||||
import google.registry.backup.VersionedEntity;
|
|
||||||
import google.registry.beam.common.RegistryJpaIO;
|
|
||||||
import google.registry.beam.initsql.Transforms.RemoveDomainBaseForeignKeys;
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import google.registry.model.billing.BillingEvent;
|
|
||||||
import google.registry.model.common.Cursor;
|
|
||||||
import google.registry.model.contact.ContactResource;
|
|
||||||
import google.registry.model.domain.DomainBase;
|
|
||||||
import google.registry.model.domain.token.AllocationToken;
|
|
||||||
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.reporting.HistoryEntry;
|
|
||||||
import google.registry.model.tld.Registry;
|
|
||||||
import google.registry.persistence.PersistenceModule.TransactionIsolationLevel;
|
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Optional;
|
|
||||||
import org.apache.beam.sdk.Pipeline;
|
|
||||||
import org.apache.beam.sdk.PipelineResult;
|
|
||||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
|
||||||
import org.apache.beam.sdk.transforms.ParDo;
|
|
||||||
import org.apache.beam.sdk.transforms.Wait;
|
|
||||||
import org.apache.beam.sdk.values.PCollection;
|
|
||||||
import org.apache.beam.sdk.values.PCollectionTuple;
|
|
||||||
import org.apache.beam.sdk.values.TupleTag;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A BEAM pipeline that populates a SQL database with data from a Datastore backup.
|
|
||||||
*
|
|
||||||
* <p>This pipeline migrates EPP resources and related entities that cross-reference each other. To
|
|
||||||
* avoid violating foreign key constraints, writes to SQL are ordered by entity kinds. In addition,
|
|
||||||
* the {@link DomainBase} kind is written twice (see details below). The write order is presented
|
|
||||||
* below. Although some kinds can be written concurrently, e.g. {@code ContactResource} and {@code
|
|
||||||
* RegistrarContact}, we do not expect any performance benefit since the limiting resource is the
|
|
||||||
* number of JDBC connections. Google internal users may refer to <a
|
|
||||||
* href="http://go/registry-r3-init-sql">the design doc</a> for more information.
|
|
||||||
*
|
|
||||||
* <ol>
|
|
||||||
* <li>{@link Registry}: Assumes that {@code PremiumList} and {@code ReservedList} have been set
|
|
||||||
* up in the SQL database.
|
|
||||||
* <li>{@link Cursor}: Logically can depend on {@code Registry}, but without foreign key.
|
|
||||||
* <li>{@link Registrar}: Logically depends on {@code Registry}, Foreign key not modeled yet.
|
|
||||||
* <li>{@link ContactResource}: references {@code Registrar}
|
|
||||||
* <li>{@link RegistrarContact}: references {@code Registrar}.
|
|
||||||
* <li>Cleansed {@link DomainBase}: with references to {@code BillingEvent}, {@code Recurring},
|
|
||||||
* {@code Cancellation} and {@code HostResource} removed, still references {@code Registrar}
|
|
||||||
* and {@code ContactResource}. The removal breaks circular Foreign Key references.
|
|
||||||
* <li>{@link HostResource}: references {@code DomainBase}.
|
|
||||||
* <li>{@link HistoryEntry}: maps to one of three SQL entity types and may reference {@code
|
|
||||||
* Registrar}, {@code ContactResource}, {@code HostResource}, and {@code DomainBase}.
|
|
||||||
* <li>{@link AllocationToken}: references {@code HistoryEntry}.
|
|
||||||
* <li>{@link BillingEvent.Recurring}: references {@code Registrar}, {@code DomainBase} and {@code
|
|
||||||
* HistoryEntry}.
|
|
||||||
* <li>{@link BillingEvent.OneTime}: references {@code Registrar}, {@code DomainBase}, {@code
|
|
||||||
* BillingEvent.Recurring}, {@code HistoryEntry} and {@code AllocationToken}.
|
|
||||||
* <li>{@link BillingEvent.Cancellation}: references {@code Registrar}, {@code DomainBase}, {@code
|
|
||||||
* BillingEvent.Recurring}, {@code BillingEvent.OneTime}, and {@code HistoryEntry}.
|
|
||||||
* <li>{@link PollMessage}: references {@code Registrar}, {@code DomainBase}, {@code
|
|
||||||
* ContactResource}, {@code HostResource}, and {@code HistoryEntry}.
|
|
||||||
* <li>{@link DomainBase}, original copy from Datastore.
|
|
||||||
* </ol>
|
|
||||||
*
|
|
||||||
* <p>This pipeline expects that the source Datastore has at least one entity in each of the types
|
|
||||||
* above. This assumption allows us to construct a simpler pipeline graph that can be visually
|
|
||||||
* examined, and is true in all intended use cases. However, tests must not violate this assumption
|
|
||||||
* when setting up data, otherwise they may run into foreign key constraint violations. The reason
|
|
||||||
* is that this pipeline uses the {@link Wait} transform to order the persistence by entity type.
|
|
||||||
* However, the wait is skipped if the target type has no data, resulting in subsequent entity types
|
|
||||||
* starting prematurely. E.g., if a Datastore has no {@code RegistrarContact} entities, the pipeline
|
|
||||||
* may start writing {@code DomainBase} entities before all {@code Registry}, {@code Registrar} and
|
|
||||||
* {@code ContactResource} entities have been persisted.
|
|
||||||
*/
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public class InitSqlPipeline implements Serializable {
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Datastore kinds to be written to the SQL database before the cleansed version of {@link
|
|
||||||
* DomainBase}.
|
|
||||||
*/
|
|
||||||
private static final ImmutableList<Class<?>> PHASE_ONE_ORDERED =
|
|
||||||
ImmutableList.of(
|
|
||||||
Registry.class,
|
|
||||||
Cursor.class,
|
|
||||||
Registrar.class,
|
|
||||||
ContactResource.class,
|
|
||||||
RegistrarContact.class);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Datastore kinds to be written to the SQL database after the cleansed version of {@link
|
|
||||||
* DomainBase}.
|
|
||||||
*/
|
|
||||||
private static final ImmutableList<Class<?>> PHASE_TWO_ORDERED =
|
|
||||||
ImmutableList.of(
|
|
||||||
HostResource.class,
|
|
||||||
HistoryEntry.class,
|
|
||||||
AllocationToken.class,
|
|
||||||
BillingEvent.Recurring.class,
|
|
||||||
BillingEvent.OneTime.class,
|
|
||||||
BillingEvent.Cancellation.class,
|
|
||||||
PollMessage.class,
|
|
||||||
DomainBase.class);
|
|
||||||
|
|
||||||
private final InitSqlPipelineOptions options;
|
|
||||||
|
|
||||||
InitSqlPipeline(InitSqlPipelineOptions options) {
|
|
||||||
this.options = options;
|
|
||||||
}
|
|
||||||
|
|
||||||
PipelineResult run() {
|
|
||||||
return run(Pipeline.create(options));
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
PipelineResult run(Pipeline pipeline) {
|
|
||||||
setupPipeline(pipeline);
|
|
||||||
return pipeline.run();
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
void setupPipeline(Pipeline pipeline) {
|
|
||||||
options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_READ_UNCOMMITTED);
|
|
||||||
PCollectionTuple datastoreSnapshot =
|
|
||||||
pipeline.apply(
|
|
||||||
"Load Datastore snapshot",
|
|
||||||
Transforms.loadDatastoreSnapshot(
|
|
||||||
options.getDatastoreExportDir(),
|
|
||||||
options.getCommitLogDir(),
|
|
||||||
DateTime.parse(options.getCommitLogStartTimestamp()),
|
|
||||||
DateTime.parse(options.getCommitLogEndTimestamp()),
|
|
||||||
ImmutableSet.<String>builder()
|
|
||||||
.add("DomainBase")
|
|
||||||
.addAll(toKindStrings(PHASE_ONE_ORDERED))
|
|
||||||
.addAll(toKindStrings(PHASE_TWO_ORDERED))
|
|
||||||
.build()));
|
|
||||||
|
|
||||||
// Set up the pipeline to write entity kinds from PHASE_ONE_ORDERED to SQL. Return an object
|
|
||||||
// that signals the completion of the phase.
|
|
||||||
PCollection<Void> blocker =
|
|
||||||
scheduleOnePhaseWrites(datastoreSnapshot, PHASE_ONE_ORDERED, Optional.empty(), null);
|
|
||||||
blocker =
|
|
||||||
writeToSql(
|
|
||||||
"DomainBase without circular foreign keys",
|
|
||||||
removeDomainBaseForeignKeys(datastoreSnapshot)
|
|
||||||
.apply("Wait on phase one", Wait.on(blocker)));
|
|
||||||
// Set up the pipeline to write entity kinds from PHASE_TWO_ORDERED to SQL. This phase won't
|
|
||||||
// start until all cleansed DomainBases have been written (started by line above).
|
|
||||||
scheduleOnePhaseWrites(
|
|
||||||
datastoreSnapshot, PHASE_TWO_ORDERED, Optional.of(blocker), "DomainBaseNoFkeys");
|
|
||||||
}
|
|
||||||
|
|
||||||
private PCollection<VersionedEntity> removeDomainBaseForeignKeys(
|
|
||||||
PCollectionTuple datastoreSnapshot) {
|
|
||||||
PCollection<VersionedEntity> domainBases =
|
|
||||||
datastoreSnapshot.get(Transforms.createTagForKind("DomainBase"));
|
|
||||||
return domainBases.apply(
|
|
||||||
"Remove circular foreign keys from DomainBase",
|
|
||||||
ParDo.of(new RemoveDomainBaseForeignKeys()));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Sets up the pipeline to write entities in {@code entityClasses} to SQL. Entities are written
|
|
||||||
* one kind at a time based on each kind's position in {@code entityClasses}. Concurrency exists
|
|
||||||
* within each kind.
|
|
||||||
*
|
|
||||||
* @param datastoreSnapshot the Datastore snapshot of all data to be migrated to SQL
|
|
||||||
* @param entityClasses the entity types in write order
|
|
||||||
* @param blockingPCollection the pipeline stage that blocks this phase
|
|
||||||
* @param blockingTag description of the stage (if exists) that blocks this phase. Needed for
|
|
||||||
* generating unique transform ids
|
|
||||||
* @return the output {@code PCollection} from the writing of the last entity kind. Other parts of
|
|
||||||
* the pipeline can {@link Wait} on this object
|
|
||||||
*/
|
|
||||||
private PCollection<Void> scheduleOnePhaseWrites(
|
|
||||||
PCollectionTuple datastoreSnapshot,
|
|
||||||
Collection<Class<?>> entityClasses,
|
|
||||||
Optional<PCollection<Void>> blockingPCollection,
|
|
||||||
String blockingTag) {
|
|
||||||
checkArgument(!entityClasses.isEmpty(), "Each phase must have at least one kind.");
|
|
||||||
ImmutableList<TupleTag<VersionedEntity>> tags =
|
|
||||||
toKindStrings(entityClasses).stream()
|
|
||||||
.map(Transforms::createTagForKind)
|
|
||||||
.collect(ImmutableList.toImmutableList());
|
|
||||||
|
|
||||||
PCollection<Void> prev = blockingPCollection.orElse(null);
|
|
||||||
String prevTag = blockingTag;
|
|
||||||
for (TupleTag<VersionedEntity> tag : tags) {
|
|
||||||
PCollection<VersionedEntity> curr = datastoreSnapshot.get(tag);
|
|
||||||
if (prev != null) {
|
|
||||||
curr = curr.apply("Wait on " + prevTag, Wait.on(prev));
|
|
||||||
}
|
|
||||||
prev = writeToSql(tag.getId(), curr);
|
|
||||||
prevTag = tag.getId();
|
|
||||||
}
|
|
||||||
return prev;
|
|
||||||
}
|
|
||||||
|
|
||||||
private PCollection<Void> writeToSql(String transformId, PCollection<VersionedEntity> data) {
|
|
||||||
return data.apply(
|
|
||||||
"Write to Sql: " + transformId,
|
|
||||||
RegistryJpaIO.<VersionedEntity>write()
|
|
||||||
.withName(transformId)
|
|
||||||
.withBatchSize(options.getSqlWriteBatchSize())
|
|
||||||
.withShards(options.getSqlWriteShards())
|
|
||||||
.withJpaConverter(Transforms::convertVersionedEntityToSqlEntity)
|
|
||||||
.disableUpdateAutoTimestamp());
|
|
||||||
}
|
|
||||||
|
|
||||||
private static ImmutableList<String> toKindStrings(Collection<Class<?>> entityClasses) {
|
|
||||||
return entityClasses.stream().map(Key::getKind).collect(ImmutableList.toImmutableList());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void main(String[] args) {
|
|
||||||
InitSqlPipelineOptions options =
|
|
||||||
PipelineOptionsFactory.fromArgs(args).withValidation().as(InitSqlPipelineOptions.class);
|
|
||||||
new InitSqlPipeline(options).run();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,47 +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.beam.initsql;
|
|
||||||
|
|
||||||
import google.registry.beam.common.RegistryPipelineOptions;
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import org.apache.beam.sdk.options.Description;
|
|
||||||
import org.apache.beam.sdk.options.Validation;
|
|
||||||
|
|
||||||
/** Pipeline options for {@link InitSqlPipeline} */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public interface InitSqlPipelineOptions extends RegistryPipelineOptions {
|
|
||||||
|
|
||||||
@Description("The root directory of the export to load.")
|
|
||||||
String getDatastoreExportDir();
|
|
||||||
|
|
||||||
void setDatastoreExportDir(String datastoreExportDir);
|
|
||||||
|
|
||||||
@Description("The directory that contains all CommitLog files.")
|
|
||||||
String getCommitLogDir();
|
|
||||||
|
|
||||||
void setCommitLogDir(String commitLogDir);
|
|
||||||
|
|
||||||
@Description("The earliest CommitLogs to load, in ISO8601 format.")
|
|
||||||
@Validation.Required
|
|
||||||
String getCommitLogStartTimestamp();
|
|
||||||
|
|
||||||
void setCommitLogStartTimestamp(String commitLogStartTimestamp);
|
|
||||||
|
|
||||||
@Description("The latest CommitLogs to load, in ISO8601 format.")
|
|
||||||
@Validation.Required
|
|
||||||
String getCommitLogEndTimestamp();
|
|
||||||
|
|
||||||
void setCommitLogEndTimestamp(String commitLogEndTimestamp);
|
|
||||||
}
|
|
|
@ -1,17 +0,0 @@
|
||||||
## Summary
|
|
||||||
|
|
||||||
This package contains a BEAM pipeline that populates a Cloud SQL database from a
|
|
||||||
Datastore backup. The pipeline uses an unsynchronized Datastore export and
|
|
||||||
overlapping CommitLogs generated by the Nomulus server to recreate a consistent
|
|
||||||
Datastore snapshot, and writes the data to a Cloud SQL instance.
|
|
||||||
|
|
||||||
## Pipeline Visualization
|
|
||||||
|
|
||||||
The golden flow graph of the InitSqlPipeline is saved both as a text-base
|
|
||||||
[DOT file](../../../../../../test/resources/google/registry/beam/initsql/pipeline_golden.dot)
|
|
||||||
and a
|
|
||||||
[.png file](../../../../../../test/resources/google/registry/beam/initsql/pipeline_golden.png).
|
|
||||||
A test compares the flow graph of the current pipeline with the golden graph,
|
|
||||||
and will fail if changes are detected. When this happens, run the Gradle task
|
|
||||||
':core:updateInitSqlPipelineGraph' to update the golden files and review the
|
|
||||||
changes.
|
|
|
@ -1,485 +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.beam.initsql;
|
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkArgument;
|
|
||||||
import static com.google.common.base.Preconditions.checkNotNull;
|
|
||||||
import static com.google.common.base.Preconditions.checkState;
|
|
||||||
import static google.registry.beam.initsql.BackupPaths.getCommitLogTimestamp;
|
|
||||||
import static google.registry.beam.initsql.BackupPaths.getExportFilePatterns;
|
|
||||||
import static google.registry.model.ofy.ObjectifyService.auditedOfy;
|
|
||||||
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
|
||||||
import static google.registry.util.DateTimeUtils.isBeforeOrAt;
|
|
||||||
import static google.registry.util.DomainNameUtils.canonicalizeHostname;
|
|
||||||
import static java.util.Comparator.comparing;
|
|
||||||
import static org.apache.beam.sdk.values.TypeDescriptors.kvs;
|
|
||||||
import static org.apache.beam.sdk.values.TypeDescriptors.strings;
|
|
||||||
|
|
||||||
import avro.shaded.com.google.common.collect.Iterators;
|
|
||||||
import com.google.appengine.api.datastore.Entity;
|
|
||||||
import com.google.appengine.api.datastore.EntityTranslator;
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
|
||||||
import com.google.common.collect.Streams;
|
|
||||||
import google.registry.backup.CommitLogImports;
|
|
||||||
import google.registry.backup.VersionedEntity;
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import google.registry.model.billing.BillingEvent.Flag;
|
|
||||||
import google.registry.model.billing.BillingEvent.Reason;
|
|
||||||
import google.registry.model.domain.DomainBase;
|
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.model.replay.SqlEntity;
|
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
|
||||||
import google.registry.tools.LevelDbLogReader;
|
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.function.Supplier;
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import org.apache.beam.sdk.coders.StringUtf8Coder;
|
|
||||||
import org.apache.beam.sdk.io.Compression;
|
|
||||||
import org.apache.beam.sdk.io.FileIO;
|
|
||||||
import org.apache.beam.sdk.io.FileIO.ReadableFile;
|
|
||||||
import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
|
|
||||||
import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
|
|
||||||
import org.apache.beam.sdk.transforms.Create;
|
|
||||||
import org.apache.beam.sdk.transforms.DoFn;
|
|
||||||
import org.apache.beam.sdk.transforms.Flatten;
|
|
||||||
import org.apache.beam.sdk.transforms.GroupByKey;
|
|
||||||
import org.apache.beam.sdk.transforms.MapElements;
|
|
||||||
import org.apache.beam.sdk.transforms.PTransform;
|
|
||||||
import org.apache.beam.sdk.transforms.ParDo;
|
|
||||||
import org.apache.beam.sdk.transforms.ProcessFunction;
|
|
||||||
import org.apache.beam.sdk.values.KV;
|
|
||||||
import org.apache.beam.sdk.values.PBegin;
|
|
||||||
import org.apache.beam.sdk.values.PCollection;
|
|
||||||
import org.apache.beam.sdk.values.PCollectionList;
|
|
||||||
import org.apache.beam.sdk.values.PCollectionTuple;
|
|
||||||
import org.apache.beam.sdk.values.TupleTag;
|
|
||||||
import org.apache.beam.sdk.values.TupleTagList;
|
|
||||||
import org.apache.beam.sdk.values.TypeDescriptor;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* {@link PTransform Pipeline transforms} used in pipelines that load from both Datastore export
|
|
||||||
* files and Nomulus CommitLog files.
|
|
||||||
*/
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public final class Transforms {
|
|
||||||
|
|
||||||
private Transforms() {}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The commitTimestamp assigned to all entities loaded from a Datastore export file. The exact
|
|
||||||
* value does not matter, but it must be lower than the timestamps of real CommitLog records.
|
|
||||||
*/
|
|
||||||
@VisibleForTesting static final long EXPORT_ENTITY_TIME_STAMP = START_OF_TIME.getMillis();
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a {@link TupleTag} that can be used to retrieve entities of the given {@code kind} from
|
|
||||||
* the Datastore snapshot returned by {@link #loadDatastoreSnapshot}.
|
|
||||||
*/
|
|
||||||
public static TupleTag<VersionedEntity> createTagForKind(String kind) {
|
|
||||||
// When used with PCollectionTuple the result must retain generic type information.
|
|
||||||
// Both the Generic param and the empty bracket below are important.
|
|
||||||
return new TupleTag<VersionedEntity>(Transforms.class.getSimpleName() + ":" + kind) {};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Composite {@link PTransform transform} that loads the Datastore snapshot right before {@code
|
|
||||||
* commitLogToTime} for caller specified {@code kinds}. The resulting snapshot has all changes
|
|
||||||
* that happened before {@code commitLogToTime}, and none at or after {@code commitLogToTime}.
|
|
||||||
*
|
|
||||||
* <p>Caller must provide the location of a Datastore export that started AFTER {@code
|
|
||||||
* commitLogFromTime} and completed BEFORE {@code commitLogToTime}, as well as the root directory
|
|
||||||
* of all CommitLog files.
|
|
||||||
*
|
|
||||||
* <p>Selection of {@code commitLogFromTime} and {@code commitLogToTime} should follow the
|
|
||||||
* guidelines below to ensure that all incremental changes concurrent with the export are covered:
|
|
||||||
*
|
|
||||||
* <ul>
|
|
||||||
* <li>Two or more CommitLogs should exist between {@code commitLogFromTime} and the starting
|
|
||||||
* time of the Datastore export. This ensures that the earlier CommitLog file was complete
|
|
||||||
* before the export started.
|
|
||||||
* <li>Two or more CommitLogs should exit between the export completion time and {@code
|
|
||||||
* commitLogToTime}.
|
|
||||||
* </ul>
|
|
||||||
*
|
|
||||||
* <p>The output from the returned transform is a {@link PCollectionTuple} consisting of {@link
|
|
||||||
* VersionedEntity VersionedEntities} grouped into {@link PCollection PCollections} by {@code
|
|
||||||
* kind}.
|
|
||||||
*/
|
|
||||||
public static PTransform<PBegin, PCollectionTuple> loadDatastoreSnapshot(
|
|
||||||
String exportDir,
|
|
||||||
String commitLogDir,
|
|
||||||
DateTime commitLogFromTime,
|
|
||||||
DateTime commitLogToTime,
|
|
||||||
Set<String> kinds) {
|
|
||||||
checkArgument(kinds != null && !kinds.isEmpty(), "At least one kind is expected.");
|
|
||||||
|
|
||||||
// Create tags to collect entities by kind in final step.
|
|
||||||
final ImmutableMap<String, TupleTag<VersionedEntity>> outputTags =
|
|
||||||
kinds.stream()
|
|
||||||
.collect(ImmutableMap.toImmutableMap(kind -> kind, Transforms::createTagForKind));
|
|
||||||
// Arbitrarily select one tag as mainOutTag and put the remaining ones in a TupleTagList.
|
|
||||||
// This separation is required by ParDo's config API.
|
|
||||||
Iterator<TupleTag<VersionedEntity>> tagsIt = outputTags.values().iterator();
|
|
||||||
final TupleTag<VersionedEntity> mainOutputTag = tagsIt.next();
|
|
||||||
final TupleTagList additionalTags = TupleTagList.of(ImmutableList.copyOf(tagsIt));
|
|
||||||
|
|
||||||
return new PTransform<PBegin, PCollectionTuple>() {
|
|
||||||
@Override
|
|
||||||
public PCollectionTuple expand(PBegin input) {
|
|
||||||
PCollection<VersionedEntity> exportedEntities =
|
|
||||||
input
|
|
||||||
.apply("Get export file patterns", getDatastoreExportFilePatterns(exportDir, kinds))
|
|
||||||
.apply("Find export files", getFilesByPatterns())
|
|
||||||
.apply("Load export data", loadExportDataFromFiles());
|
|
||||||
PCollection<VersionedEntity> commitLogEntities =
|
|
||||||
input
|
|
||||||
.apply("Get commitlog file patterns", getCommitLogFilePatterns(commitLogDir))
|
|
||||||
.apply("Find commitlog files", getFilesByPatterns())
|
|
||||||
.apply(
|
|
||||||
"Filter commitLog by time",
|
|
||||||
filterCommitLogsByTime(commitLogFromTime, commitLogToTime))
|
|
||||||
.apply("Load commitlog data", loadCommitLogsFromFiles(kinds));
|
|
||||||
return PCollectionList.of(exportedEntities)
|
|
||||||
.and(commitLogEntities)
|
|
||||||
.apply("Merge exports and CommitLogs", Flatten.pCollections())
|
|
||||||
.apply(
|
|
||||||
"Key entities by Datastore Keys",
|
|
||||||
// Converting to KV<String, VE> instead of KV<Key, VE> b/c default coder for Key
|
|
||||||
// (SerializableCoder) is not deterministic and cannot be used with GroupBy.
|
|
||||||
MapElements.into(kvs(strings(), TypeDescriptor.of(VersionedEntity.class)))
|
|
||||||
.via((VersionedEntity e) -> KV.of(e.key().toString(), e)))
|
|
||||||
.apply("Gather entities by key", GroupByKey.create())
|
|
||||||
.apply(
|
|
||||||
"Output latest version per entity",
|
|
||||||
ParDo.of(
|
|
||||||
new DoFn<KV<String, Iterable<VersionedEntity>>, VersionedEntity>() {
|
|
||||||
@ProcessElement
|
|
||||||
public void processElement(
|
|
||||||
@Element KV<String, Iterable<VersionedEntity>> kv,
|
|
||||||
MultiOutputReceiver out) {
|
|
||||||
Optional<VersionedEntity> latest =
|
|
||||||
Streams.stream(kv.getValue())
|
|
||||||
.sorted(comparing(VersionedEntity::commitTimeMills).reversed())
|
|
||||||
.findFirst();
|
|
||||||
// Throw to abort (after default retries). Investigate, fix, and rerun.
|
|
||||||
checkState(
|
|
||||||
latest.isPresent(), "Unexpected key with no data", kv.getKey());
|
|
||||||
if (latest.get().isDelete()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
String kind = latest.get().getEntity().get().getKind();
|
|
||||||
out.get(outputTags.get(kind)).output(latest.get());
|
|
||||||
}
|
|
||||||
})
|
|
||||||
.withOutputTags(mainOutputTag, additionalTags));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a {@link PTransform transform} that can generate a collection of patterns that match
|
|
||||||
* all Datastore CommitLog files.
|
|
||||||
*/
|
|
||||||
public static PTransform<PBegin, PCollection<String>> getCommitLogFilePatterns(
|
|
||||||
String commitLogDir) {
|
|
||||||
return toStringPCollection(BackupPaths.getCommitLogFilePatterns(commitLogDir));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a {@link PTransform transform} that can generate a collection of patterns that match
|
|
||||||
* all Datastore export files of the given {@code kinds}.
|
|
||||||
*/
|
|
||||||
public static PTransform<PBegin, PCollection<String>> getDatastoreExportFilePatterns(
|
|
||||||
String exportDir, Collection<String> kinds) {
|
|
||||||
return toStringPCollection(getExportFilePatterns(exportDir, kinds));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static PTransform<PBegin, PCollection<String>> getCloudSqlConnectionInfoFilePatterns(
|
|
||||||
String gcpProjectName) {
|
|
||||||
return toStringPCollection(BackupPaths.getCloudSQLCredentialFilePatterns(gcpProjectName));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a {@link PTransform} from file name patterns to file {@link Metadata Metadata records}.
|
|
||||||
*/
|
|
||||||
public static PTransform<PCollection<String>, PCollection<Metadata>> getFilesByPatterns() {
|
|
||||||
return new PTransform<PCollection<String>, PCollection<Metadata>>() {
|
|
||||||
@Override
|
|
||||||
public PCollection<Metadata> expand(PCollection<String> input) {
|
|
||||||
return input.apply(FileIO.matchAll().withEmptyMatchTreatment(EmptyMatchTreatment.ALLOW));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns CommitLog files with timestamps between {@code fromTime} (inclusive) and {@code
|
|
||||||
* endTime} (exclusive).
|
|
||||||
*/
|
|
||||||
public static PTransform<PCollection<? extends Metadata>, PCollection<Metadata>>
|
|
||||||
filterCommitLogsByTime(DateTime fromTime, DateTime toTime) {
|
|
||||||
return ParDo.of(new FilterCommitLogFileByTime(fromTime, toTime));
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns a {@link PTransform} from file {@link Metadata} to {@link VersionedEntity}. */
|
|
||||||
public static PTransform<PCollection<Metadata>, PCollection<VersionedEntity>>
|
|
||||||
loadExportDataFromFiles() {
|
|
||||||
return processFiles(
|
|
||||||
new BackupFileReader(
|
|
||||||
file ->
|
|
||||||
Iterators.transform(
|
|
||||||
LevelDbLogReader.from(file.open()),
|
|
||||||
(byte[] bytes) -> VersionedEntity.from(EXPORT_ENTITY_TIME_STAMP, bytes))));
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns a {@link PTransform} from file {@link Metadata} to {@link VersionedEntity}. */
|
|
||||||
public static PTransform<PCollection<Metadata>, PCollection<VersionedEntity>>
|
|
||||||
loadCommitLogsFromFiles(Set<String> kinds) {
|
|
||||||
return processFiles(
|
|
||||||
new BackupFileReader(
|
|
||||||
file ->
|
|
||||||
CommitLogImports.loadEntities(file.open()).stream()
|
|
||||||
.filter(e -> kinds.contains(e.key().getKind()))
|
|
||||||
.iterator()));
|
|
||||||
}
|
|
||||||
|
|
||||||
// Production data repair configs go below. See b/185954992. Note that the CommitLog replay
|
|
||||||
// process does not filter out the ignored entities listed below, a mistake that we do not fix
|
|
||||||
// for operational convenience. Instead, the Database comparison tool will filter them out. See
|
|
||||||
// ValidateSqlUtils.java for more information.
|
|
||||||
|
|
||||||
// Prober domains in bad state, without associated contacts, hosts, billings, and non-synthesized
|
|
||||||
// history. They can be safely ignored.
|
|
||||||
public static final ImmutableSet<String> IGNORED_DOMAINS =
|
|
||||||
ImmutableSet.of("6AF6D2-IQCANT", "2-IQANYT");
|
|
||||||
|
|
||||||
// Prober hosts referencing phantom registrars. They and their associated history entries can be
|
|
||||||
// safely ignored.
|
|
||||||
public static final ImmutableSet<String> IGNORED_HOSTS =
|
|
||||||
ImmutableSet.of(
|
|
||||||
"4E21_WJ0TEST-GOOGLE",
|
|
||||||
"4E21_WJ1TEST-GOOGLE",
|
|
||||||
"4E21_WJ2TEST-GOOGLE",
|
|
||||||
"4E21_WJ3TEST-GOOGLE");
|
|
||||||
|
|
||||||
// Prober contacts referencing phantom registrars. They and their associated history entries can
|
|
||||||
// be safely ignored.
|
|
||||||
public static final ImmutableSet<String> IGNORED_CONTACTS =
|
|
||||||
ImmutableSet.of(
|
|
||||||
"1_WJ0TEST-GOOGLE", "1_WJ1TEST-GOOGLE", "1_WJ2TEST-GOOGLE", "1_WJ3TEST-GOOGLE");
|
|
||||||
|
|
||||||
private static boolean isMigratable(Entity entity) {
|
|
||||||
// Checks specific to production data. See b/185954992 for details.
|
|
||||||
// The names of these bad entities in production do not conflict with other environments. For
|
|
||||||
// simplicities sake we apply them regardless of the source of the data.
|
|
||||||
if (entity.getKind().equals("DomainBase")
|
|
||||||
&& IGNORED_DOMAINS.contains(entity.getKey().getName())) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (entity.getKind().equals("ContactResource")) {
|
|
||||||
String roid = entity.getKey().getName();
|
|
||||||
return !IGNORED_CONTACTS.contains(roid);
|
|
||||||
}
|
|
||||||
if (entity.getKind().equals("HostResource")) {
|
|
||||||
String roid = entity.getKey().getName();
|
|
||||||
return !IGNORED_HOSTS.contains(roid);
|
|
||||||
}
|
|
||||||
if (entity.getKind().equals("HistoryEntry")) {
|
|
||||||
// DOMAIN_APPLICATION_CREATE is deprecated type and should not be migrated.
|
|
||||||
// The Enum name DOMAIN_APPLICATION_CREATE no longer exists in Java and cannot
|
|
||||||
// be deserialized.
|
|
||||||
if (Objects.equals(entity.getProperty("type"), "DOMAIN_APPLICATION_CREATE")) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Remove production bad data: Histories of ignored EPP resources:
|
|
||||||
com.google.appengine.api.datastore.Key parentKey = entity.getKey().getParent();
|
|
||||||
if (parentKey.getKind().equals("ContactResource")) {
|
|
||||||
String contactRoid = parentKey.getName();
|
|
||||||
return !IGNORED_CONTACTS.contains(contactRoid);
|
|
||||||
}
|
|
||||||
if (parentKey.getKind().equals("HostResource")) {
|
|
||||||
String hostRoid = parentKey.getName();
|
|
||||||
return !IGNORED_HOSTS.contains(hostRoid);
|
|
||||||
}
|
|
||||||
if (parentKey.getKind().equals("DomainBase")) {
|
|
||||||
String domainRoid = parentKey.getName();
|
|
||||||
return !IGNORED_DOMAINS.contains(domainRoid);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
static Entity repairBadData(Entity entity) {
|
|
||||||
if (entity.getKind().equals("Cancellation")
|
|
||||||
&& Objects.equals(entity.getProperty("reason"), "AUTO_RENEW")) {
|
|
||||||
// AUTO_RENEW has been moved from 'reason' to flags. Change reason to RENEW and add the
|
|
||||||
// AUTO_RENEW flag. Note: all affected entities have empty flags so we can simply assign
|
|
||||||
// instead of append. See b/185954992.
|
|
||||||
entity.setUnindexedProperty("reason", Reason.RENEW.name());
|
|
||||||
entity.setUnindexedProperty("flags", ImmutableList.of(Flag.AUTO_RENEW.name()));
|
|
||||||
} else if (entity.getKind().equals("DomainBase")) {
|
|
||||||
// Canonicalize old domain/host names from 2016 and earlier before we were enforcing this.
|
|
||||||
entity.setIndexedProperty(
|
|
||||||
"fullyQualifiedDomainName",
|
|
||||||
canonicalizeHostname((String) entity.getProperty("fullyQualifiedDomainName")));
|
|
||||||
} else if (entity.getKind().equals("HostResource")) {
|
|
||||||
entity.setIndexedProperty(
|
|
||||||
"fullyQualifiedHostName",
|
|
||||||
canonicalizeHostname((String) entity.getProperty("fullyQualifiedHostName")));
|
|
||||||
}
|
|
||||||
return entity;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static SqlEntity toSqlEntity(Object ofyEntity) {
|
|
||||||
if (ofyEntity instanceof HistoryEntry) {
|
|
||||||
HistoryEntry ofyHistory = (HistoryEntry) ofyEntity;
|
|
||||||
return (SqlEntity) ofyHistory.toChildHistoryEntity();
|
|
||||||
}
|
|
||||||
return ((DatastoreAndSqlEntity) ofyEntity).toSqlEntity().get();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Converts a {@link VersionedEntity} to an JPA entity for persistence.
|
|
||||||
*
|
|
||||||
* @return An object to be persisted to SQL, or null if the input is not to be migrated. (Not
|
|
||||||
* using Optional in return because as a one-use method, we do not want to invest the effort
|
|
||||||
* to make Optional work with BEAM)
|
|
||||||
*/
|
|
||||||
@Nullable
|
|
||||||
public static SqlEntity convertVersionedEntityToSqlEntity(VersionedEntity dsEntity) {
|
|
||||||
return dsEntity
|
|
||||||
.getEntity()
|
|
||||||
.filter(Transforms::isMigratable)
|
|
||||||
.map(Transforms::repairBadData)
|
|
||||||
.map(e -> auditedOfy().toPojo(e))
|
|
||||||
.map(Transforms::toSqlEntity)
|
|
||||||
.orElse(null);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Interface for serializable {@link Supplier suppliers}. */
|
|
||||||
public interface SerializableSupplier<T> extends Supplier<T>, Serializable {}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a {@link PTransform} that produces a {@link PCollection} containing all elements in the
|
|
||||||
* given {@link Iterable}.
|
|
||||||
*/
|
|
||||||
private static PTransform<PBegin, PCollection<String>> toStringPCollection(
|
|
||||||
Iterable<String> strings) {
|
|
||||||
return Create.of(strings).withCoder(StringUtf8Coder.of());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a {@link PTransform} from file {@link Metadata} to {@link VersionedEntity} using
|
|
||||||
* caller-provided {@code transformer}.
|
|
||||||
*/
|
|
||||||
private static PTransform<PCollection<Metadata>, PCollection<VersionedEntity>> processFiles(
|
|
||||||
DoFn<ReadableFile, VersionedEntity> transformer) {
|
|
||||||
return new PTransform<PCollection<Metadata>, PCollection<VersionedEntity>>() {
|
|
||||||
@Override
|
|
||||||
public PCollection<VersionedEntity> expand(PCollection<Metadata> input) {
|
|
||||||
return input
|
|
||||||
.apply(FileIO.readMatches().withCompression(Compression.UNCOMPRESSED))
|
|
||||||
.apply(transformer.getClass().getSimpleName(), ParDo.of(transformer));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class FilterCommitLogFileByTime extends DoFn<Metadata, Metadata> {
|
|
||||||
private final DateTime fromTime;
|
|
||||||
private final DateTime toTime;
|
|
||||||
|
|
||||||
FilterCommitLogFileByTime(DateTime fromTime, DateTime toTime) {
|
|
||||||
checkNotNull(fromTime, "fromTime");
|
|
||||||
checkNotNull(toTime, "toTime");
|
|
||||||
checkArgument(
|
|
||||||
fromTime.isBefore(toTime),
|
|
||||||
"Invalid time range: fromTime (%s) is before endTime (%s)",
|
|
||||||
fromTime,
|
|
||||||
toTime);
|
|
||||||
this.fromTime = fromTime;
|
|
||||||
this.toTime = toTime;
|
|
||||||
}
|
|
||||||
|
|
||||||
@ProcessElement
|
|
||||||
public void processElement(@Element Metadata fileMeta, OutputReceiver<Metadata> out) {
|
|
||||||
DateTime timestamp = getCommitLogTimestamp(fileMeta.resourceId().toString());
|
|
||||||
if (isBeforeOrAt(fromTime, timestamp) && timestamp.isBefore(toTime)) {
|
|
||||||
out.output(fileMeta);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Reads from a Datastore backup file and converts its content into {@link VersionedEntity
|
|
||||||
* VersionedEntities}.
|
|
||||||
*
|
|
||||||
* <p>The input file may be either a LevelDb file from a Datastore export or a CommitLog file
|
|
||||||
* generated by the Nomulus server. In either case, the file contains variable-length records and
|
|
||||||
* must be read sequentially from the beginning. If the read fails, the file needs to be retried
|
|
||||||
* from the beginning.
|
|
||||||
*/
|
|
||||||
private static class BackupFileReader extends DoFn<ReadableFile, VersionedEntity> {
|
|
||||||
private final ProcessFunction<ReadableFile, Iterator<VersionedEntity>> reader;
|
|
||||||
|
|
||||||
private BackupFileReader(ProcessFunction<ReadableFile, Iterator<VersionedEntity>> reader) {
|
|
||||||
this.reader = reader;
|
|
||||||
}
|
|
||||||
|
|
||||||
@ProcessElement
|
|
||||||
public void processElement(@Element ReadableFile file, OutputReceiver<VersionedEntity> out) {
|
|
||||||
try {
|
|
||||||
reader.apply(file).forEachRemaining(out::output);
|
|
||||||
} catch (Exception e) {
|
|
||||||
// Let the pipeline use default retry strategy on the whole file. For GCP Dataflow this
|
|
||||||
// means retrying up to 4 times (may include other files grouped with this one), and failing
|
|
||||||
// the pipeline if no success.
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Removes BillingEvents, {@link google.registry.model.poll.PollMessage PollMessages} and {@link
|
|
||||||
* google.registry.model.host.HostResource} from a {@link DomainBase}. These are circular foreign
|
|
||||||
* key constraints that prevent migration of {@code DomainBase} to SQL databases.
|
|
||||||
*
|
|
||||||
* <p>See {@link InitSqlPipeline} for more information.
|
|
||||||
*/
|
|
||||||
static class RemoveDomainBaseForeignKeys extends DoFn<VersionedEntity, VersionedEntity> {
|
|
||||||
|
|
||||||
@ProcessElement
|
|
||||||
public void processElement(
|
|
||||||
@Element VersionedEntity domainBase, OutputReceiver<VersionedEntity> out) {
|
|
||||||
checkArgument(
|
|
||||||
domainBase.getEntity().isPresent(), "Unexpected delete entity %s", domainBase.key());
|
|
||||||
Entity outputEntity =
|
|
||||||
DomainBaseUtil.removeBillingAndPollAndHosts(domainBase.getEntity().get());
|
|
||||||
out.output(
|
|
||||||
VersionedEntity.from(
|
|
||||||
domainBase.commitTimeMills(),
|
|
||||||
EntityTranslator.convertToPb(outputEntity).toByteArray()));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -41,11 +41,9 @@ import java.util.Optional;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
import org.apache.beam.sdk.Pipeline;
|
import org.apache.beam.sdk.Pipeline;
|
||||||
import org.apache.beam.sdk.PipelineResult;
|
import org.apache.beam.sdk.PipelineResult;
|
||||||
import org.apache.beam.sdk.coders.SerializableCoder;
|
|
||||||
import org.apache.beam.sdk.coders.StringUtf8Coder;
|
import org.apache.beam.sdk.coders.StringUtf8Coder;
|
||||||
import org.apache.beam.sdk.io.FileIO;
|
import org.apache.beam.sdk.io.FileIO;
|
||||||
import org.apache.beam.sdk.io.TextIO;
|
import org.apache.beam.sdk.io.TextIO;
|
||||||
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
|
|
||||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||||
import org.apache.beam.sdk.transforms.Contextful;
|
import org.apache.beam.sdk.transforms.Contextful;
|
||||||
import org.apache.beam.sdk.transforms.Count;
|
import org.apache.beam.sdk.transforms.Count;
|
||||||
|
@ -92,28 +90,11 @@ public class InvoicingPipeline implements Serializable {
|
||||||
|
|
||||||
void setupPipeline(Pipeline pipeline) {
|
void setupPipeline(Pipeline pipeline) {
|
||||||
options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_READ_COMMITTED);
|
options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_READ_COMMITTED);
|
||||||
PCollection<BillingEvent> billingEvents =
|
PCollection<BillingEvent> billingEvents = readFromCloudSql(options, pipeline);
|
||||||
options.getDatabase().equals("DATASTORE")
|
|
||||||
? readFromBigQuery(options, pipeline)
|
|
||||||
: readFromCloudSql(options, pipeline);
|
|
||||||
|
|
||||||
saveInvoiceCsv(billingEvents, options);
|
saveInvoiceCsv(billingEvents, options);
|
||||||
|
|
||||||
saveDetailedCsv(billingEvents, options);
|
saveDetailedCsv(billingEvents, options);
|
||||||
}
|
}
|
||||||
|
|
||||||
static PCollection<BillingEvent> readFromBigQuery(
|
|
||||||
InvoicingPipelineOptions options, Pipeline pipeline) {
|
|
||||||
return pipeline.apply(
|
|
||||||
"Read BillingEvents from Bigquery",
|
|
||||||
BigQueryIO.read(BillingEvent::parseFromRecord)
|
|
||||||
.fromQuery(makeQuery(options.getYearMonth(), options.getProject()))
|
|
||||||
.withCoder(SerializableCoder.of(BillingEvent.class))
|
|
||||||
.usingStandardSql()
|
|
||||||
.withoutValidation()
|
|
||||||
.withTemplateCompatibility());
|
|
||||||
}
|
|
||||||
|
|
||||||
static PCollection<BillingEvent> readFromCloudSql(
|
static PCollection<BillingEvent> readFromCloudSql(
|
||||||
InvoicingPipelineOptions options, Pipeline pipeline) {
|
InvoicingPipelineOptions options, Pipeline pipeline) {
|
||||||
Read<Object[], BillingEvent> read =
|
Read<Object[], BillingEvent> read =
|
||||||
|
|
|
@ -30,11 +30,6 @@ public interface InvoicingPipelineOptions extends RegistryPipelineOptions {
|
||||||
|
|
||||||
void setInvoiceFilePrefix(String value);
|
void setInvoiceFilePrefix(String value);
|
||||||
|
|
||||||
@Description("The database to read data from.")
|
|
||||||
String getDatabase();
|
|
||||||
|
|
||||||
void setDatabase(String value);
|
|
||||||
|
|
||||||
@Description("The GCS bucket URL for invoices and detailed reports to be uploaded.")
|
@Description("The GCS bucket URL for invoices and detailed reports to be uploaded.")
|
||||||
String getBillingBucketUrl();
|
String getBillingBucketUrl();
|
||||||
|
|
||||||
|
|
|
@ -14,30 +14,19 @@
|
||||||
|
|
||||||
package google.registry.beam.spec11;
|
package google.registry.beam.spec11;
|
||||||
|
|
||||||
import static google.registry.beam.BeamUtils.checkFieldsNotNull;
|
|
||||||
import static google.registry.beam.BeamUtils.extractField;
|
|
||||||
|
|
||||||
import com.google.auto.value.AutoValue;
|
import com.google.auto.value.AutoValue;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.collect.ImmutableList;
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
|
||||||
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A POJO representing a domain name and associated info, parsed from a {@code SchemaAndRecord}.
|
* A POJO representing a domain name and associated info, parsed from a {@code SchemaAndRecord}.
|
||||||
*
|
*
|
||||||
* <p>This is a trivially serializable class that allows Beam to transform the results of a Bigquery
|
* <p>This is a trivially serializable class that allows Beam to transform the results of a SQL
|
||||||
* query into a standard Java representation, giving us the type guarantees and ease of manipulation
|
* query into a standard Java representation.
|
||||||
* Bigquery lacks, while localizing any Bigquery-side failures to the {@link #parseFromRecord}
|
|
||||||
* function.
|
|
||||||
*/
|
*/
|
||||||
@AutoValue
|
@AutoValue
|
||||||
public abstract class DomainNameInfo implements Serializable {
|
public abstract class DomainNameInfo implements Serializable {
|
||||||
|
|
||||||
private static final ImmutableList<String> FIELD_NAMES =
|
|
||||||
ImmutableList.of("domainName", "domainRepoId", "registrarId", "registrarEmailAddress");
|
|
||||||
|
|
||||||
/** Returns the fully qualified domain name. */
|
/** Returns the fully qualified domain name. */
|
||||||
abstract String domainName();
|
abstract String domainName();
|
||||||
|
|
||||||
|
@ -50,28 +39,8 @@ public abstract class DomainNameInfo implements Serializable {
|
||||||
/** Returns the email address of the registrar associated with this domain. */
|
/** Returns the email address of the registrar associated with this domain. */
|
||||||
abstract String registrarEmailAddress();
|
abstract String registrarEmailAddress();
|
||||||
|
|
||||||
/**
|
|
||||||
* Constructs a {@link DomainNameInfo} from an Apache Avro {@code SchemaAndRecord}.
|
|
||||||
*
|
|
||||||
* @see <a
|
|
||||||
* href=http://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/generic/GenericData.Record.html>
|
|
||||||
* Apache AVRO GenericRecord</a>
|
|
||||||
*/
|
|
||||||
static DomainNameInfo parseFromRecord(SchemaAndRecord schemaAndRecord) {
|
|
||||||
checkFieldsNotNull(FIELD_NAMES, schemaAndRecord);
|
|
||||||
GenericRecord record = schemaAndRecord.getRecord();
|
|
||||||
return create(
|
|
||||||
extractField(record, "domainName"),
|
|
||||||
extractField(record, "domainRepoId"),
|
|
||||||
extractField(record, "registrarId"),
|
|
||||||
extractField(record, "registrarEmailAddress"));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a concrete {@link DomainNameInfo}.
|
* Creates a concrete {@link DomainNameInfo}.
|
||||||
*
|
|
||||||
* <p>This should only be used outside this class for testing- instances of {@link DomainNameInfo}
|
|
||||||
* should otherwise come from {@link #parseFromRecord}.
|
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
static DomainNameInfo create(
|
static DomainNameInfo create(
|
||||||
|
|
|
@ -15,7 +15,6 @@
|
||||||
package google.registry.beam.spec11;
|
package google.registry.beam.spec11;
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkArgument;
|
import static com.google.common.base.Preconditions.checkArgument;
|
||||||
import static google.registry.beam.BeamUtils.getQueryFromFile;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||||
|
|
||||||
import com.google.auto.value.AutoValue;
|
import com.google.auto.value.AutoValue;
|
||||||
|
@ -33,15 +32,12 @@ import google.registry.model.reporting.Spec11ThreatMatch.ThreatType;
|
||||||
import google.registry.persistence.PersistenceModule.TransactionIsolationLevel;
|
import google.registry.persistence.PersistenceModule.TransactionIsolationLevel;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.util.Retrier;
|
import google.registry.util.Retrier;
|
||||||
import google.registry.util.SqlTemplate;
|
|
||||||
import google.registry.util.UtilsModule;
|
import google.registry.util.UtilsModule;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import javax.inject.Singleton;
|
import javax.inject.Singleton;
|
||||||
import org.apache.beam.sdk.Pipeline;
|
import org.apache.beam.sdk.Pipeline;
|
||||||
import org.apache.beam.sdk.PipelineResult;
|
import org.apache.beam.sdk.PipelineResult;
|
||||||
import org.apache.beam.sdk.coders.SerializableCoder;
|
|
||||||
import org.apache.beam.sdk.io.TextIO;
|
import org.apache.beam.sdk.io.TextIO;
|
||||||
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
|
|
||||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||||
import org.apache.beam.sdk.transforms.DoFn;
|
import org.apache.beam.sdk.transforms.DoFn;
|
||||||
import org.apache.beam.sdk.transforms.GroupByKey;
|
import org.apache.beam.sdk.transforms.GroupByKey;
|
||||||
|
@ -104,10 +100,7 @@ public class Spec11Pipeline implements Serializable {
|
||||||
|
|
||||||
void setupPipeline(Pipeline pipeline) {
|
void setupPipeline(Pipeline pipeline) {
|
||||||
options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_READ_COMMITTED);
|
options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_READ_COMMITTED);
|
||||||
PCollection<DomainNameInfo> domains =
|
PCollection<DomainNameInfo> domains = readFromCloudSql(pipeline);
|
||||||
options.getDatabase().equals("DATASTORE")
|
|
||||||
? readFromBigQuery(options, pipeline)
|
|
||||||
: readFromCloudSql(pipeline);
|
|
||||||
|
|
||||||
PCollection<KV<DomainNameInfo, ThreatMatch>> threatMatches =
|
PCollection<KV<DomainNameInfo, ThreatMatch>> threatMatches =
|
||||||
domains.apply("Run through SafeBrowsing API", ParDo.of(safeBrowsingFn));
|
domains.apply("Run through SafeBrowsing API", ParDo.of(safeBrowsingFn));
|
||||||
|
@ -156,24 +149,6 @@ public class Spec11Pipeline implements Serializable {
|
||||||
}));
|
}));
|
||||||
}
|
}
|
||||||
|
|
||||||
static PCollection<DomainNameInfo> readFromBigQuery(
|
|
||||||
Spec11PipelineOptions options, Pipeline pipeline) {
|
|
||||||
return pipeline.apply(
|
|
||||||
"Read active domains from BigQuery",
|
|
||||||
BigQueryIO.read(DomainNameInfo::parseFromRecord)
|
|
||||||
.fromQuery(
|
|
||||||
SqlTemplate.create(getQueryFromFile(Spec11Pipeline.class, "domain_name_infos.sql"))
|
|
||||||
.put("PROJECT_ID", options.getProject())
|
|
||||||
.put("DATASTORE_EXPORT_DATASET", "latest_datastore_export")
|
|
||||||
.put("REGISTRAR_TABLE", "Registrar")
|
|
||||||
.put("DOMAIN_BASE_TABLE", "DomainBase")
|
|
||||||
.build())
|
|
||||||
.withCoder(SerializableCoder.of(DomainNameInfo.class))
|
|
||||||
.usingStandardSql()
|
|
||||||
.withoutValidation()
|
|
||||||
.withTemplateCompatibility());
|
|
||||||
}
|
|
||||||
|
|
||||||
private static KV<String, String> parseRow(Object[] row) {
|
private static KV<String, String> parseRow(Object[] row) {
|
||||||
return KV.of((String) row[0], (String) row[1]);
|
return KV.of((String) row[0], (String) row[1]);
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,9 +34,4 @@ public interface Spec11PipelineOptions extends RegistryPipelineOptions {
|
||||||
String getReportingBucketUrl();
|
String getReportingBucketUrl();
|
||||||
|
|
||||||
void setReportingBucketUrl(String value);
|
void setReportingBucketUrl(String value);
|
||||||
|
|
||||||
@Description("The database to read data from.")
|
|
||||||
String getDatabase();
|
|
||||||
|
|
||||||
void setDatabase(String value);
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,6 @@ import google.registry.model.eppinput.EppInput.InnerCommand;
|
||||||
import google.registry.model.eppinput.EppInput.ResourceCommandWrapper;
|
import google.registry.model.eppinput.EppInput.ResourceCommandWrapper;
|
||||||
import google.registry.model.eppoutput.Result;
|
import google.registry.model.eppoutput.Result;
|
||||||
import google.registry.model.eppoutput.Result.Code;
|
import google.registry.model.eppoutput.Result.Code;
|
||||||
import google.registry.persistence.transaction.TransactionManagerFactory.ReadOnlyModeException;
|
|
||||||
import java.lang.annotation.Documented;
|
import java.lang.annotation.Documented;
|
||||||
import java.lang.annotation.Inherited;
|
import java.lang.annotation.Inherited;
|
||||||
import java.lang.annotation.Retention;
|
import java.lang.annotation.Retention;
|
||||||
|
@ -262,12 +261,4 @@ public abstract class EppException extends Exception {
|
||||||
super("Specified protocol version is not implemented");
|
super("Specified protocol version is not implemented");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Registry is currently undergoing maintenance and is in read-only mode. */
|
|
||||||
@EppResultCode(Code.COMMAND_FAILED)
|
|
||||||
public static class ReadOnlyModeEppException extends EppException {
|
|
||||||
ReadOnlyModeEppException(ReadOnlyModeException cause) {
|
|
||||||
super("Registry is currently undergoing maintenance and is in read-only mode", cause);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,6 @@ import static google.registry.xml.XmlTransformer.prettyPrint;
|
||||||
|
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.flogger.FluentLogger;
|
import com.google.common.flogger.FluentLogger;
|
||||||
import google.registry.flows.EppException.ReadOnlyModeEppException;
|
|
||||||
import google.registry.flows.FlowModule.DryRun;
|
import google.registry.flows.FlowModule.DryRun;
|
||||||
import google.registry.flows.FlowModule.InputXml;
|
import google.registry.flows.FlowModule.InputXml;
|
||||||
import google.registry.flows.FlowModule.RegistrarId;
|
import google.registry.flows.FlowModule.RegistrarId;
|
||||||
|
@ -29,7 +28,6 @@ import google.registry.flows.session.LoginFlow;
|
||||||
import google.registry.model.eppcommon.Trid;
|
import google.registry.model.eppcommon.Trid;
|
||||||
import google.registry.model.eppoutput.EppOutput;
|
import google.registry.model.eppoutput.EppOutput;
|
||||||
import google.registry.monitoring.whitebox.EppMetric;
|
import google.registry.monitoring.whitebox.EppMetric;
|
||||||
import google.registry.persistence.transaction.TransactionManagerFactory.ReadOnlyModeException;
|
|
||||||
import javax.inject.Inject;
|
import javax.inject.Inject;
|
||||||
import javax.inject.Provider;
|
import javax.inject.Provider;
|
||||||
|
|
||||||
|
@ -99,8 +97,6 @@ public class FlowRunner {
|
||||||
return e.output;
|
return e.output;
|
||||||
} catch (EppRuntimeException e) {
|
} catch (EppRuntimeException e) {
|
||||||
throw e.getCause();
|
throw e.getCause();
|
||||||
} catch (ReadOnlyModeException e) {
|
|
||||||
throw new ReadOnlyModeEppException(e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -50,7 +50,6 @@ import org.joda.time.DateTime;
|
||||||
/**
|
/**
|
||||||
* An EPP flow that creates a new contact.
|
* An EPP flow that creates a new contact.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link ResourceAlreadyExistsForThisClientException}
|
* @error {@link ResourceAlreadyExistsForThisClientException}
|
||||||
* @error {@link ResourceCreateContentionException}
|
* @error {@link ResourceCreateContentionException}
|
||||||
|
|
|
@ -24,7 +24,6 @@ import static google.registry.model.ResourceTransferUtils.denyPendingTransfer;
|
||||||
import static google.registry.model.ResourceTransferUtils.handlePendingTransferOnDelete;
|
import static google.registry.model.ResourceTransferUtils.handlePendingTransferOnDelete;
|
||||||
import static google.registry.model.eppoutput.Result.Code.SUCCESS;
|
import static google.registry.model.eppoutput.Result.Code.SUCCESS;
|
||||||
import static google.registry.model.transfer.TransferStatus.SERVER_CANCELLED;
|
import static google.registry.model.transfer.TransferStatus.SERVER_CANCELLED;
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.assertAsyncActionsAreAllowed;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
@ -58,7 +57,6 @@ import org.joda.time.DateTime;
|
||||||
* references to the host before the deletion is allowed to proceed. A poll message will be written
|
* references to the host before the deletion is allowed to proceed. A poll message will be written
|
||||||
* with the success or failure message when the process is complete.
|
* with the success or failure message when the process is complete.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceNotOwnedException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceNotOwnedException}
|
||||||
|
@ -92,7 +90,6 @@ public final class ContactDeleteFlow implements TransactionalFlow {
|
||||||
extensionManager.register(MetadataExtension.class);
|
extensionManager.register(MetadataExtension.class);
|
||||||
validateRegistrarIsLoggedIn(registrarId);
|
validateRegistrarIsLoggedIn(registrarId);
|
||||||
extensionManager.validate();
|
extensionManager.validate();
|
||||||
assertAsyncActionsAreAllowed();
|
|
||||||
DateTime now = tm().getTransactionTime();
|
DateTime now = tm().getTransactionTime();
|
||||||
checkLinkedDomains(targetId, now, ContactResource.class);
|
checkLinkedDomains(targetId, now, ContactResource.class);
|
||||||
ContactResource existingContact = loadAndVerifyExistence(ContactResource.class, targetId, now);
|
ContactResource existingContact = loadAndVerifyExistence(ContactResource.class, targetId, now);
|
||||||
|
|
|
@ -54,7 +54,6 @@ import org.joda.time.DateTime;
|
||||||
* transfer is automatically approved. Within that window, this flow allows the losing client to
|
* transfer is automatically approved. Within that window, this flow allows the losing client to
|
||||||
* explicitly approve the transfer request, which then becomes effective immediately.
|
* explicitly approve the transfer request, which then becomes effective immediately.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceNotOwnedException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceNotOwnedException}
|
||||||
|
|
|
@ -54,7 +54,6 @@ import org.joda.time.DateTime;
|
||||||
* transfer is automatically approved. Within that window, this flow allows the gaining client to
|
* transfer is automatically approved. Within that window, this flow allows the gaining client to
|
||||||
* withdraw the transfer request.
|
* withdraw the transfer request.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -53,7 +53,6 @@ import org.joda.time.DateTime;
|
||||||
* transfer is automatically approved. Within that window, this flow allows the losing client to
|
* transfer is automatically approved. Within that window, this flow allows the losing client to
|
||||||
* reject the transfer request.
|
* reject the transfer request.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -63,7 +63,6 @@ import org.joda.time.Duration;
|
||||||
* by the losing registrar or rejected, and the gaining registrar can also cancel the transfer
|
* by the losing registrar or rejected, and the gaining registrar can also cancel the transfer
|
||||||
* request.
|
* request.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -55,7 +55,6 @@ import org.joda.time.DateTime;
|
||||||
/**
|
/**
|
||||||
* An EPP flow that updates a contact.
|
* An EPP flow that updates a contact.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.AddRemoveSameValueException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.AddRemoveSameValueException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -139,7 +139,6 @@ import org.joda.time.Duration;
|
||||||
* google.registry.flows.domain.token.AllocationTokenFlowUtils.AlreadyRedeemedAllocationTokenException}
|
* google.registry.flows.domain.token.AllocationTokenFlowUtils.AlreadyRedeemedAllocationTokenException}
|
||||||
* @error {@link
|
* @error {@link
|
||||||
* google.registry.flows.domain.token.AllocationTokenFlowUtils.InvalidAllocationTokenException}
|
* google.registry.flows.domain.token.AllocationTokenFlowUtils.InvalidAllocationTokenException}
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.exceptions.OnlyToolCanPassMetadataException}
|
* @error {@link google.registry.flows.exceptions.OnlyToolCanPassMetadataException}
|
||||||
* @error {@link ResourceAlreadyExistsForThisClientException}
|
* @error {@link ResourceAlreadyExistsForThisClientException}
|
||||||
* @error {@link ResourceCreateContentionException}
|
* @error {@link ResourceCreateContentionException}
|
||||||
|
|
|
@ -103,7 +103,6 @@ import org.joda.time.Duration;
|
||||||
/**
|
/**
|
||||||
* An EPP flow that deletes a domain.
|
* An EPP flow that deletes a domain.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.EppException.UnimplementedExtensionException}
|
* @error {@link google.registry.flows.EppException.UnimplementedExtensionException}
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -95,7 +95,6 @@ import org.joda.time.Duration;
|
||||||
* longer than 10 years unless it comes in at the exact millisecond that the domain would have
|
* longer than 10 years unless it comes in at the exact millisecond that the domain would have
|
||||||
* expired.
|
* expired.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.FlowUtils.UnknownCurrencyEppException}
|
* @error {@link google.registry.flows.FlowUtils.UnknownCurrencyEppException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -94,7 +94,6 @@ import org.joda.time.DateTime;
|
||||||
* restored to a single year expiration starting at the restore time, regardless of what the
|
* restored to a single year expiration starting at the restore time, regardless of what the
|
||||||
* original expiration time was.
|
* original expiration time was.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.EppException.UnimplementedExtensionException}
|
* @error {@link google.registry.flows.EppException.UnimplementedExtensionException}
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.FlowUtils.UnknownCurrencyEppException}
|
* @error {@link google.registry.flows.FlowUtils.UnknownCurrencyEppException}
|
||||||
|
|
|
@ -78,7 +78,6 @@ import org.joda.time.DateTime;
|
||||||
* timestamps such that they only would become active when the transfer period passed. In this flow,
|
* timestamps such that they only would become active when the transfer period passed. In this flow,
|
||||||
* those speculative objects are deleted and replaced with new ones with the correct approval time.
|
* those speculative objects are deleted and replaced with new ones with the correct approval time.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -65,7 +65,6 @@ import org.joda.time.DateTime;
|
||||||
* timestamps such that they only would become active when the transfer period passed. In this flow,
|
* timestamps such that they only would become active when the transfer period passed. In this flow,
|
||||||
* those speculative objects are deleted.
|
* those speculative objects are deleted.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -67,7 +67,6 @@ import org.joda.time.DateTime;
|
||||||
* timestamps such that they only would become active when the transfer period passed. In this flow,
|
* timestamps such that they only would become active when the transfer period passed. In this flow,
|
||||||
* those speculative objects are deleted.
|
* those speculative objects are deleted.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
|
|
@ -94,7 +94,6 @@ import org.joda.time.DateTime;
|
||||||
* rejection or cancellation of the request, they will be deleted (and in the approval case,
|
* rejection or cancellation of the request, they will be deleted (and in the approval case,
|
||||||
* replaced with new ones with the correct approval time).
|
* replaced with new ones with the correct approval time).
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.FlowUtils.UnknownCurrencyEppException}
|
* @error {@link google.registry.flows.FlowUtils.UnknownCurrencyEppException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.BadAuthInfoForResourceException}
|
||||||
|
|
|
@ -101,7 +101,6 @@ import org.joda.time.DateTime;
|
||||||
* superuser. As such, adding or removing these statuses incurs a billing event. There will be only
|
* superuser. As such, adding or removing these statuses incurs a billing event. There will be only
|
||||||
* one charge per update, even if several such statuses are updated at once.
|
* one charge per update, even if several such statuses are updated at once.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.EppException.UnimplementedExtensionException}
|
* @error {@link google.registry.flows.EppException.UnimplementedExtensionException}
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.AddRemoveSameValueException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.AddRemoveSameValueException}
|
||||||
|
|
|
@ -65,7 +65,6 @@ import org.joda.time.DateTime;
|
||||||
* hosts cannot have any. This flow allows creating a host name, and if necessary enqueues tasks to
|
* hosts cannot have any. This flow allows creating a host name, and if necessary enqueues tasks to
|
||||||
* update DNS.
|
* update DNS.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.IpAddressVersionMismatchException}
|
* @error {@link google.registry.flows.FlowUtils.IpAddressVersionMismatchException}
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link ResourceAlreadyExistsForThisClientException}
|
* @error {@link ResourceAlreadyExistsForThisClientException}
|
||||||
|
|
|
@ -21,7 +21,6 @@ import static google.registry.flows.ResourceFlowUtils.verifyNoDisallowedStatuses
|
||||||
import static google.registry.flows.ResourceFlowUtils.verifyResourceOwnership;
|
import static google.registry.flows.ResourceFlowUtils.verifyResourceOwnership;
|
||||||
import static google.registry.flows.host.HostFlowUtils.validateHostName;
|
import static google.registry.flows.host.HostFlowUtils.validateHostName;
|
||||||
import static google.registry.model.eppoutput.Result.Code.SUCCESS;
|
import static google.registry.model.eppoutput.Result.Code.SUCCESS;
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.assertAsyncActionsAreAllowed;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
@ -55,7 +54,6 @@ import org.joda.time.DateTime;
|
||||||
* references to the host before the deletion is allowed to proceed. A poll message will be written
|
* references to the host before the deletion is allowed to proceed. A poll message will be written
|
||||||
* with the success or failure message when the process is complete.
|
* with the success or failure message when the process is complete.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceNotOwnedException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceNotOwnedException}
|
||||||
|
@ -93,7 +91,6 @@ public final class HostDeleteFlow implements TransactionalFlow {
|
||||||
extensionManager.register(MetadataExtension.class);
|
extensionManager.register(MetadataExtension.class);
|
||||||
validateRegistrarIsLoggedIn(registrarId);
|
validateRegistrarIsLoggedIn(registrarId);
|
||||||
extensionManager.validate();
|
extensionManager.validate();
|
||||||
assertAsyncActionsAreAllowed();
|
|
||||||
DateTime now = tm().getTransactionTime();
|
DateTime now = tm().getTransactionTime();
|
||||||
validateHostName(targetId);
|
validateHostName(targetId);
|
||||||
checkLinkedDomains(targetId, now, HostResource.class);
|
checkLinkedDomains(targetId, now, HostResource.class);
|
||||||
|
|
|
@ -28,7 +28,6 @@ import static google.registry.flows.host.HostFlowUtils.verifySuperordinateDomain
|
||||||
import static google.registry.flows.host.HostFlowUtils.verifySuperordinateDomainOwnership;
|
import static google.registry.flows.host.HostFlowUtils.verifySuperordinateDomainOwnership;
|
||||||
import static google.registry.model.index.ForeignKeyIndex.loadAndGetKey;
|
import static google.registry.model.index.ForeignKeyIndex.loadAndGetKey;
|
||||||
import static google.registry.model.reporting.HistoryEntry.Type.HOST_UPDATE;
|
import static google.registry.model.reporting.HistoryEntry.Type.HOST_UPDATE;
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.assertAsyncActionsAreAllowed;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||||
import static google.registry.util.CollectionUtils.isNullOrEmpty;
|
import static google.registry.util.CollectionUtils.isNullOrEmpty;
|
||||||
|
|
||||||
|
@ -79,7 +78,6 @@ import org.joda.time.DateTime;
|
||||||
* when it is renamed from external to internal at least one must be added. If the host is renamed
|
* when it is renamed from external to internal at least one must be added. If the host is renamed
|
||||||
* or IP addresses are added, tasks are enqueued to update DNS accordingly.
|
* or IP addresses are added, tasks are enqueued to update DNS accordingly.
|
||||||
*
|
*
|
||||||
* @error {@link google.registry.flows.EppException.ReadOnlyModeEppException}
|
|
||||||
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
* @error {@link google.registry.flows.FlowUtils.NotLoggedInException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.AddRemoveSameValueException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.AddRemoveSameValueException}
|
||||||
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
* @error {@link google.registry.flows.ResourceFlowUtils.ResourceDoesNotExistException}
|
||||||
|
@ -137,9 +135,6 @@ public final class HostUpdateFlow implements TransactionalFlow {
|
||||||
validateHostName(targetId);
|
validateHostName(targetId);
|
||||||
HostResource existingHost = loadAndVerifyExistence(HostResource.class, targetId, now);
|
HostResource existingHost = loadAndVerifyExistence(HostResource.class, targetId, now);
|
||||||
boolean isHostRename = suppliedNewHostName != null;
|
boolean isHostRename = suppliedNewHostName != null;
|
||||||
if (isHostRename) {
|
|
||||||
assertAsyncActionsAreAllowed();
|
|
||||||
}
|
|
||||||
String oldHostName = targetId;
|
String oldHostName = targetId;
|
||||||
String newHostName = firstNonNull(suppliedNewHostName, oldHostName);
|
String newHostName = firstNonNull(suppliedNewHostName, oldHostName);
|
||||||
DomainBase oldSuperordinateDomain =
|
DomainBase oldSuperordinateDomain =
|
||||||
|
|
|
@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableSet;
|
||||||
import com.googlecode.objectify.Key;
|
import com.googlecode.objectify.Key;
|
||||||
import google.registry.model.EppResource;
|
import google.registry.model.EppResource;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.index.EppResourceIndex;
|
import google.registry.model.index.EppResourceIndex;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -30,6 +31,7 @@ import google.registry.model.index.EppResourceIndex;
|
||||||
* <p>The inputs provided by this class are not deletion-aware and do not project the resources
|
* <p>The inputs provided by this class are not deletion-aware and do not project the resources
|
||||||
* forward in time. That is the responsibility of mappers that use these inputs.
|
* forward in time. That is the responsibility of mappers that use these inputs.
|
||||||
*/
|
*/
|
||||||
|
@DeleteAfterMigration
|
||||||
public final class EppResourceInputs {
|
public final class EppResourceInputs {
|
||||||
|
|
||||||
private EppResourceInputs() {}
|
private EppResourceInputs() {}
|
||||||
|
|
|
@ -46,8 +46,6 @@ import google.registry.model.domain.DomainHistory;
|
||||||
import google.registry.model.domain.GracePeriod;
|
import google.registry.model.domain.GracePeriod;
|
||||||
import google.registry.model.domain.rgp.GracePeriodStatus;
|
import google.registry.model.domain.rgp.GracePeriodStatus;
|
||||||
import google.registry.model.domain.token.AllocationToken;
|
import google.registry.model.domain.token.AllocationToken;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
import google.registry.model.transfer.TransferData.TransferServerApproveEntity;
|
import google.registry.model.transfer.TransferData.TransferServerApproveEntity;
|
||||||
import google.registry.persistence.BillingVKey.BillingEventVKey;
|
import google.registry.persistence.BillingVKey.BillingEventVKey;
|
||||||
import google.registry.persistence.BillingVKey.BillingRecurrenceVKey;
|
import google.registry.persistence.BillingVKey.BillingRecurrenceVKey;
|
||||||
|
@ -347,7 +345,7 @@ public abstract class BillingEvent extends ImmutableObject
|
||||||
})
|
})
|
||||||
@AttributeOverride(name = "id", column = @Column(name = "billing_event_id"))
|
@AttributeOverride(name = "id", column = @Column(name = "billing_event_id"))
|
||||||
@WithLongVKey(compositeKey = true)
|
@WithLongVKey(compositeKey = true)
|
||||||
public static class OneTime extends BillingEvent implements DatastoreAndSqlEntity {
|
public static class OneTime extends BillingEvent {
|
||||||
|
|
||||||
/** The billable value. */
|
/** The billable value. */
|
||||||
@Type(type = JodaMoneyType.TYPE_NAME)
|
@Type(type = JodaMoneyType.TYPE_NAME)
|
||||||
|
@ -559,7 +557,7 @@ public abstract class BillingEvent extends ImmutableObject
|
||||||
})
|
})
|
||||||
@AttributeOverride(name = "id", column = @Column(name = "billing_recurrence_id"))
|
@AttributeOverride(name = "id", column = @Column(name = "billing_recurrence_id"))
|
||||||
@WithLongVKey(compositeKey = true)
|
@WithLongVKey(compositeKey = true)
|
||||||
public static class Recurring extends BillingEvent implements DatastoreAndSqlEntity {
|
public static class Recurring extends BillingEvent {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The billing event recurs every year between {@link #eventTime} and this time on the
|
* The billing event recurs every year between {@link #eventTime} and this time on the
|
||||||
|
@ -696,7 +694,7 @@ public abstract class BillingEvent extends ImmutableObject
|
||||||
})
|
})
|
||||||
@AttributeOverride(name = "id", column = @Column(name = "billing_cancellation_id"))
|
@AttributeOverride(name = "id", column = @Column(name = "billing_cancellation_id"))
|
||||||
@WithLongVKey(compositeKey = true)
|
@WithLongVKey(compositeKey = true)
|
||||||
public static class Cancellation extends BillingEvent implements DatastoreAndSqlEntity {
|
public static class Cancellation extends BillingEvent {
|
||||||
|
|
||||||
/** The billing time of the charge that is being cancelled. */
|
/** The billing time of the charge that is being cancelled. */
|
||||||
@Index
|
@Index
|
||||||
|
@ -819,7 +817,7 @@ public abstract class BillingEvent extends ImmutableObject
|
||||||
@ReportedOn
|
@ReportedOn
|
||||||
@Entity
|
@Entity
|
||||||
@WithLongVKey(compositeKey = true)
|
@WithLongVKey(compositeKey = true)
|
||||||
public static class Modification extends BillingEvent implements DatastoreOnlyEntity {
|
public static class Modification extends BillingEvent {
|
||||||
|
|
||||||
/** The change in cost that should be applied to the original billing event. */
|
/** The change in cost that should be applied to the original billing event. */
|
||||||
Money cost;
|
Money cost;
|
||||||
|
|
|
@ -16,7 +16,6 @@ package google.registry.model.bulkquery;
|
||||||
|
|
||||||
import google.registry.model.domain.DomainBase;
|
import google.registry.model.domain.DomainBase;
|
||||||
import google.registry.model.domain.DomainContent;
|
import google.registry.model.domain.DomainContent;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.WithStringVKey;
|
import google.registry.persistence.WithStringVKey;
|
||||||
import javax.persistence.Access;
|
import javax.persistence.Access;
|
||||||
|
@ -34,7 +33,7 @@ import javax.persistence.Entity;
|
||||||
@Entity(name = "Domain")
|
@Entity(name = "Domain")
|
||||||
@WithStringVKey
|
@WithStringVKey
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
public class DomainBaseLite extends DomainContent implements SqlOnlyEntity {
|
public class DomainBaseLite extends DomainContent {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@javax.persistence.Id
|
@javax.persistence.Id
|
||||||
|
|
|
@ -17,7 +17,6 @@ package google.registry.model.bulkquery;
|
||||||
import com.google.common.base.Objects;
|
import com.google.common.base.Objects;
|
||||||
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
||||||
import google.registry.model.host.HostResource;
|
import google.registry.model.host.HostResource;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import javax.persistence.Access;
|
import javax.persistence.Access;
|
||||||
|
@ -33,7 +32,7 @@ import javax.persistence.IdClass;
|
||||||
@Entity
|
@Entity
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
@IdClass(DomainHistoryHost.class)
|
@IdClass(DomainHistoryHost.class)
|
||||||
public class DomainHistoryHost implements Serializable, SqlOnlyEntity {
|
public class DomainHistoryHost implements Serializable {
|
||||||
|
|
||||||
@Id private Long domainHistoryHistoryRevisionId;
|
@Id private Long domainHistoryHistoryRevisionId;
|
||||||
@Id private String domainHistoryDomainRepoId;
|
@Id private String domainHistoryDomainRepoId;
|
||||||
|
|
|
@ -20,7 +20,6 @@ import google.registry.model.domain.DomainContent;
|
||||||
import google.registry.model.domain.DomainHistory;
|
import google.registry.model.domain.DomainHistory;
|
||||||
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
||||||
import google.registry.model.domain.Period;
|
import google.registry.model.domain.Period;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -48,7 +47,7 @@ import javax.persistence.PostLoad;
|
||||||
@Entity(name = "DomainHistory")
|
@Entity(name = "DomainHistory")
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
@IdClass(DomainHistoryId.class)
|
@IdClass(DomainHistoryId.class)
|
||||||
public class DomainHistoryLite extends HistoryEntry implements SqlOnlyEntity {
|
public class DomainHistoryLite extends HistoryEntry {
|
||||||
|
|
||||||
// Store DomainContent instead of DomainBase so we don't pick up its @Id
|
// Store DomainContent instead of DomainBase so we don't pick up its @Id
|
||||||
// Nullable for the sake of pre-Registry-3.0 history objects
|
// Nullable for the sake of pre-Registry-3.0 history objects
|
||||||
|
|
|
@ -16,7 +16,6 @@ package google.registry.model.bulkquery;
|
||||||
|
|
||||||
import com.google.common.base.Objects;
|
import com.google.common.base.Objects;
|
||||||
import google.registry.model.host.HostResource;
|
import google.registry.model.host.HostResource;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import javax.persistence.Access;
|
import javax.persistence.Access;
|
||||||
|
@ -29,7 +28,7 @@ import javax.persistence.IdClass;
|
||||||
@Entity
|
@Entity
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
@IdClass(DomainHost.class)
|
@IdClass(DomainHost.class)
|
||||||
public class DomainHost implements Serializable, SqlOnlyEntity {
|
public class DomainHost implements Serializable {
|
||||||
|
|
||||||
@Id private String domainRepoId;
|
@Id private String domainRepoId;
|
||||||
|
|
||||||
|
|
|
@ -31,7 +31,6 @@ import google.registry.model.UnsafeSerializable;
|
||||||
import google.registry.model.UpdateAutoTimestamp;
|
import google.registry.model.UpdateAutoTimestamp;
|
||||||
import google.registry.model.annotations.InCrossTld;
|
import google.registry.model.annotations.InCrossTld;
|
||||||
import google.registry.model.common.Cursor.CursorId;
|
import google.registry.model.common.Cursor.CursorId;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.model.tld.Registry;
|
import google.registry.model.tld.Registry;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -53,7 +52,7 @@ import org.joda.time.DateTime;
|
||||||
@javax.persistence.Entity
|
@javax.persistence.Entity
|
||||||
@IdClass(CursorId.class)
|
@IdClass(CursorId.class)
|
||||||
@InCrossTld
|
@InCrossTld
|
||||||
public class Cursor extends ImmutableObject implements DatastoreAndSqlEntity, UnsafeSerializable {
|
public class Cursor extends ImmutableObject implements UnsafeSerializable {
|
||||||
|
|
||||||
/** The scope of a global cursor. A global cursor is a cursor that is not specific to one tld. */
|
/** The scope of a global cursor. A global cursor is a cursor that is not specific to one tld. */
|
||||||
public static final String GLOBAL = "GLOBAL";
|
public static final String GLOBAL = "GLOBAL";
|
||||||
|
|
|
@ -27,7 +27,6 @@ import google.registry.config.RegistryEnvironment;
|
||||||
import google.registry.model.CacheUtils;
|
import google.registry.model.CacheUtils;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.common.TimedTransitionProperty.TimedTransition;
|
import google.registry.model.common.TimedTransitionProperty.TimedTransition;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import javax.persistence.Entity;
|
import javax.persistence.Entity;
|
||||||
|
@ -42,7 +41,7 @@ import org.joda.time.DateTime;
|
||||||
*/
|
*/
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
@Entity
|
@Entity
|
||||||
public class DatabaseMigrationStateSchedule extends CrossTldSingleton implements SqlOnlyEntity {
|
public class DatabaseMigrationStateSchedule extends CrossTldSingleton {
|
||||||
|
|
||||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||||
|
|
||||||
|
@ -218,7 +217,7 @@ public class DatabaseMigrationStateSchedule extends CrossTldSingleton implements
|
||||||
MigrationState.DATASTORE_ONLY,
|
MigrationState.DATASTORE_ONLY,
|
||||||
"migrationTransitionMap must start with DATASTORE_ONLY");
|
"migrationTransitionMap must start with DATASTORE_ONLY");
|
||||||
validateTransitionAtCurrentTime(transitions);
|
validateTransitionAtCurrentTime(transitions);
|
||||||
jpaTm().putIgnoringReadOnlyWithoutBackup(new DatabaseMigrationStateSchedule(transitions));
|
jpaTm().putWithoutBackup(new DatabaseMigrationStateSchedule(transitions));
|
||||||
CACHE.invalidateAll();
|
CACHE.invalidateAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@ import com.googlecode.objectify.annotation.Entity;
|
||||||
import com.googlecode.objectify.annotation.Id;
|
import com.googlecode.objectify.annotation.Id;
|
||||||
import google.registry.model.BackupGroupRoot;
|
import google.registry.model.BackupGroupRoot;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -39,7 +38,7 @@ import javax.annotation.Nullable;
|
||||||
*/
|
*/
|
||||||
@Entity
|
@Entity
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class EntityGroupRoot extends BackupGroupRoot implements DatastoreOnlyEntity {
|
public class EntityGroupRoot extends BackupGroupRoot {
|
||||||
|
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
@Id
|
@Id
|
||||||
|
|
|
@ -20,8 +20,6 @@ import google.registry.model.EppResource;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.UnsafeSerializable;
|
import google.registry.model.UnsafeSerializable;
|
||||||
import google.registry.model.contact.ContactHistory.ContactHistoryId;
|
import google.registry.model.contact.ContactHistory.ContactHistoryId;
|
||||||
import google.registry.model.replay.DatastoreEntity;
|
|
||||||
import google.registry.model.replay.SqlEntity;
|
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
@ -58,7 +56,7 @@ import javax.persistence.PostLoad;
|
||||||
@EntitySubclass
|
@EntitySubclass
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
@IdClass(ContactHistoryId.class)
|
@IdClass(ContactHistoryId.class)
|
||||||
public class ContactHistory extends HistoryEntry implements SqlEntity, UnsafeSerializable {
|
public class ContactHistory extends HistoryEntry implements UnsafeSerializable {
|
||||||
|
|
||||||
// Store ContactBase instead of ContactResource so we don't pick up its @Id
|
// Store ContactBase instead of ContactResource so we don't pick up its @Id
|
||||||
// Nullable for the sake of pre-Registry-3.0 history objects
|
// Nullable for the sake of pre-Registry-3.0 history objects
|
||||||
|
@ -128,12 +126,6 @@ public class ContactHistory extends HistoryEntry implements SqlEntity, UnsafeSer
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// In Datastore, save as a HistoryEntry object regardless of this object's type
|
|
||||||
@Override
|
|
||||||
public Optional<DatastoreEntity> toDatastoreEntity() {
|
|
||||||
return Optional.of(asHistoryEntry());
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Class to represent the composite primary key of {@link ContactHistory} entity. */
|
/** Class to represent the composite primary key of {@link ContactHistory} entity. */
|
||||||
public static class ContactHistoryId extends ImmutableObject implements Serializable {
|
public static class ContactHistoryId extends ImmutableObject implements Serializable {
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@ import com.googlecode.objectify.annotation.Entity;
|
||||||
import google.registry.model.EppResource.ForeignKeyedEppResource;
|
import google.registry.model.EppResource.ForeignKeyedEppResource;
|
||||||
import google.registry.model.annotations.ExternalMessagingName;
|
import google.registry.model.annotations.ExternalMessagingName;
|
||||||
import google.registry.model.annotations.ReportedOn;
|
import google.registry.model.annotations.ReportedOn;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.WithStringVKey;
|
import google.registry.persistence.WithStringVKey;
|
||||||
import javax.persistence.Access;
|
import javax.persistence.Access;
|
||||||
|
@ -46,8 +45,7 @@ import org.joda.time.DateTime;
|
||||||
@ExternalMessagingName("contact")
|
@ExternalMessagingName("contact")
|
||||||
@WithStringVKey
|
@WithStringVKey
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
public class ContactResource extends ContactBase
|
public class ContactResource extends ContactBase implements ForeignKeyedEppResource {
|
||||||
implements DatastoreAndSqlEntity, ForeignKeyedEppResource {
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VKey<ContactResource> createVKey() {
|
public VKey<ContactResource> createVKey() {
|
||||||
|
|
|
@ -21,7 +21,6 @@ import google.registry.model.annotations.ExternalMessagingName;
|
||||||
import google.registry.model.annotations.ReportedOn;
|
import google.registry.model.annotations.ReportedOn;
|
||||||
import google.registry.model.domain.secdns.DelegationSignerData;
|
import google.registry.model.domain.secdns.DelegationSignerData;
|
||||||
import google.registry.model.host.HostResource;
|
import google.registry.model.host.HostResource;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.WithStringVKey;
|
import google.registry.persistence.WithStringVKey;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -74,8 +73,7 @@ import org.joda.time.DateTime;
|
||||||
@WithStringVKey
|
@WithStringVKey
|
||||||
@ExternalMessagingName("domain")
|
@ExternalMessagingName("domain")
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
public class DomainBase extends DomainContent
|
public class DomainBase extends DomainContent implements ForeignKeyedEppResource {
|
||||||
implements DatastoreAndSqlEntity, ForeignKeyedEppResource {
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@javax.persistence.Id
|
@javax.persistence.Id
|
||||||
|
|
|
@ -27,8 +27,6 @@ import google.registry.model.domain.GracePeriod.GracePeriodHistory;
|
||||||
import google.registry.model.domain.secdns.DelegationSignerData;
|
import google.registry.model.domain.secdns.DelegationSignerData;
|
||||||
import google.registry.model.domain.secdns.DomainDsDataHistory;
|
import google.registry.model.domain.secdns.DomainDsDataHistory;
|
||||||
import google.registry.model.host.HostResource;
|
import google.registry.model.host.HostResource;
|
||||||
import google.registry.model.replay.DatastoreEntity;
|
|
||||||
import google.registry.model.replay.SqlEntity;
|
|
||||||
import google.registry.model.reporting.DomainTransactionRecord;
|
import google.registry.model.reporting.DomainTransactionRecord;
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
|
@ -80,7 +78,7 @@ import org.hibernate.Hibernate;
|
||||||
@EntitySubclass
|
@EntitySubclass
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
@IdClass(DomainHistoryId.class)
|
@IdClass(DomainHistoryId.class)
|
||||||
public class DomainHistory extends HistoryEntry implements SqlEntity {
|
public class DomainHistory extends HistoryEntry {
|
||||||
|
|
||||||
// Store DomainContent instead of DomainBase so we don't pick up its @Id
|
// Store DomainContent instead of DomainBase so we don't pick up its @Id
|
||||||
// Nullable for the sake of pre-Registry-3.0 history objects
|
// Nullable for the sake of pre-Registry-3.0 history objects
|
||||||
|
@ -295,12 +293,6 @@ public class DomainHistory extends HistoryEntry implements SqlEntity {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// In Datastore, save as a HistoryEntry object regardless of this object's type
|
|
||||||
@Override
|
|
||||||
public Optional<DatastoreEntity> toDatastoreEntity() {
|
|
||||||
return Optional.of(asHistoryEntry());
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void fillAuxiliaryFieldsFromDomain(DomainHistory domainHistory) {
|
private static void fillAuxiliaryFieldsFromDomain(DomainHistory domainHistory) {
|
||||||
if (domainHistory.domainContent != null) {
|
if (domainHistory.domainContent != null) {
|
||||||
domainHistory.nsHosts = nullToEmptyImmutableCopy(domainHistory.domainContent.nsHosts);
|
domainHistory.nsHosts = nullToEmptyImmutableCopy(domainHistory.domainContent.nsHosts);
|
||||||
|
|
|
@ -24,8 +24,6 @@ import google.registry.model.billing.BillingEvent;
|
||||||
import google.registry.model.billing.BillingEvent.Recurring;
|
import google.registry.model.billing.BillingEvent.Recurring;
|
||||||
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
||||||
import google.registry.model.domain.rgp.GracePeriodStatus;
|
import google.registry.model.domain.rgp.GracePeriodStatus;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.persistence.BillingVKey.BillingEventVKey;
|
import google.registry.persistence.BillingVKey.BillingEventVKey;
|
||||||
import google.registry.persistence.BillingVKey.BillingRecurrenceVKey;
|
import google.registry.persistence.BillingVKey.BillingRecurrenceVKey;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
|
@ -52,7 +50,7 @@ import org.joda.time.DateTime;
|
||||||
@Index(columnList = "billing_event_id"),
|
@Index(columnList = "billing_event_id"),
|
||||||
@Index(columnList = "billing_recurrence_id")
|
@Index(columnList = "billing_recurrence_id")
|
||||||
})
|
})
|
||||||
public class GracePeriod extends GracePeriodBase implements DatastoreAndSqlEntity {
|
public class GracePeriod extends GracePeriodBase {
|
||||||
|
|
||||||
@Id
|
@Id
|
||||||
@Access(AccessType.PROPERTY)
|
@Access(AccessType.PROPERTY)
|
||||||
|
@ -197,7 +195,7 @@ public class GracePeriod extends GracePeriodBase implements DatastoreAndSqlEntit
|
||||||
/** Entity class to represent a historic {@link GracePeriod}. */
|
/** Entity class to represent a historic {@link GracePeriod}. */
|
||||||
@Entity(name = "GracePeriodHistory")
|
@Entity(name = "GracePeriodHistory")
|
||||||
@Table(indexes = @Index(columnList = "domainRepoId"))
|
@Table(indexes = @Index(columnList = "domainRepoId"))
|
||||||
public static class GracePeriodHistory extends GracePeriodBase implements SqlOnlyEntity {
|
public static class GracePeriodHistory extends GracePeriodBase {
|
||||||
@Id Long gracePeriodHistoryRevisionId;
|
@Id Long gracePeriodHistoryRevisionId;
|
||||||
|
|
||||||
/** ID for the associated {@link DomainHistory} entity. */
|
/** ID for the associated {@link DomainHistory} entity. */
|
||||||
|
|
|
@ -23,7 +23,6 @@ import google.registry.model.Buildable;
|
||||||
import google.registry.model.CreateAutoTimestamp;
|
import google.registry.model.CreateAutoTimestamp;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.UpdateAutoTimestamp;
|
import google.registry.model.UpdateAutoTimestamp;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.util.DateTimeUtils;
|
import google.registry.util.DateTimeUtils;
|
||||||
import java.time.ZonedDateTime;
|
import java.time.ZonedDateTime;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
@ -76,7 +75,7 @@ import org.joda.time.Duration;
|
||||||
@Index(name = "idx_registry_lock_verification_code", columnList = "verificationCode"),
|
@Index(name = "idx_registry_lock_verification_code", columnList = "verificationCode"),
|
||||||
@Index(name = "idx_registry_lock_registrar_id", columnList = "registrarId")
|
@Index(name = "idx_registry_lock_registrar_id", columnList = "registrarId")
|
||||||
})
|
})
|
||||||
public final class RegistryLock extends ImmutableObject implements Buildable, SqlOnlyEntity {
|
public final class RegistryLock extends ImmutableObject implements Buildable {
|
||||||
|
|
||||||
@Id
|
@Id
|
||||||
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
||||||
|
|
|
@ -19,7 +19,6 @@ import static google.registry.model.IdService.allocateId;
|
||||||
import google.registry.model.UnsafeSerializable;
|
import google.registry.model.UnsafeSerializable;
|
||||||
import google.registry.model.domain.DomainHistory;
|
import google.registry.model.domain.DomainHistory;
|
||||||
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import javax.persistence.Access;
|
import javax.persistence.Access;
|
||||||
import javax.persistence.AccessType;
|
import javax.persistence.AccessType;
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
|
@ -28,8 +27,7 @@ import javax.persistence.Id;
|
||||||
|
|
||||||
/** Entity class to represent a historic {@link DelegationSignerData}. */
|
/** Entity class to represent a historic {@link DelegationSignerData}. */
|
||||||
@Entity
|
@Entity
|
||||||
public class DomainDsDataHistory extends DomainDsDataBase
|
public class DomainDsDataHistory extends DomainDsDataBase implements UnsafeSerializable {
|
||||||
implements SqlOnlyEntity, UnsafeSerializable {
|
|
||||||
|
|
||||||
@Id Long dsDataHistoryRevisionId;
|
@Id Long dsDataHistoryRevisionId;
|
||||||
|
|
||||||
|
|
|
@ -47,7 +47,6 @@ import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
|
||||||
import google.registry.model.common.TimedTransitionProperty;
|
import google.registry.model.common.TimedTransitionProperty;
|
||||||
import google.registry.model.common.TimedTransitionProperty.TimeMapper;
|
import google.registry.model.common.TimedTransitionProperty.TimeMapper;
|
||||||
import google.registry.model.common.TimedTransitionProperty.TimedTransition;
|
import google.registry.model.common.TimedTransitionProperty.TimedTransition;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.persistence.DomainHistoryVKey;
|
import google.registry.persistence.DomainHistoryVKey;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
|
@ -80,7 +79,7 @@ import org.joda.time.DateTime;
|
||||||
@javax.persistence.Index(columnList = "tokenType"),
|
@javax.persistence.Index(columnList = "tokenType"),
|
||||||
@javax.persistence.Index(columnList = "redemption_domain_repo_id")
|
@javax.persistence.Index(columnList = "redemption_domain_repo_id")
|
||||||
})
|
})
|
||||||
public class AllocationToken extends BackupGroupRoot implements Buildable, DatastoreAndSqlEntity {
|
public class AllocationToken extends BackupGroupRoot implements Buildable {
|
||||||
|
|
||||||
// Promotions should only move forward, and ENDED / CANCELLED are terminal states.
|
// Promotions should only move forward, and ENDED / CANCELLED are terminal states.
|
||||||
private static final ImmutableMultimap<TokenStatus, TokenStatus> VALID_TOKEN_STATUS_TRANSITIONS =
|
private static final ImmutableMultimap<TokenStatus, TokenStatus> VALID_TOKEN_STATUS_TRANSITIONS =
|
||||||
|
|
|
@ -20,8 +20,6 @@ import google.registry.model.EppResource;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.UnsafeSerializable;
|
import google.registry.model.UnsafeSerializable;
|
||||||
import google.registry.model.host.HostHistory.HostHistoryId;
|
import google.registry.model.host.HostHistory.HostHistoryId;
|
||||||
import google.registry.model.replay.DatastoreEntity;
|
|
||||||
import google.registry.model.replay.SqlEntity;
|
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
@ -59,7 +57,7 @@ import javax.persistence.PostLoad;
|
||||||
@EntitySubclass
|
@EntitySubclass
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
@IdClass(HostHistoryId.class)
|
@IdClass(HostHistoryId.class)
|
||||||
public class HostHistory extends HistoryEntry implements SqlEntity, UnsafeSerializable {
|
public class HostHistory extends HistoryEntry implements UnsafeSerializable {
|
||||||
|
|
||||||
// Store HostBase instead of HostResource so we don't pick up its @Id
|
// Store HostBase instead of HostResource so we don't pick up its @Id
|
||||||
// Nullable for the sake of pre-Registry-3.0 history objects
|
// Nullable for the sake of pre-Registry-3.0 history objects
|
||||||
|
@ -128,12 +126,6 @@ public class HostHistory extends HistoryEntry implements SqlEntity, UnsafeSerial
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// In Datastore, save as a HistoryEntry object regardless of this object's type
|
|
||||||
@Override
|
|
||||||
public Optional<DatastoreEntity> toDatastoreEntity() {
|
|
||||||
return Optional.of(asHistoryEntry());
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Class to represent the composite primary key of {@link HostHistory} entity. */
|
/** Class to represent the composite primary key of {@link HostHistory} entity. */
|
||||||
public static class HostHistoryId extends ImmutableObject implements Serializable {
|
public static class HostHistoryId extends ImmutableObject implements Serializable {
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@ import com.googlecode.objectify.annotation.Entity;
|
||||||
import google.registry.model.EppResource.ForeignKeyedEppResource;
|
import google.registry.model.EppResource.ForeignKeyedEppResource;
|
||||||
import google.registry.model.annotations.ExternalMessagingName;
|
import google.registry.model.annotations.ExternalMessagingName;
|
||||||
import google.registry.model.annotations.ReportedOn;
|
import google.registry.model.annotations.ReportedOn;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.WithStringVKey;
|
import google.registry.persistence.WithStringVKey;
|
||||||
import javax.persistence.Access;
|
import javax.persistence.Access;
|
||||||
|
@ -55,8 +54,7 @@ import javax.persistence.AccessType;
|
||||||
@ExternalMessagingName("host")
|
@ExternalMessagingName("host")
|
||||||
@WithStringVKey
|
@WithStringVKey
|
||||||
@Access(AccessType.FIELD) // otherwise it'll use the default if the repoId (property)
|
@Access(AccessType.FIELD) // otherwise it'll use the default if the repoId (property)
|
||||||
public class HostResource extends HostBase
|
public class HostResource extends HostBase implements ForeignKeyedEppResource {
|
||||||
implements DatastoreAndSqlEntity, ForeignKeyedEppResource {
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@javax.persistence.Id
|
@javax.persistence.Id
|
||||||
|
|
|
@ -26,14 +26,13 @@ import google.registry.model.BackupGroupRoot;
|
||||||
import google.registry.model.EppResource;
|
import google.registry.model.EppResource;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.annotations.ReportedOn;
|
import google.registry.model.annotations.ReportedOn;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
|
|
||||||
/** An index that allows for quick enumeration of all EppResource entities (e.g. via map reduce). */
|
/** An index that allows for quick enumeration of all EppResource entities (e.g. via map reduce). */
|
||||||
@ReportedOn
|
@ReportedOn
|
||||||
@Entity
|
@Entity
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class EppResourceIndex extends BackupGroupRoot implements DatastoreOnlyEntity {
|
public class EppResourceIndex extends BackupGroupRoot {
|
||||||
|
|
||||||
@Id String id;
|
@Id String id;
|
||||||
|
|
||||||
|
|
|
@ -25,13 +25,12 @@ import google.registry.model.EppResource;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.annotations.VirtualEntity;
|
import google.registry.model.annotations.VirtualEntity;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
|
|
||||||
/** A virtual entity to represent buckets to which EppResourceIndex objects are randomly added. */
|
/** A virtual entity to represent buckets to which EppResourceIndex objects are randomly added. */
|
||||||
@Entity
|
@Entity
|
||||||
@VirtualEntity
|
@VirtualEntity
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class EppResourceIndexBucket extends ImmutableObject implements DatastoreOnlyEntity {
|
public class EppResourceIndexBucket extends ImmutableObject {
|
||||||
|
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
@Id
|
@Id
|
||||||
|
|
|
@ -50,7 +50,6 @@ import google.registry.model.annotations.ReportedOn;
|
||||||
import google.registry.model.contact.ContactResource;
|
import google.registry.model.contact.ContactResource;
|
||||||
import google.registry.model.domain.DomainBase;
|
import google.registry.model.domain.DomainBase;
|
||||||
import google.registry.model.host.HostResource;
|
import google.registry.model.host.HostResource;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.transaction.CriteriaQueryBuilder;
|
import google.registry.persistence.transaction.CriteriaQueryBuilder;
|
||||||
import google.registry.persistence.transaction.JpaTransactionManager;
|
import google.registry.persistence.transaction.JpaTransactionManager;
|
||||||
|
@ -74,20 +73,17 @@ public abstract class ForeignKeyIndex<E extends EppResource> extends BackupGroup
|
||||||
/** The {@link ForeignKeyIndex} type for {@link ContactResource} entities. */
|
/** The {@link ForeignKeyIndex} type for {@link ContactResource} entities. */
|
||||||
@ReportedOn
|
@ReportedOn
|
||||||
@Entity
|
@Entity
|
||||||
public static class ForeignKeyContactIndex extends ForeignKeyIndex<ContactResource>
|
public static class ForeignKeyContactIndex extends ForeignKeyIndex<ContactResource> {}
|
||||||
implements DatastoreOnlyEntity {}
|
|
||||||
|
|
||||||
/** The {@link ForeignKeyIndex} type for {@link DomainBase} entities. */
|
/** The {@link ForeignKeyIndex} type for {@link DomainBase} entities. */
|
||||||
@ReportedOn
|
@ReportedOn
|
||||||
@Entity
|
@Entity
|
||||||
public static class ForeignKeyDomainIndex extends ForeignKeyIndex<DomainBase>
|
public static class ForeignKeyDomainIndex extends ForeignKeyIndex<DomainBase> {}
|
||||||
implements DatastoreOnlyEntity {}
|
|
||||||
|
|
||||||
/** The {@link ForeignKeyIndex} type for {@link HostResource} entities. */
|
/** The {@link ForeignKeyIndex} type for {@link HostResource} entities. */
|
||||||
@ReportedOn
|
@ReportedOn
|
||||||
@Entity
|
@Entity
|
||||||
public static class ForeignKeyHostIndex extends ForeignKeyIndex<HostResource>
|
public static class ForeignKeyHostIndex extends ForeignKeyIndex<HostResource> {}
|
||||||
implements DatastoreOnlyEntity {}
|
|
||||||
|
|
||||||
private static final ImmutableBiMap<
|
private static final ImmutableBiMap<
|
||||||
Class<? extends EppResource>, Class<? extends ForeignKeyIndex<?>>>
|
Class<? extends EppResource>, Class<? extends ForeignKeyIndex<?>>>
|
||||||
|
|
|
@ -34,7 +34,6 @@ import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.annotations.NotBackedUp;
|
import google.registry.model.annotations.NotBackedUp;
|
||||||
import google.registry.model.annotations.NotBackedUp.Reason;
|
import google.registry.model.annotations.NotBackedUp.Reason;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
import google.registry.util.NonFinalForTesting;
|
import google.registry.util.NonFinalForTesting;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
|
@ -53,7 +52,7 @@ import org.joda.time.DateTime;
|
||||||
@Entity
|
@Entity
|
||||||
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class CommitLogBucket extends ImmutableObject implements Buildable, DatastoreOnlyEntity {
|
public class CommitLogBucket extends ImmutableObject implements Buildable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ranges from 1 to {@link RegistryConfig#getCommitLogBucketCount()}, inclusive; starts at 1 since
|
* Ranges from 1 to {@link RegistryConfig#getCommitLogBucketCount()}, inclusive; starts at 1 since
|
||||||
|
|
|
@ -28,7 +28,6 @@ import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.annotations.NotBackedUp;
|
import google.registry.model.annotations.NotBackedUp;
|
||||||
import google.registry.model.annotations.NotBackedUp.Reason;
|
import google.registry.model.annotations.NotBackedUp.Reason;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
@ -47,7 +46,7 @@ import org.joda.time.DateTime;
|
||||||
@Entity
|
@Entity
|
||||||
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class CommitLogCheckpoint extends ImmutableObject implements DatastoreOnlyEntity {
|
public class CommitLogCheckpoint extends ImmutableObject {
|
||||||
|
|
||||||
/** Shared singleton parent entity for commit log checkpoints. */
|
/** Shared singleton parent entity for commit log checkpoints. */
|
||||||
@Parent
|
@Parent
|
||||||
|
|
|
@ -24,14 +24,13 @@ import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.annotations.NotBackedUp;
|
import google.registry.model.annotations.NotBackedUp;
|
||||||
import google.registry.model.annotations.NotBackedUp.Reason;
|
import google.registry.model.annotations.NotBackedUp.Reason;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
/** Singleton parent entity for all commit log checkpoints. */
|
/** Singleton parent entity for all commit log checkpoints. */
|
||||||
@Entity
|
@Entity
|
||||||
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class CommitLogCheckpointRoot extends ImmutableObject implements DatastoreOnlyEntity {
|
public class CommitLogCheckpointRoot extends ImmutableObject {
|
||||||
|
|
||||||
public static final long SINGLETON_ID = 1; // There is always exactly one of these.
|
public static final long SINGLETON_ID = 1; // There is always exactly one of these.
|
||||||
|
|
||||||
|
|
|
@ -26,7 +26,6 @@ import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.annotations.NotBackedUp;
|
import google.registry.model.annotations.NotBackedUp;
|
||||||
import google.registry.model.annotations.NotBackedUp.Reason;
|
import google.registry.model.annotations.NotBackedUp.Reason;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
import java.util.LinkedHashSet;
|
import java.util.LinkedHashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -41,7 +40,7 @@ import org.joda.time.DateTime;
|
||||||
@Entity
|
@Entity
|
||||||
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class CommitLogManifest extends ImmutableObject implements DatastoreOnlyEntity {
|
public class CommitLogManifest extends ImmutableObject {
|
||||||
|
|
||||||
/** Commit log manifests are parented on a random bucket. */
|
/** Commit log manifests are parented on a random bucket. */
|
||||||
@Parent
|
@Parent
|
||||||
|
|
|
@ -30,13 +30,12 @@ import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.annotations.NotBackedUp;
|
import google.registry.model.annotations.NotBackedUp;
|
||||||
import google.registry.model.annotations.NotBackedUp.Reason;
|
import google.registry.model.annotations.NotBackedUp.Reason;
|
||||||
import google.registry.model.replay.DatastoreOnlyEntity;
|
|
||||||
|
|
||||||
/** Representation of a saved entity in a {@link CommitLogManifest} (not deletes). */
|
/** Representation of a saved entity in a {@link CommitLogManifest} (not deletes). */
|
||||||
@Entity
|
@Entity
|
||||||
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
@NotBackedUp(reason = Reason.COMMIT_LOGS)
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class CommitLogMutation extends ImmutableObject implements DatastoreOnlyEntity {
|
public class CommitLogMutation extends ImmutableObject {
|
||||||
|
|
||||||
/** The manifest this belongs to. */
|
/** The manifest this belongs to. */
|
||||||
@Parent
|
@Parent
|
||||||
|
|
|
@ -38,8 +38,6 @@ import google.registry.model.annotations.InCrossTld;
|
||||||
import google.registry.model.contact.ContactHistory;
|
import google.registry.model.contact.ContactHistory;
|
||||||
import google.registry.model.domain.DomainHistory;
|
import google.registry.model.domain.DomainHistory;
|
||||||
import google.registry.model.host.HostHistory;
|
import google.registry.model.host.HostHistory;
|
||||||
import google.registry.model.replay.DatastoreEntity;
|
|
||||||
import google.registry.model.replay.SqlEntity;
|
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.transaction.QueryComposer;
|
import google.registry.persistence.transaction.QueryComposer;
|
||||||
|
@ -357,28 +355,6 @@ public class DatastoreTransactionManager implements TransactionManager {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void putIgnoringReadOnlyWithoutBackup(Object entity) {
|
|
||||||
syncIfTransactionless(
|
|
||||||
getOfy().saveIgnoringReadOnlyWithoutBackup().entities(toDatastoreEntity(entity)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void deleteIgnoringReadOnlyWithoutBackup(VKey<?> key) {
|
|
||||||
syncIfTransactionless(getOfy().deleteIgnoringReadOnlyWithoutBackup().key(key.getOfyKey()));
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Performs the write ignoring read-only restrictions and also writes commit logs. */
|
|
||||||
public void putIgnoringReadOnlyWithBackup(Object entity) {
|
|
||||||
syncIfTransactionless(
|
|
||||||
getOfy().saveIgnoringReadOnlyWithBackup().entities(toDatastoreEntity(entity)));
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Performs the delete ignoring read-only restrictions and also writes commit logs. */
|
|
||||||
public void deleteIgnoringReadOnlyWithBackup(VKey<?> key) {
|
|
||||||
syncIfTransactionless(getOfy().deleteIgnoringReadOnlyWithBackup().key(key.getOfyKey()));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Executes the given {@link Result} instance synchronously if not in a transaction.
|
* Executes the given {@link Result} instance synchronously if not in a transaction.
|
||||||
*
|
*
|
||||||
|
@ -413,22 +389,14 @@ public class DatastoreTransactionManager implements TransactionManager {
|
||||||
return toSqlEntity(getOfy().load().key(key.getOfyKey()).now());
|
return toSqlEntity(getOfy().load().key(key.getOfyKey()).now());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/** Converts a possible {@link HistoryEntry} child to a {@link HistoryEntry}. */
|
||||||
* Converts a possible {@link SqlEntity} to a {@link DatastoreEntity}.
|
|
||||||
*
|
|
||||||
* <p>One example is that this would convert a {@link DomainHistory} to a {@link HistoryEntry}.
|
|
||||||
*/
|
|
||||||
private static Object toDatastoreEntity(@Nullable Object obj) {
|
private static Object toDatastoreEntity(@Nullable Object obj) {
|
||||||
if (obj instanceof SqlEntity) {
|
if (obj instanceof HistoryEntry) {
|
||||||
Optional<DatastoreEntity> possibleDatastoreEntity = ((SqlEntity) obj).toDatastoreEntity();
|
return ((HistoryEntry) obj).asHistoryEntry();
|
||||||
if (possibleDatastoreEntity.isPresent()) {
|
|
||||||
return possibleDatastoreEntity.get();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
return obj;
|
return obj;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Converts many possible {@link SqlEntity} objects to {@link DatastoreEntity} objects. */
|
|
||||||
private static ImmutableList<Object> toDatastoreEntities(ImmutableCollection<?> collection) {
|
private static ImmutableList<Object> toDatastoreEntities(ImmutableCollection<?> collection) {
|
||||||
return collection.stream()
|
return collection.stream()
|
||||||
.map(DatastoreTransactionManager::toDatastoreEntity)
|
.map(DatastoreTransactionManager::toDatastoreEntity)
|
||||||
|
@ -436,21 +404,15 @@ public class DatastoreTransactionManager implements TransactionManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Converts an object to the corresponding {@link SqlEntity} if necessary and possible.
|
* Converts an object to the corresponding child {@link HistoryEntry} if necessary and possible.
|
||||||
*
|
*
|
||||||
* <p>This should be used when returning objects from Datastore to make sure they reflect the most
|
* <p>This should be used when returning objects from Datastore to make sure they reflect the most
|
||||||
* recent type of the object in question.
|
* recent type of the object in question.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public static <T> T toSqlEntity(@Nullable T obj) {
|
public static <T> T toSqlEntity(@Nullable T obj) {
|
||||||
// NB: The Key of the object in question may not necessarily be the resulting class that we
|
if (obj instanceof HistoryEntry) {
|
||||||
// wish to have. For example, because all *History classes are @EntitySubclasses, their Keys
|
return (T) ((HistoryEntry) obj).toChildHistoryEntity();
|
||||||
// will have type HistoryEntry -- even if you create them based off the *History class.
|
|
||||||
if (obj instanceof DatastoreEntity && !(obj instanceof SqlEntity)) {
|
|
||||||
Optional<SqlEntity> possibleSqlEntity = ((DatastoreEntity) obj).toSqlEntity();
|
|
||||||
if (possibleSqlEntity.isPresent()) {
|
|
||||||
return (T) possibleSqlEntity.get();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
return obj;
|
return obj;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,6 @@ import static com.google.common.base.Preconditions.checkState;
|
||||||
import static com.google.common.collect.Maps.uniqueIndex;
|
import static com.google.common.collect.Maps.uniqueIndex;
|
||||||
import static com.googlecode.objectify.ObjectifyService.ofy;
|
import static com.googlecode.objectify.ObjectifyService.ofy;
|
||||||
import static google.registry.config.RegistryConfig.getBaseOfyRetryDuration;
|
import static google.registry.config.RegistryConfig.getBaseOfyRetryDuration;
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.assertNotReadOnlyMode;
|
|
||||||
import static google.registry.util.CollectionUtils.union;
|
import static google.registry.util.CollectionUtils.union;
|
||||||
|
|
||||||
import com.google.appengine.api.datastore.DatastoreFailureException;
|
import com.google.appengine.api.datastore.DatastoreFailureException;
|
||||||
|
@ -134,7 +133,6 @@ public class Ofy {
|
||||||
* <p>We only allow this in transactions so commit logs can be written in tandem with the delete.
|
* <p>We only allow this in transactions so commit logs can be written in tandem with the delete.
|
||||||
*/
|
*/
|
||||||
public Deleter delete() {
|
public Deleter delete() {
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return deleteIgnoringReadOnlyWithBackup();
|
return deleteIgnoringReadOnlyWithBackup();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -144,7 +142,6 @@ public class Ofy {
|
||||||
* <p>No backups get written.
|
* <p>No backups get written.
|
||||||
*/
|
*/
|
||||||
public Deleter deleteWithoutBackup() {
|
public Deleter deleteWithoutBackup() {
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return deleteIgnoringReadOnlyWithoutBackup();
|
return deleteIgnoringReadOnlyWithoutBackup();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -155,7 +152,6 @@ public class Ofy {
|
||||||
* <p>We only allow this in transactions so commit logs can be written in tandem with the save.
|
* <p>We only allow this in transactions so commit logs can be written in tandem with the save.
|
||||||
*/
|
*/
|
||||||
public Saver save() {
|
public Saver save() {
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return saveIgnoringReadOnlyWithBackup();
|
return saveIgnoringReadOnlyWithBackup();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -165,7 +161,6 @@ public class Ofy {
|
||||||
* <p>No backups get written.
|
* <p>No backups get written.
|
||||||
*/
|
*/
|
||||||
public Saver saveWithoutBackup() {
|
public Saver saveWithoutBackup() {
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return saveIgnoringReadOnlyWithoutBackup();
|
return saveIgnoringReadOnlyWithoutBackup();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -44,7 +44,6 @@ import google.registry.model.host.HostResource;
|
||||||
import google.registry.model.poll.PendingActionNotificationResponse.ContactPendingActionNotificationResponse;
|
import google.registry.model.poll.PendingActionNotificationResponse.ContactPendingActionNotificationResponse;
|
||||||
import google.registry.model.poll.PendingActionNotificationResponse.DomainPendingActionNotificationResponse;
|
import google.registry.model.poll.PendingActionNotificationResponse.DomainPendingActionNotificationResponse;
|
||||||
import google.registry.model.poll.PendingActionNotificationResponse.HostPendingActionNotificationResponse;
|
import google.registry.model.poll.PendingActionNotificationResponse.HostPendingActionNotificationResponse;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.model.transfer.TransferData.TransferServerApproveEntity;
|
import google.registry.model.transfer.TransferData.TransferServerApproveEntity;
|
||||||
import google.registry.model.transfer.TransferResponse;
|
import google.registry.model.transfer.TransferResponse;
|
||||||
|
@ -99,7 +98,7 @@ import org.joda.time.DateTime;
|
||||||
@javax.persistence.Index(columnList = "eventTime")
|
@javax.persistence.Index(columnList = "eventTime")
|
||||||
})
|
})
|
||||||
public abstract class PollMessage extends ImmutableObject
|
public abstract class PollMessage extends ImmutableObject
|
||||||
implements Buildable, DatastoreAndSqlEntity, TransferServerApproveEntity, UnsafeSerializable {
|
implements Buildable, TransferServerApproveEntity, UnsafeSerializable {
|
||||||
|
|
||||||
/** Entity id. */
|
/** Entity id. */
|
||||||
@Id
|
@Id
|
||||||
|
|
|
@ -27,7 +27,6 @@ import com.googlecode.objectify.annotation.Ignore;
|
||||||
import google.registry.model.BackupGroupRoot;
|
import google.registry.model.BackupGroupRoot;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.rde.RdeRevision.RdeRevisionId;
|
import google.registry.model.rde.RdeRevision.RdeRevisionId;
|
||||||
import google.registry.model.replay.NonReplicatedEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.converter.LocalDateConverter;
|
import google.registry.persistence.converter.LocalDateConverter;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
@ -51,7 +50,7 @@ import org.joda.time.LocalDate;
|
||||||
@Entity
|
@Entity
|
||||||
@javax.persistence.Entity
|
@javax.persistence.Entity
|
||||||
@IdClass(RdeRevisionId.class)
|
@IdClass(RdeRevisionId.class)
|
||||||
public final class RdeRevision extends BackupGroupRoot implements NonReplicatedEntity {
|
public final class RdeRevision extends BackupGroupRoot {
|
||||||
|
|
||||||
/** String triplet of tld, date, and mode, e.g. {@code soy_2015-09-01_full}. */
|
/** String triplet of tld, date, and mode, e.g. {@code soy_2015-09-01_full}. */
|
||||||
@Id @Transient String id;
|
@Id @Transient String id;
|
||||||
|
|
|
@ -79,7 +79,6 @@ import google.registry.model.annotations.InCrossTld;
|
||||||
import google.registry.model.annotations.ReportedOn;
|
import google.registry.model.annotations.ReportedOn;
|
||||||
import google.registry.model.common.EntityGroupRoot;
|
import google.registry.model.common.EntityGroupRoot;
|
||||||
import google.registry.model.registrar.Registrar.BillingAccountEntry.CurrencyMapper;
|
import google.registry.model.registrar.Registrar.BillingAccountEntry.CurrencyMapper;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.model.tld.Registry;
|
import google.registry.model.tld.Registry;
|
||||||
import google.registry.model.tld.Registry.TldType;
|
import google.registry.model.tld.Registry.TldType;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
|
@ -119,7 +118,7 @@ import org.joda.time.DateTime;
|
||||||
})
|
})
|
||||||
@InCrossTld
|
@InCrossTld
|
||||||
public class Registrar extends ImmutableObject
|
public class Registrar extends ImmutableObject
|
||||||
implements Buildable, DatastoreAndSqlEntity, Jsonifiable, UnsafeSerializable {
|
implements Buildable, Jsonifiable, UnsafeSerializable {
|
||||||
|
|
||||||
/** Represents the type of a registrar entity. */
|
/** Represents the type of a registrar entity. */
|
||||||
public enum Type {
|
public enum Type {
|
||||||
|
|
|
@ -50,7 +50,6 @@ import google.registry.model.UnsafeSerializable;
|
||||||
import google.registry.model.annotations.InCrossTld;
|
import google.registry.model.annotations.InCrossTld;
|
||||||
import google.registry.model.annotations.ReportedOn;
|
import google.registry.model.annotations.ReportedOn;
|
||||||
import google.registry.model.registrar.RegistrarContact.RegistrarPocId;
|
import google.registry.model.registrar.RegistrarContact.RegistrarPocId;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -82,8 +81,7 @@ import javax.persistence.Transient;
|
||||||
})
|
})
|
||||||
@IdClass(RegistrarPocId.class)
|
@IdClass(RegistrarPocId.class)
|
||||||
@InCrossTld
|
@InCrossTld
|
||||||
public class RegistrarContact extends ImmutableObject
|
public class RegistrarContact extends ImmutableObject implements Jsonifiable, UnsafeSerializable {
|
||||||
implements DatastoreAndSqlEntity, Jsonifiable, UnsafeSerializable {
|
|
||||||
|
|
||||||
@Parent @Transient Key<Registrar> parent;
|
@Parent @Transient Key<Registrar> parent;
|
||||||
|
|
||||||
|
|
|
@ -1,33 +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.replay;
|
|
||||||
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
/** An entity that has the same Java object representation in SQL and Datastore. */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public interface DatastoreAndSqlEntity extends DatastoreEntity, SqlEntity {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
default Optional<DatastoreEntity> toDatastoreEntity() {
|
|
||||||
return Optional.of(this);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
default Optional<SqlEntity> toSqlEntity() {
|
|
||||||
return Optional.of(this);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,32 +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.replay;
|
|
||||||
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* An object that can be stored in Datastore and serialized using Objectify's {@link
|
|
||||||
* com.googlecode.objectify.cmd.Saver#toEntity(Object)} code.
|
|
||||||
*
|
|
||||||
* <p>This is used when replaying {@link google.registry.model.ofy.CommitLogManifest}s to import
|
|
||||||
* transactions and data into the secondary SQL store during the first, Datastore-primary, phase of
|
|
||||||
* the migration.
|
|
||||||
*/
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public interface DatastoreEntity {
|
|
||||||
|
|
||||||
Optional<SqlEntity> toSqlEntity();
|
|
||||||
}
|
|
|
@ -1,27 +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.replay;
|
|
||||||
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
/** An entity that is only stored in Datastore, that should not be replayed to SQL. */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public interface DatastoreOnlyEntity extends DatastoreEntity {
|
|
||||||
@Override
|
|
||||||
default Optional<SqlEntity> toSqlEntity() {
|
|
||||||
return Optional.empty();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -27,7 +27,7 @@ import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
/** Datastore entity to keep track of the last SQL transaction imported into the datastore. */
|
/** Datastore entity to keep track of the last SQL transaction imported into the datastore. */
|
||||||
@Entity
|
@Entity
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class LastSqlTransaction extends ImmutableObject implements DatastoreOnlyEntity {
|
public class LastSqlTransaction extends ImmutableObject {
|
||||||
|
|
||||||
/** The key for this singleton. */
|
/** The key for this singleton. */
|
||||||
public static final Key<LastSqlTransaction> KEY = Key.create(LastSqlTransaction.class, 1);
|
public static final Key<LastSqlTransaction> KEY = Key.create(LastSqlTransaction.class, 1);
|
||||||
|
|
|
@ -1,37 +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.replay;
|
|
||||||
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Represents an entity that should not participate in asynchronous replication.
|
|
||||||
*
|
|
||||||
* <p>We expect that this is a result of the entity being dually-written.
|
|
||||||
*/
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public interface NonReplicatedEntity extends DatastoreEntity, SqlEntity {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
default Optional<DatastoreEntity> toDatastoreEntity() {
|
|
||||||
return Optional.empty();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
default Optional<SqlEntity> toSqlEntity() {
|
|
||||||
return Optional.empty();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -35,7 +35,7 @@ import org.joda.time.DateTime;
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
@NotBackedUp(reason = TRANSIENT)
|
@NotBackedUp(reason = TRANSIENT)
|
||||||
@Entity
|
@Entity
|
||||||
public class ReplayGap extends ImmutableObject implements DatastoreOnlyEntity {
|
public class ReplayGap extends ImmutableObject {
|
||||||
@Id long transactionId;
|
@Id long transactionId;
|
||||||
|
|
||||||
// We can't use a CreateAutoTimestamp here because this ends up getting persisted in an ofy
|
// We can't use a CreateAutoTimestamp here because this ends up getting persisted in an ofy
|
||||||
|
|
|
@ -1,48 +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.replay;
|
|
||||||
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
|
||||||
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* An object that can be stored in Cloud SQL using {@link
|
|
||||||
* javax.persistence.EntityManager#persist(Object)}
|
|
||||||
*
|
|
||||||
* <p>This will be used when replaying SQL transactions into Datastore, during the second,
|
|
||||||
* SQL-primary, phase of the migration from Datastore to SQL.
|
|
||||||
*/
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public interface SqlEntity {
|
|
||||||
|
|
||||||
Optional<DatastoreEntity> toDatastoreEntity();
|
|
||||||
|
|
||||||
/** Returns this entity's primary key field(s) in a string. */
|
|
||||||
default String getPrimaryKeyString() {
|
|
||||||
return jpaTm()
|
|
||||||
.transact(
|
|
||||||
() ->
|
|
||||||
String.format(
|
|
||||||
"%s_%s",
|
|
||||||
this.getClass().getSimpleName(),
|
|
||||||
jpaTm()
|
|
||||||
.getEntityManager()
|
|
||||||
.getEntityManagerFactory()
|
|
||||||
.getPersistenceUnitUtil()
|
|
||||||
.getIdentifier(this)));
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,27 +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 java.util.Optional;
|
|
||||||
|
|
||||||
/** An entity that is only stored in SQL, that should not be replayed to Datastore. */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public interface SqlOnlyEntity extends SqlEntity {
|
|
||||||
@Override
|
|
||||||
default Optional<DatastoreEntity> toDatastoreEntity() {
|
|
||||||
return Optional.empty();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -25,7 +25,7 @@ import org.joda.time.DateTime;
|
||||||
|
|
||||||
@Entity
|
@Entity
|
||||||
@DeleteAfterMigration
|
@DeleteAfterMigration
|
||||||
public class SqlReplayCheckpoint extends CrossTldSingleton implements SqlOnlyEntity {
|
public class SqlReplayCheckpoint extends CrossTldSingleton {
|
||||||
|
|
||||||
@Column(nullable = false)
|
@Column(nullable = false)
|
||||||
private DateTime lastReplayTime;
|
private DateTime lastReplayTime;
|
||||||
|
@ -43,6 +43,6 @@ public class SqlReplayCheckpoint extends CrossTldSingleton implements SqlOnlyEnt
|
||||||
SqlReplayCheckpoint checkpoint = new SqlReplayCheckpoint();
|
SqlReplayCheckpoint checkpoint = new SqlReplayCheckpoint();
|
||||||
checkpoint.lastReplayTime = lastReplayTime;
|
checkpoint.lastReplayTime = lastReplayTime;
|
||||||
// this will overwrite the existing object due to the constant revisionId
|
// this will overwrite the existing object due to the constant revisionId
|
||||||
jpaTm().putIgnoringReadOnlyWithoutBackup(checkpoint);
|
jpaTm().put(checkpoint);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,7 +24,6 @@ import google.registry.model.Buildable;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.UnsafeSerializable;
|
import google.registry.model.UnsafeSerializable;
|
||||||
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
import google.registry.model.domain.DomainHistory.DomainHistoryId;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
import javax.persistence.Entity;
|
import javax.persistence.Entity;
|
||||||
import javax.persistence.EnumType;
|
import javax.persistence.EnumType;
|
||||||
|
@ -47,7 +46,7 @@ import org.joda.time.DateTime;
|
||||||
@Embed
|
@Embed
|
||||||
@Entity
|
@Entity
|
||||||
public class DomainTransactionRecord extends ImmutableObject
|
public class DomainTransactionRecord extends ImmutableObject
|
||||||
implements Buildable, DatastoreAndSqlEntity, UnsafeSerializable {
|
implements Buildable, UnsafeSerializable {
|
||||||
|
|
||||||
@Id
|
@Id
|
||||||
@Ignore
|
@Ignore
|
||||||
|
|
|
@ -48,8 +48,6 @@ import google.registry.model.host.HostBase;
|
||||||
import google.registry.model.host.HostHistory;
|
import google.registry.model.host.HostHistory;
|
||||||
import google.registry.model.host.HostHistory.HostHistoryId;
|
import google.registry.model.host.HostHistory.HostHistoryId;
|
||||||
import google.registry.model.host.HostResource;
|
import google.registry.model.host.HostResource;
|
||||||
import google.registry.model.replay.DatastoreEntity;
|
|
||||||
import google.registry.model.replay.SqlEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -86,8 +84,7 @@ import org.joda.time.DateTime;
|
||||||
@Entity
|
@Entity
|
||||||
@MappedSuperclass
|
@MappedSuperclass
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
public class HistoryEntry extends ImmutableObject
|
public class HistoryEntry extends ImmutableObject implements Buildable, UnsafeSerializable {
|
||||||
implements Buildable, DatastoreEntity, UnsafeSerializable {
|
|
||||||
|
|
||||||
/** Represents the type of history entry. */
|
/** Represents the type of history entry. */
|
||||||
public enum Type {
|
public enum Type {
|
||||||
|
@ -405,12 +402,6 @@ public class HistoryEntry extends ImmutableObject
|
||||||
return resultEntity;
|
return resultEntity;
|
||||||
}
|
}
|
||||||
|
|
||||||
// In SQL, save the child type
|
|
||||||
@Override
|
|
||||||
public Optional<SqlEntity> toSqlEntity() {
|
|
||||||
return Optional.of((SqlEntity) toChildHistoryEntity());
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Creates a {@link VKey} instance from a {@link Key} instance. */
|
/** Creates a {@link VKey} instance from a {@link Key} instance. */
|
||||||
public static VKey<? extends HistoryEntry> createVKey(Key<HistoryEntry> key) {
|
public static VKey<? extends HistoryEntry> createVKey(Key<HistoryEntry> key) {
|
||||||
String repoId = key.getParent().getName();
|
String repoId = key.getParent().getName();
|
||||||
|
|
|
@ -21,7 +21,6 @@ import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import google.registry.model.Buildable;
|
import google.registry.model.Buildable;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.util.DomainNameUtils;
|
import google.registry.util.DomainNameUtils;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
|
@ -40,7 +39,7 @@ import org.joda.time.LocalDate;
|
||||||
@Index(name = "spec11threatmatch_tld_idx", columnList = "tld"),
|
@Index(name = "spec11threatmatch_tld_idx", columnList = "tld"),
|
||||||
@Index(name = "spec11threatmatch_check_date_idx", columnList = "checkDate")
|
@Index(name = "spec11threatmatch_check_date_idx", columnList = "checkDate")
|
||||||
})
|
})
|
||||||
public class Spec11ThreatMatch extends ImmutableObject implements Buildable, SqlOnlyEntity {
|
public class Spec11ThreatMatch extends ImmutableObject implements Buildable {
|
||||||
|
|
||||||
/** The type of threat detected. */
|
/** The type of threat detected. */
|
||||||
public enum ThreatType {
|
public enum ThreatType {
|
||||||
|
|
|
@ -30,7 +30,6 @@ import com.googlecode.objectify.annotation.Id;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.annotations.NotBackedUp;
|
import google.registry.model.annotations.NotBackedUp;
|
||||||
import google.registry.model.annotations.NotBackedUp.Reason;
|
import google.registry.model.annotations.NotBackedUp.Reason;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.transaction.JpaTransactionManager;
|
import google.registry.persistence.transaction.JpaTransactionManager;
|
||||||
import google.registry.persistence.transaction.TransactionManager;
|
import google.registry.persistence.transaction.TransactionManager;
|
||||||
|
@ -62,7 +61,7 @@ import org.joda.time.Duration;
|
||||||
@javax.persistence.Entity
|
@javax.persistence.Entity
|
||||||
@Table
|
@Table
|
||||||
@IdClass(Lock.LockId.class)
|
@IdClass(Lock.LockId.class)
|
||||||
public class Lock extends ImmutableObject implements DatastoreAndSqlEntity, Serializable {
|
public class Lock extends ImmutableObject implements Serializable {
|
||||||
|
|
||||||
private static final long serialVersionUID = 756397280691684645L;
|
private static final long serialVersionUID = 756397280691684645L;
|
||||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||||
|
@ -293,7 +292,7 @@ public class Lock extends ImmutableObject implements DatastoreAndSqlEntity, Seri
|
||||||
create(resourceName, scope, requestStatusChecker.getLogId(), now, leaseLength);
|
create(resourceName, scope, requestStatusChecker.getLogId(), now, leaseLength);
|
||||||
// Locks are not parented under an EntityGroupRoot (so as to avoid write
|
// Locks are not parented under an EntityGroupRoot (so as to avoid write
|
||||||
// contention) and don't need to be backed up.
|
// contention) and don't need to be backed up.
|
||||||
transactionManager.putIgnoringReadOnlyWithoutBackup(newLock);
|
transactionManager.put(newLock);
|
||||||
|
|
||||||
return AcquireResult.create(now, lock, newLock, lockState);
|
return AcquireResult.create(now, lock, newLock, lockState);
|
||||||
};
|
};
|
||||||
|
@ -325,7 +324,7 @@ public class Lock extends ImmutableObject implements DatastoreAndSqlEntity, Seri
|
||||||
// Use deleteIgnoringReadOnly() so that we don't create a commit log entry for deleting
|
// Use deleteIgnoringReadOnly() so that we don't create a commit log entry for deleting
|
||||||
// the lock.
|
// the lock.
|
||||||
logger.atInfo().log("Deleting lock: %s", lockId);
|
logger.atInfo().log("Deleting lock: %s", lockId);
|
||||||
transactionManager.deleteIgnoringReadOnlyWithoutBackup(key);
|
transactionManager.delete(key);
|
||||||
|
|
||||||
lockMetrics.recordRelease(
|
lockMetrics.recordRelease(
|
||||||
resourceName,
|
resourceName,
|
||||||
|
|
|
@ -27,7 +27,6 @@ import google.registry.model.CacheUtils;
|
||||||
import google.registry.model.annotations.NotBackedUp;
|
import google.registry.model.annotations.NotBackedUp;
|
||||||
import google.registry.model.annotations.NotBackedUp.Reason;
|
import google.registry.model.annotations.NotBackedUp.Reason;
|
||||||
import google.registry.model.common.CrossTldSingleton;
|
import google.registry.model.common.CrossTldSingleton;
|
||||||
import google.registry.model.replay.NonReplicatedEntity;
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
@ -41,7 +40,7 @@ import javax.persistence.Transient;
|
||||||
@Unindex
|
@Unindex
|
||||||
@NotBackedUp(reason = Reason.AUTO_GENERATED)
|
@NotBackedUp(reason = Reason.AUTO_GENERATED)
|
||||||
// TODO(b/27427316): Replace this with an entry in KMSKeyring
|
// TODO(b/27427316): Replace this with an entry in KMSKeyring
|
||||||
public class ServerSecret extends CrossTldSingleton implements NonReplicatedEntity {
|
public class ServerSecret extends CrossTldSingleton {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cache of the singleton ServerSecret instance that creates it if not present.
|
* Cache of the singleton ServerSecret instance that creates it if not present.
|
||||||
|
|
|
@ -21,7 +21,6 @@ import static google.registry.util.DateTimeUtils.isBeforeOrAt;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import javax.persistence.CollectionTable;
|
import javax.persistence.CollectionTable;
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
|
@ -46,7 +45,7 @@ import org.joda.time.DateTime;
|
||||||
* functional specifications - SMD Revocation List</a>
|
* functional specifications - SMD Revocation List</a>
|
||||||
*/
|
*/
|
||||||
@Entity
|
@Entity
|
||||||
public class SignedMarkRevocationList extends ImmutableObject implements SqlOnlyEntity {
|
public class SignedMarkRevocationList extends ImmutableObject {
|
||||||
|
|
||||||
@Id
|
@Id
|
||||||
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
||||||
|
|
|
@ -61,7 +61,6 @@ import google.registry.model.common.TimedTransitionProperty;
|
||||||
import google.registry.model.common.TimedTransitionProperty.TimedTransition;
|
import google.registry.model.common.TimedTransitionProperty.TimedTransition;
|
||||||
import google.registry.model.domain.fee.BaseFee.FeeType;
|
import google.registry.model.domain.fee.BaseFee.FeeType;
|
||||||
import google.registry.model.domain.fee.Fee;
|
import google.registry.model.domain.fee.Fee;
|
||||||
import google.registry.model.replay.DatastoreAndSqlEntity;
|
|
||||||
import google.registry.model.tld.label.PremiumList;
|
import google.registry.model.tld.label.PremiumList;
|
||||||
import google.registry.model.tld.label.ReservedList;
|
import google.registry.model.tld.label.ReservedList;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
|
@ -90,8 +89,7 @@ import org.joda.time.Duration;
|
||||||
@Entity
|
@Entity
|
||||||
@javax.persistence.Entity(name = "Tld")
|
@javax.persistence.Entity(name = "Tld")
|
||||||
@InCrossTld
|
@InCrossTld
|
||||||
public class Registry extends ImmutableObject
|
public class Registry extends ImmutableObject implements Buildable, UnsafeSerializable {
|
||||||
implements Buildable, DatastoreAndSqlEntity, UnsafeSerializable {
|
|
||||||
|
|
||||||
@Parent @Transient Key<EntityGroupRoot> parent = getCrossTldKey();
|
@Parent @Transient Key<EntityGroupRoot> parent = getCrossTldKey();
|
||||||
|
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.hash.BloomFilter;
|
||||||
import google.registry.model.Buildable;
|
import google.registry.model.Buildable;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.annotations.ReportedOn;
|
import google.registry.model.annotations.ReportedOn;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.model.tld.Registry;
|
import google.registry.model.tld.Registry;
|
||||||
import google.registry.model.tld.label.PremiumList.PremiumEntry;
|
import google.registry.model.tld.label.PremiumList.PremiumEntry;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
@ -53,8 +52,7 @@ import org.joda.money.Money;
|
||||||
@ReportedOn
|
@ReportedOn
|
||||||
@javax.persistence.Entity
|
@javax.persistence.Entity
|
||||||
@Table(indexes = {@Index(columnList = "name", name = "premiumlist_name_idx")})
|
@Table(indexes = {@Index(columnList = "name", name = "premiumlist_name_idx")})
|
||||||
public final class PremiumList extends BaseDomainLabelList<BigDecimal, PremiumEntry>
|
public final class PremiumList extends BaseDomainLabelList<BigDecimal, PremiumEntry> {
|
||||||
implements SqlOnlyEntity {
|
|
||||||
|
|
||||||
@Column(nullable = false)
|
@Column(nullable = false)
|
||||||
CurrencyUnit currency;
|
CurrencyUnit currency;
|
||||||
|
@ -120,7 +118,7 @@ public final class PremiumList extends BaseDomainLabelList<BigDecimal, PremiumEn
|
||||||
*/
|
*/
|
||||||
@javax.persistence.Entity(name = "PremiumEntry")
|
@javax.persistence.Entity(name = "PremiumEntry")
|
||||||
public static class PremiumEntry extends DomainLabelEntry<BigDecimal, PremiumList.PremiumEntry>
|
public static class PremiumEntry extends DomainLabelEntry<BigDecimal, PremiumList.PremiumEntry>
|
||||||
implements Buildable, SqlOnlyEntity, Serializable {
|
implements Buildable, Serializable {
|
||||||
|
|
||||||
@ImmutableObject.Insignificant @javax.persistence.Id Long revisionId;
|
@ImmutableObject.Insignificant @javax.persistence.Id Long revisionId;
|
||||||
|
|
||||||
|
|
|
@ -33,7 +33,6 @@ import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.util.concurrent.UncheckedExecutionException;
|
import com.google.common.util.concurrent.UncheckedExecutionException;
|
||||||
import google.registry.model.Buildable;
|
import google.registry.model.Buildable;
|
||||||
import google.registry.model.CacheUtils;
|
import google.registry.model.CacheUtils;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.model.tld.Registry;
|
import google.registry.model.tld.Registry;
|
||||||
import google.registry.model.tld.label.DomainLabelMetrics.MetricsReservedListMatch;
|
import google.registry.model.tld.label.DomainLabelMetrics.MetricsReservedListMatch;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
@ -61,8 +60,7 @@ import org.joda.time.DateTime;
|
||||||
@javax.persistence.Entity
|
@javax.persistence.Entity
|
||||||
@Table(indexes = {@Index(columnList = "name", name = "reservedlist_name_idx")})
|
@Table(indexes = {@Index(columnList = "name", name = "reservedlist_name_idx")})
|
||||||
public final class ReservedList
|
public final class ReservedList
|
||||||
extends BaseDomainLabelList<ReservationType, ReservedList.ReservedListEntry>
|
extends BaseDomainLabelList<ReservationType, ReservedList.ReservedListEntry> {
|
||||||
implements SqlOnlyEntity {
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mapping from domain name to its reserved list info.
|
* Mapping from domain name to its reserved list info.
|
||||||
|
@ -88,7 +86,7 @@ public final class ReservedList
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Hibernate hook called on the insert of a new ReservedList. Stores the associated {@link
|
* Hibernate hook called on the insert of a new ReservedList. Stores the associated {@link
|
||||||
* ReservedEntry}'s.
|
* ReservedListEntry}'s.
|
||||||
*
|
*
|
||||||
* <p>We need to persist the list entries, but only on the initial insert (not on update) since
|
* <p>We need to persist the list entries, but only on the initial insert (not on update) since
|
||||||
* the entries themselves never get changed, so we only annotate it with {@link PostPersist}, not
|
* the entries themselves never get changed, so we only annotate it with {@link PostPersist}, not
|
||||||
|
@ -113,7 +111,7 @@ public final class ReservedList
|
||||||
*/
|
*/
|
||||||
@javax.persistence.Entity(name = "ReservedEntry")
|
@javax.persistence.Entity(name = "ReservedEntry")
|
||||||
public static class ReservedListEntry extends DomainLabelEntry<ReservationType, ReservedListEntry>
|
public static class ReservedListEntry extends DomainLabelEntry<ReservationType, ReservedListEntry>
|
||||||
implements Buildable, SqlOnlyEntity, Serializable {
|
implements Buildable, Serializable {
|
||||||
|
|
||||||
@Insignificant @Id Long revisionId;
|
@Insignificant @Id Long revisionId;
|
||||||
|
|
||||||
|
|
|
@ -15,7 +15,6 @@
|
||||||
package google.registry.model.tmch;
|
package google.registry.model.tmch;
|
||||||
|
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
import javax.persistence.Entity;
|
import javax.persistence.Entity;
|
||||||
|
@ -30,7 +29,7 @@ import javax.persistence.Id;
|
||||||
* work.
|
* work.
|
||||||
*/
|
*/
|
||||||
@Entity(name = "ClaimsEntry")
|
@Entity(name = "ClaimsEntry")
|
||||||
class ClaimsEntry extends ImmutableObject implements SqlOnlyEntity, Serializable {
|
class ClaimsEntry extends ImmutableObject implements Serializable {
|
||||||
@Id private Long revisionId;
|
@Id private Long revisionId;
|
||||||
@Id private String domainLabel;
|
@Id private String domainLabel;
|
||||||
|
|
||||||
|
|
|
@ -23,7 +23,6 @@ import static google.registry.persistence.transaction.TransactionManagerFactory.
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import google.registry.model.CreateAutoTimestamp;
|
import google.registry.model.CreateAutoTimestamp;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.model.tld.label.ReservedList.ReservedListEntry;
|
import google.registry.model.tld.label.ReservedList.ReservedListEntry;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
@ -52,7 +51,7 @@ import org.joda.time.DateTime;
|
||||||
*/
|
*/
|
||||||
@Entity(name = "ClaimsList")
|
@Entity(name = "ClaimsList")
|
||||||
@Table
|
@Table
|
||||||
public class ClaimsList extends ImmutableObject implements SqlOnlyEntity {
|
public class ClaimsList extends ImmutableObject {
|
||||||
|
|
||||||
@Id
|
@Id
|
||||||
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
||||||
|
|
|
@ -18,7 +18,6 @@ import static com.google.common.base.Preconditions.checkNotNull;
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
||||||
|
|
||||||
import google.registry.model.common.CrossTldSingleton;
|
import google.registry.model.common.CrossTldSingleton;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import javax.annotation.concurrent.Immutable;
|
import javax.annotation.concurrent.Immutable;
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
|
@ -27,7 +26,7 @@ import org.joda.time.DateTime;
|
||||||
/** Singleton for ICANN's TMCH CA certificate revocation list (CRL). */
|
/** Singleton for ICANN's TMCH CA certificate revocation list (CRL). */
|
||||||
@javax.persistence.Entity
|
@javax.persistence.Entity
|
||||||
@Immutable
|
@Immutable
|
||||||
public final class TmchCrl extends CrossTldSingleton implements SqlOnlyEntity {
|
public final class TmchCrl extends CrossTldSingleton {
|
||||||
|
|
||||||
@Column(name = "certificateRevocations", nullable = false)
|
@Column(name = "certificateRevocations", nullable = false)
|
||||||
String crl;
|
String crl;
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.collect.Ordering;
|
||||||
import com.googlecode.objectify.Key;
|
import com.googlecode.objectify.Key;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import google.registry.model.ofy.CommitLogManifest;
|
import google.registry.model.ofy.CommitLogManifest;
|
||||||
import google.registry.persistence.transaction.Transaction;
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -69,12 +68,6 @@ public final class CommitLogRevisionsTranslatorFactory
|
||||||
@Override
|
@Override
|
||||||
ImmutableSortedMap<DateTime, Key<CommitLogManifest>> transformBeforeSave(
|
ImmutableSortedMap<DateTime, Key<CommitLogManifest>> transformBeforeSave(
|
||||||
ImmutableSortedMap<DateTime, Key<CommitLogManifest>> revisions) {
|
ImmutableSortedMap<DateTime, Key<CommitLogManifest>> revisions) {
|
||||||
|
|
||||||
// Don't do anything if we're just doing object serialization.
|
|
||||||
if (Transaction.inSerializationMode()) {
|
|
||||||
return revisions;
|
|
||||||
}
|
|
||||||
|
|
||||||
DateTime now = ofyTm().getTransactionTime();
|
DateTime now = ofyTm().getTransactionTime();
|
||||||
DateTime threshold = now.minus(getCommitLogDatastoreRetention());
|
DateTime threshold = now.minus(getCommitLogDatastoreRetention());
|
||||||
DateTime preThresholdTime = firstNonNull(revisions.floorKey(threshold), START_OF_TIME);
|
DateTime preThresholdTime = firstNonNull(revisions.floorKey(threshold), START_OF_TIME);
|
||||||
|
|
|
@ -14,13 +14,10 @@
|
||||||
|
|
||||||
package google.registry.model.translators;
|
package google.registry.model.translators;
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkState;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
||||||
import static org.joda.time.DateTimeZone.UTC;
|
import static org.joda.time.DateTimeZone.UTC;
|
||||||
|
|
||||||
import google.registry.model.UpdateAutoTimestamp;
|
import google.registry.model.UpdateAutoTimestamp;
|
||||||
import google.registry.persistence.transaction.Transaction;
|
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
|
@ -49,19 +46,6 @@ public class UpdateAutoTimestampTranslatorFactory
|
||||||
/** Save a timestamp, setting it to the current time. */
|
/** Save a timestamp, setting it to the current time. */
|
||||||
@Override
|
@Override
|
||||||
public Date saveValue(UpdateAutoTimestamp pojoValue) {
|
public Date saveValue(UpdateAutoTimestamp pojoValue) {
|
||||||
|
|
||||||
// If we're in the course of Transaction serialization, we have to use the transaction time
|
|
||||||
// here and the JPA transaction manager which is what will ultimately be saved during the
|
|
||||||
// commit.
|
|
||||||
// Note that this branch doesn't respect "auto update disabled", as this state is
|
|
||||||
// specifically to address replay, so we add a runtime check for this.
|
|
||||||
if (Transaction.inSerializationMode()) {
|
|
||||||
checkState(
|
|
||||||
UpdateAutoTimestamp.autoUpdateEnabled(),
|
|
||||||
"Auto-update disabled during transaction serialization.");
|
|
||||||
return jpaTm().getTransactionTime().toDate();
|
|
||||||
}
|
|
||||||
|
|
||||||
return UpdateAutoTimestamp.autoUpdateEnabled()
|
return UpdateAutoTimestamp.autoUpdateEnabled()
|
||||||
? ofyTm().getTransactionTime().toDate()
|
? ofyTm().getTransactionTime().toDate()
|
||||||
: pojoValue.getTimestamp().toDate();
|
: pojoValue.getTimestamp().toDate();
|
||||||
|
|
|
@ -25,7 +25,6 @@ import google.registry.batch.DeleteExpiredDomainsAction;
|
||||||
import google.registry.batch.DeleteLoadTestDataAction;
|
import google.registry.batch.DeleteLoadTestDataAction;
|
||||||
import google.registry.batch.DeleteProberDataAction;
|
import google.registry.batch.DeleteProberDataAction;
|
||||||
import google.registry.batch.ExpandRecurringBillingEventsAction;
|
import google.registry.batch.ExpandRecurringBillingEventsAction;
|
||||||
import google.registry.batch.RefreshDnsOnHostRenameAction;
|
|
||||||
import google.registry.batch.RelockDomainAction;
|
import google.registry.batch.RelockDomainAction;
|
||||||
import google.registry.batch.ResaveAllEppResourcesAction;
|
import google.registry.batch.ResaveAllEppResourcesAction;
|
||||||
import google.registry.batch.ResaveAllEppResourcesPipelineAction;
|
import google.registry.batch.ResaveAllEppResourcesPipelineAction;
|
||||||
|
@ -180,8 +179,6 @@ interface BackendRequestComponent {
|
||||||
|
|
||||||
RefreshDnsAction refreshDnsAction();
|
RefreshDnsAction refreshDnsAction();
|
||||||
|
|
||||||
RefreshDnsOnHostRenameAction refreshDnsOnHostRenameAction();
|
|
||||||
|
|
||||||
RelockDomainAction relockDomainAction();
|
RelockDomainAction relockDomainAction();
|
||||||
|
|
||||||
ResaveAllEppResourcesAction resaveAllEppResourcesAction();
|
ResaveAllEppResourcesAction resaveAllEppResourcesAction();
|
||||||
|
|
|
@ -18,7 +18,6 @@ import static com.google.common.base.Preconditions.checkArgument;
|
||||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||||
import static com.google.common.collect.ImmutableMap.toImmutableMap;
|
import static com.google.common.collect.ImmutableMap.toImmutableMap;
|
||||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||||
import static google.registry.model.ofy.DatastoreTransactionManager.toSqlEntity;
|
|
||||||
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
||||||
import static java.util.AbstractMap.SimpleEntry;
|
import static java.util.AbstractMap.SimpleEntry;
|
||||||
import static java.util.stream.Collectors.joining;
|
import static java.util.stream.Collectors.joining;
|
||||||
|
@ -30,17 +29,11 @@ import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Streams;
|
import com.google.common.collect.Streams;
|
||||||
import com.google.common.flogger.FluentLogger;
|
import com.google.common.flogger.FluentLogger;
|
||||||
import com.googlecode.objectify.Key;
|
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
|
||||||
import google.registry.model.common.DatabaseMigrationStateSchedule.ReplayDirection;
|
|
||||||
import google.registry.model.index.EppResourceIndex;
|
import google.registry.model.index.EppResourceIndex;
|
||||||
import google.registry.model.index.ForeignKeyIndex.ForeignKeyContactIndex;
|
import google.registry.model.index.ForeignKeyIndex.ForeignKeyContactIndex;
|
||||||
import google.registry.model.index.ForeignKeyIndex.ForeignKeyDomainIndex;
|
import google.registry.model.index.ForeignKeyIndex.ForeignKeyDomainIndex;
|
||||||
import google.registry.model.index.ForeignKeyIndex.ForeignKeyHostIndex;
|
import google.registry.model.index.ForeignKeyIndex.ForeignKeyHostIndex;
|
||||||
import google.registry.model.ofy.DatastoreTransactionManager;
|
|
||||||
import google.registry.model.replay.NonReplicatedEntity;
|
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
|
||||||
import google.registry.persistence.JpaRetries;
|
import google.registry.persistence.JpaRetries;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.util.Clock;
|
import google.registry.util.Clock;
|
||||||
|
@ -137,15 +130,14 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
// Postgresql-specific: 'set transaction' command must be called inside a transaction
|
// Postgresql-specific: 'set transaction' command must be called inside a transaction
|
||||||
assertInTransaction();
|
assertInTransaction();
|
||||||
|
|
||||||
ReadOnlyCheckingEntityManager entityManager =
|
EntityManager entityManager = getEntityManager();
|
||||||
(ReadOnlyCheckingEntityManager) getEntityManager();
|
|
||||||
// Isolation is hardcoded to REPEATABLE READ, as specified by parent's Javadoc.
|
// Isolation is hardcoded to REPEATABLE READ, as specified by parent's Javadoc.
|
||||||
entityManager
|
entityManager
|
||||||
.createNativeQuery("SET TRANSACTION ISOLATION LEVEL REPEATABLE READ")
|
.createNativeQuery("SET TRANSACTION ISOLATION LEVEL REPEATABLE READ")
|
||||||
.executeUpdateIgnoringReadOnly();
|
.executeUpdate();
|
||||||
entityManager
|
entityManager
|
||||||
.createNativeQuery(String.format("SET TRANSACTION SNAPSHOT '%s'", snapshotId))
|
.createNativeQuery(String.format("SET TRANSACTION SNAPSHOT '%s'", snapshotId))
|
||||||
.executeUpdateIgnoringReadOnly();
|
.executeUpdate();
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -178,45 +170,12 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <T> T transact(Supplier<T> work) {
|
public <T> T transact(Supplier<T> work) {
|
||||||
return transact(work, true);
|
return retrier.callWithRetry(() -> transactNoRetry(work), JpaRetries::isFailedTxnRetriable);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <T> T transactWithoutBackup(Supplier<T> work) {
|
public <T> T transactWithoutBackup(Supplier<T> work) {
|
||||||
return transact(work, false);
|
return transact(work);
|
||||||
}
|
|
||||||
|
|
||||||
private <T> T transact(Supplier<T> work, boolean withBackup) {
|
|
||||||
return retrier.callWithRetry(
|
|
||||||
() -> {
|
|
||||||
if (inTransaction()) {
|
|
||||||
return work.get();
|
|
||||||
}
|
|
||||||
TransactionInfo txnInfo = transactionInfo.get();
|
|
||||||
txnInfo.entityManager = createReadOnlyCheckingEntityManager();
|
|
||||||
EntityTransaction txn = txnInfo.entityManager.getTransaction();
|
|
||||||
try {
|
|
||||||
txn.begin();
|
|
||||||
txnInfo.start(clock, withBackup);
|
|
||||||
T result = work.get();
|
|
||||||
txnInfo.recordTransaction();
|
|
||||||
txn.commit();
|
|
||||||
return result;
|
|
||||||
} catch (RuntimeException | Error e) {
|
|
||||||
// Error is unchecked!
|
|
||||||
try {
|
|
||||||
txn.rollback();
|
|
||||||
logger.atWarning().log("Error during transaction; transaction rolled back.");
|
|
||||||
} catch (Throwable rollbackException) {
|
|
||||||
logger.atSevere().withCause(rollbackException).log(
|
|
||||||
"Rollback failed; suppressing error.");
|
|
||||||
}
|
|
||||||
throw e;
|
|
||||||
} finally {
|
|
||||||
txnInfo.clear();
|
|
||||||
}
|
|
||||||
},
|
|
||||||
JpaRetries::isFailedTxnRetriable);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -225,13 +184,12 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
return work.get();
|
return work.get();
|
||||||
}
|
}
|
||||||
TransactionInfo txnInfo = transactionInfo.get();
|
TransactionInfo txnInfo = transactionInfo.get();
|
||||||
txnInfo.entityManager = createReadOnlyCheckingEntityManager();
|
txnInfo.entityManager = emf.createEntityManager();
|
||||||
EntityTransaction txn = txnInfo.entityManager.getTransaction();
|
EntityTransaction txn = txnInfo.entityManager.getTransaction();
|
||||||
try {
|
try {
|
||||||
txn.begin();
|
txn.begin();
|
||||||
txnInfo.start(clock, true);
|
txnInfo.start(clock);
|
||||||
T result = work.get();
|
T result = work.get();
|
||||||
txnInfo.recordTransaction();
|
|
||||||
txn.commit();
|
txn.commit();
|
||||||
return result;
|
return result;
|
||||||
} catch (RuntimeException | Error e) {
|
} catch (RuntimeException | Error e) {
|
||||||
|
@ -320,9 +278,7 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
assertInTransaction();
|
assertInTransaction();
|
||||||
// Necessary due to the changes in HistoryEntry representation during the migration to SQL
|
transactionInfo.get().insertObject(entity);
|
||||||
Object toPersist = toSqlEntity(entity);
|
|
||||||
transactionInfo.get().insertObject(toPersist);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -354,9 +310,7 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
assertInTransaction();
|
assertInTransaction();
|
||||||
// Necessary due to the changes in HistoryEntry representation during the migration to SQL
|
transactionInfo.get().updateObject(entity);
|
||||||
Object toPersist = toSqlEntity(entity);
|
|
||||||
transactionInfo.get().updateObject(toPersist);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -393,9 +347,7 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
}
|
}
|
||||||
assertInTransaction();
|
assertInTransaction();
|
||||||
checkArgument(exists(entity), "Given entity does not exist");
|
checkArgument(exists(entity), "Given entity does not exist");
|
||||||
// Necessary due to the changes in HistoryEntry representation during the migration to SQL
|
transactionInfo.get().updateObject(entity);
|
||||||
Object toPersist = toSqlEntity(entity);
|
|
||||||
transactionInfo.get().updateObject(toPersist);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -431,7 +383,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
@Override
|
@Override
|
||||||
public boolean exists(Object entity) {
|
public boolean exists(Object entity) {
|
||||||
checkArgumentNotNull(entity, "entity must be specified");
|
checkArgumentNotNull(entity, "entity must be specified");
|
||||||
entity = toSqlEntity(entity);
|
|
||||||
EntityType<?> entityType = getEntityType(entity.getClass());
|
EntityType<?> entityType = getEntityType(entity.getClass());
|
||||||
ImmutableSet<EntityId> entityIds = getEntityIdsFromEntity(entityType, entity);
|
ImmutableSet<EntityId> entityIds = getEntityIdsFromEntity(entityType, entity);
|
||||||
return exists(entityType.getName(), entityIds);
|
return exists(entityType.getName(), entityIds);
|
||||||
|
@ -475,7 +426,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
@Override
|
@Override
|
||||||
public <T> ImmutableList<T> loadByEntitiesIfPresent(Iterable<T> entities) {
|
public <T> ImmutableList<T> loadByEntitiesIfPresent(Iterable<T> entities) {
|
||||||
return Streams.stream(entities)
|
return Streams.stream(entities)
|
||||||
.map(DatastoreTransactionManager::toSqlEntity)
|
|
||||||
.filter(this::exists)
|
.filter(this::exists)
|
||||||
.map(this::loadByEntity)
|
.map(this::loadByEntity)
|
||||||
.collect(toImmutableList());
|
.collect(toImmutableList());
|
||||||
|
@ -510,16 +460,14 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
public <T> T loadByEntity(T entity) {
|
public <T> T loadByEntity(T entity) {
|
||||||
checkArgumentNotNull(entity, "entity must be specified");
|
checkArgumentNotNull(entity, "entity must be specified");
|
||||||
assertInTransaction();
|
assertInTransaction();
|
||||||
// If the caller requested a HistoryEntry, load the corresponding *History class
|
|
||||||
T possibleChild = toSqlEntity(entity);
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
T returnValue =
|
T returnValue =
|
||||||
(T)
|
(T)
|
||||||
loadByKey(
|
loadByKey(
|
||||||
VKey.createSql(
|
VKey.createSql(
|
||||||
possibleChild.getClass(),
|
entity.getClass(),
|
||||||
// Casting to Serializable is safe according to JPA (JSR 338 sec. 2.4).
|
// Casting to Serializable is safe according to JPA (JSR 338 sec. 2.4).
|
||||||
(Serializable) emf.getPersistenceUnitUtil().getIdentifier(possibleChild)));
|
(Serializable) emf.getPersistenceUnitUtil().getIdentifier(entity)));
|
||||||
return returnValue;
|
return returnValue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -570,7 +518,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
String.format("DELETE FROM %s WHERE %s", entityType.getName(), getAndClause(entityIds));
|
String.format("DELETE FROM %s WHERE %s", entityType.getName(), getAndClause(entityIds));
|
||||||
Query query = query(sql);
|
Query query = query(sql);
|
||||||
entityIds.forEach(entityId -> query.setParameter(entityId.name, entityId.value));
|
entityIds.forEach(entityId -> query.setParameter(entityId.name, entityId.value));
|
||||||
transactionInfo.get().addDelete(key);
|
|
||||||
return query.executeUpdate();
|
return query.executeUpdate();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -592,7 +539,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
return entity;
|
return entity;
|
||||||
}
|
}
|
||||||
assertInTransaction();
|
assertInTransaction();
|
||||||
entity = toSqlEntity(entity);
|
|
||||||
T managedEntity = entity;
|
T managedEntity = entity;
|
||||||
if (!getEntityManager().contains(entity)) {
|
if (!getEntityManager().contains(entity)) {
|
||||||
// We don't add the entity to "objectsToSave": once deleted, the object should never be
|
// We don't add the entity to "objectsToSave": once deleted, the object should never be
|
||||||
|
@ -600,13 +546,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
managedEntity = getEntityManager().merge(entity);
|
managedEntity = getEntityManager().merge(entity);
|
||||||
}
|
}
|
||||||
getEntityManager().remove(managedEntity);
|
getEntityManager().remove(managedEntity);
|
||||||
|
|
||||||
// We check shouldReplicate() in TransactionInfo.addDelete(), but we have to check it here as
|
|
||||||
// well prior to attempting to create a datastore key because a non-replicated entity may not
|
|
||||||
// have one.
|
|
||||||
if (shouldReplicate(entity.getClass())) {
|
|
||||||
transactionInfo.get().addDelete(VKey.from(Key.create(entity)));
|
|
||||||
}
|
|
||||||
return managedEntity;
|
return managedEntity;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -640,38 +579,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void putIgnoringReadOnlyWithoutBackup(Object entity) {
|
|
||||||
checkArgumentNotNull(entity);
|
|
||||||
if (isEntityOfIgnoredClass(entity)) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
assertInTransaction();
|
|
||||||
// Necessary due to the changes in HistoryEntry representation during the migration to SQL
|
|
||||||
Object toPersist = toSqlEntity(entity);
|
|
||||||
TransactionInfo txn = transactionInfo.get();
|
|
||||||
Object merged = txn.entityManager.mergeIgnoringReadOnly(toPersist);
|
|
||||||
txn.objectsToSave.add(merged);
|
|
||||||
txn.addUpdate(toPersist);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void deleteIgnoringReadOnlyWithoutBackup(VKey<?> key) {
|
|
||||||
checkArgumentNotNull(key, "key must be specified");
|
|
||||||
assertInTransaction();
|
|
||||||
if (IGNORED_ENTITY_CLASSES.contains(key.getKind())) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
EntityType<?> entityType = getEntityType(key.getKind());
|
|
||||||
ImmutableSet<EntityId> entityIds = getEntityIdsFromSqlKey(entityType, key.getSqlKey());
|
|
||||||
String sql =
|
|
||||||
String.format("DELETE FROM %s WHERE %s", entityType.getName(), getAndClause(entityIds));
|
|
||||||
ReadOnlyCheckingQuery query = transactionInfo.get().entityManager.createQuery(sql);
|
|
||||||
entityIds.forEach(entityId -> query.setParameter(entityId.name, entityId.value));
|
|
||||||
transactionInfo.get().addDelete(key);
|
|
||||||
query.executeUpdateIgnoringReadOnly();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <T> void assertDelete(VKey<T> key) {
|
public <T> void assertDelete(VKey<T> key) {
|
||||||
if (internalDelete(key) != 1) {
|
if (internalDelete(key) != 1) {
|
||||||
|
@ -680,10 +587,6 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private ReadOnlyCheckingEntityManager createReadOnlyCheckingEntityManager() {
|
|
||||||
return new ReadOnlyCheckingEntityManager(emf.createEntityManager());
|
|
||||||
}
|
|
||||||
|
|
||||||
private <T> EntityType<T> getEntityType(Class<T> clazz) {
|
private <T> EntityType<T> getEntityType(Class<T> clazz) {
|
||||||
return emf.getMetamodel().entity(clazz);
|
return emf.getMetamodel().entity(clazz);
|
||||||
}
|
}
|
||||||
|
@ -820,46 +723,28 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
return entity;
|
return entity;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns true if the entity class should be replicated from SQL to datastore. */
|
|
||||||
private static boolean shouldReplicate(Class<?> entityClass) {
|
|
||||||
return !NonReplicatedEntity.class.isAssignableFrom(entityClass)
|
|
||||||
&& !SqlOnlyEntity.class.isAssignableFrom(entityClass);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class TransactionInfo {
|
private static class TransactionInfo {
|
||||||
ReadOnlyCheckingEntityManager entityManager;
|
EntityManager entityManager;
|
||||||
boolean inTransaction = false;
|
boolean inTransaction = false;
|
||||||
DateTime transactionTime;
|
DateTime transactionTime;
|
||||||
|
|
||||||
// Serializable representation of the transaction to be persisted in the Transaction table.
|
|
||||||
Transaction.Builder contentsBuilder;
|
|
||||||
|
|
||||||
// The set of entity objects that have been either persisted (via insert()) or merged (via
|
// The set of entity objects that have been either persisted (via insert()) or merged (via
|
||||||
// put()/update()). If the entity manager returns these as a result of a find() or query
|
// put()/update()). If the entity manager returns these as a result of a find() or query
|
||||||
// operation, we can not detach them -- detaching removes them from the transaction and causes
|
// operation, we can not detach them -- detaching removes them from the transaction and causes
|
||||||
// them to not be saved to the database -- so we throw an exception instead.
|
// them to not be saved to the database -- so we throw an exception instead.
|
||||||
Set<Object> objectsToSave = Collections.newSetFromMap(new IdentityHashMap<Object, Boolean>());
|
Set<Object> objectsToSave = Collections.newSetFromMap(new IdentityHashMap<>());
|
||||||
|
|
||||||
/** Start a new transaction. */
|
/** Start a new transaction. */
|
||||||
private void start(Clock clock, boolean withBackup) {
|
private void start(Clock clock) {
|
||||||
checkArgumentNotNull(clock);
|
checkArgumentNotNull(clock);
|
||||||
inTransaction = true;
|
inTransaction = true;
|
||||||
transactionTime = clock.nowUtc();
|
transactionTime = clock.nowUtc();
|
||||||
Supplier<Boolean> sqlToDsReplaySupplier =
|
|
||||||
() ->
|
|
||||||
DatabaseMigrationStateSchedule.getValueAtTime(transactionTime)
|
|
||||||
.getReplayDirection()
|
|
||||||
.equals(ReplayDirection.SQL_TO_DATASTORE);
|
|
||||||
if (withBackup && sqlToDsReplaySupplier.get()) {
|
|
||||||
contentsBuilder = new Transaction.Builder();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void clear() {
|
private void clear() {
|
||||||
inTransaction = false;
|
inTransaction = false;
|
||||||
transactionTime = null;
|
transactionTime = null;
|
||||||
contentsBuilder = null;
|
objectsToSave = Collections.newSetFromMap(new IdentityHashMap<>());
|
||||||
objectsToSave = Collections.newSetFromMap(new IdentityHashMap<Object, Boolean>());
|
|
||||||
if (entityManager != null) {
|
if (entityManager != null) {
|
||||||
// Close this EntityManager just let the connection pool be able to reuse it, it doesn't
|
// Close this EntityManager just let the connection pool be able to reuse it, it doesn't
|
||||||
// close the underlying database connection.
|
// close the underlying database connection.
|
||||||
|
@ -868,39 +753,16 @@ public class JpaTransactionManagerImpl implements JpaTransactionManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void addUpdate(Object entity) {
|
|
||||||
if (contentsBuilder != null && shouldReplicate(entity.getClass())) {
|
|
||||||
contentsBuilder.addUpdate(entity);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void addDelete(VKey<?> key) {
|
|
||||||
if (contentsBuilder != null && shouldReplicate(key.getKind())) {
|
|
||||||
contentsBuilder.addDelete(key);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void recordTransaction() {
|
|
||||||
if (contentsBuilder != null) {
|
|
||||||
Transaction persistedTxn = contentsBuilder.build();
|
|
||||||
if (!persistedTxn.isEmpty()) {
|
|
||||||
entityManager.persist(persistedTxn.toEntity());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Does the full "update" on an object including all internal housekeeping. */
|
/** Does the full "update" on an object including all internal housekeeping. */
|
||||||
private void updateObject(Object object) {
|
private void updateObject(Object object) {
|
||||||
Object merged = entityManager.merge(object);
|
Object merged = entityManager.merge(object);
|
||||||
objectsToSave.add(merged);
|
objectsToSave.add(merged);
|
||||||
addUpdate(object);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Does the full "insert" on a new object including all internal housekeeping. */
|
/** Does the full "insert" on a new object including all internal housekeeping. */
|
||||||
private void insertObject(Object object) {
|
private void insertObject(Object object) {
|
||||||
entityManager.persist(object);
|
entityManager.persist(object);
|
||||||
objectsToSave.add(object);
|
objectsToSave.add(object);
|
||||||
addUpdate(object);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns true if the object has been persisted/merged and will be saved on commit. */
|
/** Returns true if the object has been persisted/merged and will be saved on commit. */
|
||||||
|
|
|
@ -1,322 +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.persistence.transaction;
|
|
||||||
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.assertNotReadOnlyMode;
|
|
||||||
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import javax.persistence.EntityGraph;
|
|
||||||
import javax.persistence.EntityManager;
|
|
||||||
import javax.persistence.EntityManagerFactory;
|
|
||||||
import javax.persistence.EntityTransaction;
|
|
||||||
import javax.persistence.FlushModeType;
|
|
||||||
import javax.persistence.LockModeType;
|
|
||||||
import javax.persistence.Query;
|
|
||||||
import javax.persistence.StoredProcedureQuery;
|
|
||||||
import javax.persistence.TypedQuery;
|
|
||||||
import javax.persistence.criteria.CriteriaBuilder;
|
|
||||||
import javax.persistence.criteria.CriteriaDelete;
|
|
||||||
import javax.persistence.criteria.CriteriaQuery;
|
|
||||||
import javax.persistence.criteria.CriteriaUpdate;
|
|
||||||
import javax.persistence.metamodel.Metamodel;
|
|
||||||
|
|
||||||
/** An {@link EntityManager} that throws exceptions on write actions if in read-only mode. */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
public class ReadOnlyCheckingEntityManager implements EntityManager {
|
|
||||||
|
|
||||||
private final EntityManager delegate;
|
|
||||||
|
|
||||||
public ReadOnlyCheckingEntityManager(EntityManager delegate) {
|
|
||||||
this.delegate = delegate;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void persist(Object entity) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
delegate.persist(entity);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T merge(T entity) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.merge(entity);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void remove(Object entity) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
delegate.remove(entity);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T find(Class<T> entityClass, Object primaryKey) {
|
|
||||||
return delegate.find(entityClass, primaryKey);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T find(Class<T> entityClass, Object primaryKey, Map<String, Object> properties) {
|
|
||||||
return delegate.find(entityClass, primaryKey, properties);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T find(Class<T> entityClass, Object primaryKey, LockModeType lockMode) {
|
|
||||||
return delegate.find(entityClass, primaryKey, lockMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T find(
|
|
||||||
Class<T> entityClass,
|
|
||||||
Object primaryKey,
|
|
||||||
LockModeType lockMode,
|
|
||||||
Map<String, Object> properties) {
|
|
||||||
return delegate.find(entityClass, primaryKey, lockMode, properties);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T getReference(Class<T> entityClass, Object primaryKey) {
|
|
||||||
return delegate.getReference(entityClass, primaryKey);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void flush() {
|
|
||||||
delegate.flush();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setFlushMode(FlushModeType flushMode) {
|
|
||||||
delegate.setFlushMode(flushMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public FlushModeType getFlushMode() {
|
|
||||||
return delegate.getFlushMode();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void lock(Object entity, LockModeType lockMode) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
delegate.lock(entity, lockMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void lock(Object entity, LockModeType lockMode, Map<String, Object> properties) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
delegate.lock(entity, lockMode, properties);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void refresh(Object entity) {
|
|
||||||
delegate.refresh(entity);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void refresh(Object entity, Map<String, Object> properties) {
|
|
||||||
delegate.refresh(entity, properties);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void refresh(Object entity, LockModeType lockMode) {
|
|
||||||
delegate.refresh(entity, lockMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void refresh(Object entity, LockModeType lockMode, Map<String, Object> properties) {
|
|
||||||
delegate.refresh(entity, lockMode, properties);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void clear() {
|
|
||||||
delegate.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void detach(Object entity) {
|
|
||||||
delegate.detach(entity);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean contains(Object entity) {
|
|
||||||
return delegate.contains(entity);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LockModeType getLockMode(Object entity) {
|
|
||||||
return delegate.getLockMode(entity);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setProperty(String propertyName, Object value) {
|
|
||||||
delegate.setProperty(propertyName, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Map<String, Object> getProperties() {
|
|
||||||
return delegate.getProperties();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ReadOnlyCheckingQuery createQuery(String qlString) {
|
|
||||||
return new ReadOnlyCheckingQuery(delegate.createQuery(qlString));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> TypedQuery<T> createQuery(CriteriaQuery<T> criteriaQuery) {
|
|
||||||
return new ReadOnlyCheckingTypedQuery<>(delegate.createQuery(criteriaQuery));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query createQuery(CriteriaUpdate updateQuery) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.createQuery(updateQuery);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query createQuery(CriteriaDelete deleteQuery) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.createQuery(deleteQuery);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> TypedQuery<T> createQuery(String qlString, Class<T> resultClass) {
|
|
||||||
return new ReadOnlyCheckingTypedQuery<>(delegate.createQuery(qlString, resultClass));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query createNamedQuery(String name) {
|
|
||||||
return new ReadOnlyCheckingQuery(delegate.createNamedQuery(name));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> TypedQuery<T> createNamedQuery(String name, Class<T> resultClass) {
|
|
||||||
return new ReadOnlyCheckingTypedQuery<>(delegate.createNamedQuery(name, resultClass));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ReadOnlyCheckingQuery createNativeQuery(String sqlString) {
|
|
||||||
return new ReadOnlyCheckingQuery(delegate.createNativeQuery(sqlString));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query createNativeQuery(String sqlString, Class resultClass) {
|
|
||||||
return new ReadOnlyCheckingQuery(delegate.createNativeQuery(sqlString, resultClass));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query createNativeQuery(String sqlString, String resultSetMapping) {
|
|
||||||
return new ReadOnlyCheckingQuery(delegate.createNativeQuery(sqlString, resultSetMapping));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public StoredProcedureQuery createNamedStoredProcedureQuery(String name) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.createNamedStoredProcedureQuery(name);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public StoredProcedureQuery createStoredProcedureQuery(String procedureName) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.createStoredProcedureQuery(procedureName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public StoredProcedureQuery createStoredProcedureQuery(
|
|
||||||
String procedureName, Class... resultClasses) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.createStoredProcedureQuery(procedureName, resultClasses);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public StoredProcedureQuery createStoredProcedureQuery(
|
|
||||||
String procedureName, String... resultSetMappings) {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.createStoredProcedureQuery(procedureName, resultSetMappings);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void joinTransaction() {
|
|
||||||
delegate.joinTransaction();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isJoinedToTransaction() {
|
|
||||||
return delegate.isJoinedToTransaction();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T unwrap(Class<T> cls) {
|
|
||||||
return delegate.unwrap(cls);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object getDelegate() {
|
|
||||||
return delegate.getDelegate();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() {
|
|
||||||
delegate.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isOpen() {
|
|
||||||
return delegate.isOpen();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public EntityTransaction getTransaction() {
|
|
||||||
return delegate.getTransaction();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public EntityManagerFactory getEntityManagerFactory() {
|
|
||||||
return delegate.getEntityManagerFactory();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public CriteriaBuilder getCriteriaBuilder() {
|
|
||||||
return delegate.getCriteriaBuilder();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Metamodel getMetamodel() {
|
|
||||||
return delegate.getMetamodel();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> EntityGraph<T> createEntityGraph(Class<T> rootType) {
|
|
||||||
return delegate.createEntityGraph(rootType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public EntityGraph<?> createEntityGraph(String graphName) {
|
|
||||||
return delegate.createEntityGraph(graphName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public EntityGraph<?> getEntityGraph(String graphName) {
|
|
||||||
return delegate.getEntityGraph(graphName);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> List<EntityGraph<? super T>> getEntityGraphs(Class<T> entityClass) {
|
|
||||||
return delegate.getEntityGraphs(entityClass);
|
|
||||||
}
|
|
||||||
|
|
||||||
public <T> T mergeIgnoringReadOnly(T entity) {
|
|
||||||
return delegate.merge(entity);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,211 +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.persistence.transaction;
|
|
||||||
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.assertNotReadOnlyMode;
|
|
||||||
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.Calendar;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.stream.Stream;
|
|
||||||
import javax.persistence.FlushModeType;
|
|
||||||
import javax.persistence.LockModeType;
|
|
||||||
import javax.persistence.Parameter;
|
|
||||||
import javax.persistence.Query;
|
|
||||||
import javax.persistence.TemporalType;
|
|
||||||
|
|
||||||
/** A {@link Query} that throws exceptions on write actions if in read-only mode. */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
class ReadOnlyCheckingQuery implements Query {
|
|
||||||
|
|
||||||
private final Query delegate;
|
|
||||||
|
|
||||||
ReadOnlyCheckingQuery(Query delegate) {
|
|
||||||
this.delegate = delegate;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List getResultList() {
|
|
||||||
return delegate.getResultList();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Stream getResultStream() {
|
|
||||||
return delegate.getResultStream();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object getSingleResult() {
|
|
||||||
return delegate.getSingleResult();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int executeUpdate() {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.executeUpdate();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setMaxResults(int maxResult) {
|
|
||||||
return delegate.setMaxResults(maxResult);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int getMaxResults() {
|
|
||||||
return delegate.getMaxResults();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setFirstResult(int startPosition) {
|
|
||||||
return delegate.setFirstResult(startPosition);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int getFirstResult() {
|
|
||||||
return delegate.getFirstResult();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setHint(String hintName, Object value) {
|
|
||||||
return delegate.setHint(hintName, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Map<String, Object> getHints() {
|
|
||||||
return delegate.getHints();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> Query setParameter(Parameter<T> param, T value) {
|
|
||||||
return delegate.setParameter(param, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setParameter(Parameter<Calendar> param, Calendar value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(param, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setParameter(Parameter<Date> param, Date value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(param, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setParameter(String name, Object value) {
|
|
||||||
return delegate.setParameter(name, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setParameter(String name, Calendar value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(name, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setParameter(String name, Date value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(name, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setParameter(int position, Object value) {
|
|
||||||
return delegate.setParameter(position, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setParameter(int position, Calendar value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(position, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setParameter(int position, Date value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(position, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Set<Parameter<?>> getParameters() {
|
|
||||||
return delegate.getParameters();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Parameter<?> getParameter(String name) {
|
|
||||||
return delegate.getParameter(name);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> Parameter<T> getParameter(String name, Class<T> type) {
|
|
||||||
return delegate.getParameter(name, type);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Parameter<?> getParameter(int position) {
|
|
||||||
return delegate.getParameter(position);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> Parameter<T> getParameter(int position, Class<T> type) {
|
|
||||||
return delegate.getParameter(position, type);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isBound(Parameter<?> param) {
|
|
||||||
return delegate.isBound(param);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T getParameterValue(Parameter<T> param) {
|
|
||||||
return delegate.getParameterValue(param);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object getParameterValue(String name) {
|
|
||||||
return delegate.getParameterValue(name);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object getParameterValue(int position) {
|
|
||||||
return delegate.getParameterValue(position);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setFlushMode(FlushModeType flushMode) {
|
|
||||||
return delegate.setFlushMode(flushMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public FlushModeType getFlushMode() {
|
|
||||||
return delegate.getFlushMode();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Query setLockMode(LockModeType lockMode) {
|
|
||||||
return delegate.setLockMode(lockMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LockModeType getLockMode() {
|
|
||||||
return delegate.getLockMode();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T unwrap(Class<T> cls) {
|
|
||||||
return delegate.unwrap(cls);
|
|
||||||
}
|
|
||||||
|
|
||||||
public int executeUpdateIgnoringReadOnly() {
|
|
||||||
return delegate.executeUpdate();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,208 +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.persistence.transaction;
|
|
||||||
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.assertNotReadOnlyMode;
|
|
||||||
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import java.util.Calendar;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.stream.Stream;
|
|
||||||
import javax.persistence.FlushModeType;
|
|
||||||
import javax.persistence.LockModeType;
|
|
||||||
import javax.persistence.Parameter;
|
|
||||||
import javax.persistence.TemporalType;
|
|
||||||
import javax.persistence.TypedQuery;
|
|
||||||
|
|
||||||
/** A {@link TypedQuery <T>} that throws exceptions on write actions if in read-only mode. */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
class ReadOnlyCheckingTypedQuery<T> implements TypedQuery<T> {
|
|
||||||
|
|
||||||
private final TypedQuery<T> delegate;
|
|
||||||
|
|
||||||
ReadOnlyCheckingTypedQuery(TypedQuery<T> delegate) {
|
|
||||||
this.delegate = delegate;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<T> getResultList() {
|
|
||||||
return delegate.getResultList();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Stream<T> getResultStream() {
|
|
||||||
return delegate.getResultStream();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public T getSingleResult() {
|
|
||||||
return delegate.getSingleResult();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int executeUpdate() {
|
|
||||||
assertNotReadOnlyMode();
|
|
||||||
return delegate.executeUpdate();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setMaxResults(int maxResult) {
|
|
||||||
return delegate.setMaxResults(maxResult);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int getMaxResults() {
|
|
||||||
return delegate.getMaxResults();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setFirstResult(int startPosition) {
|
|
||||||
return delegate.setFirstResult(startPosition);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int getFirstResult() {
|
|
||||||
return delegate.getFirstResult();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setHint(String hintName, Object value) {
|
|
||||||
return delegate.setHint(hintName, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Map<String, Object> getHints() {
|
|
||||||
return delegate.getHints();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T1> TypedQuery<T> setParameter(Parameter<T1> param, T1 value) {
|
|
||||||
return delegate.setParameter(param, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setParameter(
|
|
||||||
Parameter<Calendar> param, Calendar value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(param, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setParameter(Parameter<Date> param, Date value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(param, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setParameter(String name, Object value) {
|
|
||||||
return delegate.setParameter(name, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setParameter(String name, Calendar value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(name, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setParameter(String name, Date value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(name, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setParameter(int position, Object value) {
|
|
||||||
return delegate.setParameter(position, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setParameter(int position, Calendar value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(position, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setParameter(int position, Date value, TemporalType temporalType) {
|
|
||||||
return delegate.setParameter(position, value, temporalType);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Set<Parameter<?>> getParameters() {
|
|
||||||
return delegate.getParameters();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Parameter<?> getParameter(String name) {
|
|
||||||
return delegate.getParameter(name);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <X> Parameter<X> getParameter(String name, Class<X> type) {
|
|
||||||
return delegate.getParameter(name, type);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Parameter<?> getParameter(int position) {
|
|
||||||
return delegate.getParameter(position);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <X> Parameter<X> getParameter(int position, Class<X> type) {
|
|
||||||
return delegate.getParameter(position, type);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isBound(Parameter<?> param) {
|
|
||||||
return delegate.isBound(param);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <X> X getParameterValue(Parameter<X> param) {
|
|
||||||
return delegate.getParameterValue(param);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object getParameterValue(String name) {
|
|
||||||
return delegate.getParameterValue(name);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object getParameterValue(int position) {
|
|
||||||
return delegate.getParameterValue(position);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setFlushMode(FlushModeType flushMode) {
|
|
||||||
return delegate.setFlushMode(flushMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public FlushModeType getFlushMode() {
|
|
||||||
return delegate.getFlushMode();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TypedQuery<T> setLockMode(LockModeType lockMode) {
|
|
||||||
return delegate.setLockMode(lockMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LockModeType getLockMode() {
|
|
||||||
return delegate.getLockMode();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <X> X unwrap(Class<X> cls) {
|
|
||||||
return delegate.unwrap(cls);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,336 +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.persistence.transaction;
|
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkArgument;
|
|
||||||
import static com.google.common.base.Preconditions.checkNotNull;
|
|
||||||
import static google.registry.model.ofy.ObjectifyService.auditedOfy;
|
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.ofyTm;
|
|
||||||
|
|
||||||
import com.google.appengine.api.datastore.Entity;
|
|
||||||
import com.google.appengine.api.datastore.EntityTranslator;
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
|
||||||
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.SqlEntity;
|
|
||||||
import google.registry.persistence.VKey;
|
|
||||||
import java.io.ByteArrayInputStream;
|
|
||||||
import java.io.ByteArrayOutputStream;
|
|
||||||
import java.io.EOFException;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.InputStream;
|
|
||||||
import java.io.ObjectInputStream;
|
|
||||||
import java.io.ObjectOutputStream;
|
|
||||||
import java.io.ObjectStreamClass;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A SQL transaction that can be serialized and stored in its own table.
|
|
||||||
*
|
|
||||||
* <p>Transaction is used to store transactions committed to Cloud SQL in a Transaction table during
|
|
||||||
* the second phase of our migration, during which time we will be asynchronously replaying Cloud
|
|
||||||
* SQL transactions to datastore.
|
|
||||||
*
|
|
||||||
* <p>TODO(mmuller): Use these from {@link TransactionManager} to store the contents of an SQL
|
|
||||||
* transaction for asynchronous propagation to datastore. Implement a cron endpoint that reads them
|
|
||||||
* from the Transaction table and calls writeToDatastore().
|
|
||||||
*/
|
|
||||||
public class Transaction extends ImmutableObject implements Buildable {
|
|
||||||
|
|
||||||
// Version id for persisted objects. Use the creation date for the value, as it's reasonably
|
|
||||||
// unique and inherently informative.
|
|
||||||
private static final int VERSION_ID = 20200604;
|
|
||||||
|
|
||||||
// Keep a per-thread flag to keep track of whether we're serializing an entity for a transaction.
|
|
||||||
// This is used by internal translators to avoid doing things that are dependent on being in a
|
|
||||||
// datastore transaction and alter the persisted representation of the entity.
|
|
||||||
private static ThreadLocal<Boolean> inSerializationMode = ThreadLocal.withInitial(() -> false);
|
|
||||||
|
|
||||||
private transient ImmutableList<Mutation> mutations;
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
public ImmutableList<Mutation> getMutations() {
|
|
||||||
return mutations;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Write the entire transaction to the datastore in a datastore transaction. */
|
|
||||||
public void writeToDatastore() {
|
|
||||||
ofyTm()
|
|
||||||
.transact(
|
|
||||||
() -> {
|
|
||||||
for (Mutation mutation : mutations) {
|
|
||||||
mutation.writeToDatastore();
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Serialize a transaction to a byte array. */
|
|
||||||
public byte[] serialize() {
|
|
||||||
try {
|
|
||||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
|
||||||
ObjectOutputStream out = new ObjectOutputStream(baos);
|
|
||||||
|
|
||||||
// Write the transaction version id. This serves as both a version id and a "magic number" to
|
|
||||||
// protect us against trying to deserialize some random byte array.
|
|
||||||
out.writeInt(VERSION_ID);
|
|
||||||
|
|
||||||
// Write all of the mutations, preceded by their count.
|
|
||||||
out.writeInt(mutations.size());
|
|
||||||
try {
|
|
||||||
inSerializationMode.set(true);
|
|
||||||
for (Mutation mutation : mutations) {
|
|
||||||
mutation.serializeTo(out);
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
inSerializationMode.set(false);
|
|
||||||
}
|
|
||||||
|
|
||||||
out.close();
|
|
||||||
return baos.toByteArray();
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new IllegalArgumentException();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Transaction deserialize(byte[] serializedTransaction) throws IOException {
|
|
||||||
ObjectInputStream in =
|
|
||||||
new LenientObjectInputStream(new ByteArrayInputStream(serializedTransaction));
|
|
||||||
|
|
||||||
// Verify that the data is what we expect.
|
|
||||||
int version = in.readInt();
|
|
||||||
checkArgument(
|
|
||||||
version == VERSION_ID, "Invalid version id. Expected %s but got %s", VERSION_ID, version);
|
|
||||||
|
|
||||||
Transaction.Builder builder = new Transaction.Builder();
|
|
||||||
int mutationCount = in.readInt();
|
|
||||||
for (int i = 0; i < mutationCount; ++i) {
|
|
||||||
try {
|
|
||||||
builder.add(Mutation.deserializeFrom(in));
|
|
||||||
} catch (EOFException e) {
|
|
||||||
throw new RuntimeException("Serialized transaction terminated prematurely", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (in.read() != -1) {
|
|
||||||
throw new RuntimeException("Unread data at the end of a serialized transaction.");
|
|
||||||
}
|
|
||||||
return builder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns true if the transaction contains no mutations. */
|
|
||||||
public boolean isEmpty() {
|
|
||||||
return mutations.isEmpty();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns true if we are serializing a transaction in the current thread.
|
|
||||||
*
|
|
||||||
* <p>This should be checked by any Ofy translators prior to making any changes to an entity's
|
|
||||||
* state representation based on the assumption that we are currently persisting the entity to
|
|
||||||
* datastore.
|
|
||||||
*/
|
|
||||||
public static boolean inSerializationMode() {
|
|
||||||
return inSerializationMode.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Builder asBuilder() {
|
|
||||||
return new Builder(clone(this));
|
|
||||||
}
|
|
||||||
|
|
||||||
public final TransactionEntity toEntity() {
|
|
||||||
return new TransactionEntity(serialize());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class Builder extends GenericBuilder<Transaction, Builder> {
|
|
||||||
|
|
||||||
ImmutableList.Builder<Mutation> listBuilder = new ImmutableList.Builder<>();
|
|
||||||
|
|
||||||
Builder() {}
|
|
||||||
|
|
||||||
protected Builder(Transaction instance) {
|
|
||||||
super(instance);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder addUpdate(Object entity) {
|
|
||||||
checkNotNull(entity);
|
|
||||||
listBuilder.add(new Update(entity));
|
|
||||||
return thisCastToDerived();
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder addDelete(VKey<?> key) {
|
|
||||||
checkNotNull(key);
|
|
||||||
listBuilder.add(new Delete(key));
|
|
||||||
return thisCastToDerived();
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Adds a mutation (mainly intended for serialization) */
|
|
||||||
Builder add(Mutation mutation) {
|
|
||||||
checkNotNull(mutation);
|
|
||||||
listBuilder.add(mutation);
|
|
||||||
return thisCastToDerived();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Transaction build() {
|
|
||||||
getInstance().mutations = listBuilder.build();
|
|
||||||
return super.build();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Base class for database record mutations. */
|
|
||||||
public abstract static class Mutation {
|
|
||||||
|
|
||||||
enum Type {
|
|
||||||
UPDATE,
|
|
||||||
DELETE
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Write the changes in the mutation to the datastore. */
|
|
||||||
public abstract void writeToDatastore();
|
|
||||||
|
|
||||||
/** Serialize the mutation to the output stream. */
|
|
||||||
public abstract void serializeTo(ObjectOutputStream out) throws IOException;
|
|
||||||
|
|
||||||
/** Deserialize a mutation from the input stream. */
|
|
||||||
public static Mutation deserializeFrom(ObjectInputStream in) throws IOException {
|
|
||||||
try {
|
|
||||||
Type type = (Type) in.readObject();
|
|
||||||
switch (type) {
|
|
||||||
case UPDATE:
|
|
||||||
return Update.deserializeFrom(in);
|
|
||||||
case DELETE:
|
|
||||||
return Delete.deserializeFrom(in);
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException("Unknown enum value: " + type);
|
|
||||||
}
|
|
||||||
} catch (ClassNotFoundException e) {
|
|
||||||
throw new IllegalArgumentException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Record update.
|
|
||||||
*
|
|
||||||
* <p>Note that we don't have to distinguish between add and update, since this is for replay into
|
|
||||||
* the datastore which makes no such distinction.
|
|
||||||
*
|
|
||||||
* <p>Update serializes its entity using Objectify serialization.
|
|
||||||
*/
|
|
||||||
public static class Update extends Mutation {
|
|
||||||
private Object entity;
|
|
||||||
|
|
||||||
Update(Object entity) {
|
|
||||||
this.entity =
|
|
||||||
(entity instanceof SqlEntity) ? ((SqlEntity) entity).toDatastoreEntity().get() : entity;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void writeToDatastore() {
|
|
||||||
ofyTm().putIgnoringReadOnlyWithBackup(entity);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void serializeTo(ObjectOutputStream out) throws IOException {
|
|
||||||
out.writeObject(Type.UPDATE);
|
|
||||||
Entity realEntity = auditedOfy().toEntity(entity);
|
|
||||||
EntityProto proto = EntityTranslator.convertToPb(realEntity);
|
|
||||||
out.write(VERSION_ID);
|
|
||||||
proto.writeDelimitedTo(out);
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
public Object getEntity() {
|
|
||||||
return entity;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Update deserializeFrom(ObjectInputStream in) throws IOException {
|
|
||||||
EntityProto proto = new EntityProto();
|
|
||||||
proto.parseDelimitedFrom(in);
|
|
||||||
return new Update(auditedOfy().toPojo(EntityTranslator.createFromPb(proto)));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Record deletion.
|
|
||||||
*
|
|
||||||
* <p>Delete serializes its VKey using Java native serialization.
|
|
||||||
*/
|
|
||||||
public static class Delete extends Mutation {
|
|
||||||
private final VKey<?> key;
|
|
||||||
|
|
||||||
Delete(VKey<?> key) {
|
|
||||||
this.key = key;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void writeToDatastore() {
|
|
||||||
ofyTm().deleteIgnoringReadOnlyWithBackup(key);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void serializeTo(ObjectOutputStream out) throws IOException {
|
|
||||||
out.writeObject(Type.DELETE);
|
|
||||||
|
|
||||||
// Java object serialization works for this.
|
|
||||||
out.writeObject(key);
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
public VKey<?> getKey() {
|
|
||||||
return key;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Delete deserializeFrom(ObjectInputStream in) throws IOException {
|
|
||||||
try {
|
|
||||||
return new Delete((VKey<?>) in.readObject());
|
|
||||||
} catch (ClassNotFoundException e) {
|
|
||||||
throw new IllegalArgumentException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* ObjectInputStream that ignores the UIDs of serialized objects.
|
|
||||||
*
|
|
||||||
* <p>We only really need to deserialize VKeys. However, VKeys have a class object associated with
|
|
||||||
* them, and if the class is changed and we haven't defined a serialVersionUID for it, we get an
|
|
||||||
* exception during deserialization.
|
|
||||||
*
|
|
||||||
* <p>It's safe for us to ignore this condition: we only care about attaching the correct local
|
|
||||||
* class object to the VKey. So this class effectively does so by replacing the class descriptor
|
|
||||||
* if it's version UID doesn't match that of the local class.
|
|
||||||
*/
|
|
||||||
private static class LenientObjectInputStream extends ObjectInputStream {
|
|
||||||
|
|
||||||
public LenientObjectInputStream(InputStream in) throws IOException {
|
|
||||||
super(in);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected ObjectStreamClass readClassDescriptor() throws IOException, ClassNotFoundException {
|
|
||||||
ObjectStreamClass persistedDescriptor = super.readClassDescriptor();
|
|
||||||
Class localClass = Class.forName(persistedDescriptor.getName());
|
|
||||||
ObjectStreamClass localDescriptor = ObjectStreamClass.lookup(localClass);
|
|
||||||
if (localDescriptor != null) {
|
|
||||||
if (persistedDescriptor.getSerialVersionUID() != localDescriptor.getSerialVersionUID()) {
|
|
||||||
return localDescriptor;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return persistedDescriptor;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -16,7 +16,7 @@ package google.registry.persistence.transaction;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.replay.SqlOnlyEntity;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import javax.persistence.Entity;
|
import javax.persistence.Entity;
|
||||||
import javax.persistence.GeneratedValue;
|
import javax.persistence.GeneratedValue;
|
||||||
import javax.persistence.GenerationType;
|
import javax.persistence.GenerationType;
|
||||||
|
@ -30,7 +30,8 @@ import javax.persistence.Table;
|
||||||
*/
|
*/
|
||||||
@Entity
|
@Entity
|
||||||
@Table(name = "Transaction")
|
@Table(name = "Transaction")
|
||||||
public class TransactionEntity extends ImmutableObject implements SqlOnlyEntity {
|
@DeleteAfterMigration
|
||||||
|
public class TransactionEntity extends ImmutableObject {
|
||||||
|
|
||||||
@Id
|
@Id
|
||||||
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
||||||
|
|
|
@ -311,10 +311,4 @@ public interface TransactionManager {
|
||||||
|
|
||||||
/** Returns true if the transaction manager is DatastoreTransactionManager, false otherwise. */
|
/** Returns true if the transaction manager is DatastoreTransactionManager, false otherwise. */
|
||||||
boolean isOfy();
|
boolean isOfy();
|
||||||
|
|
||||||
/** Performs the write ignoring any read-only restrictions or backup, for use only in replay. */
|
|
||||||
void putIgnoringReadOnlyWithoutBackup(Object entity);
|
|
||||||
|
|
||||||
/** Performs the delete ignoring any read-only restrictions or backup, for use only in replay. */
|
|
||||||
void deleteIgnoringReadOnlyWithoutBackup(VKey<?> key);
|
|
||||||
}
|
}
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue