mirror of
https://github.com/google/nomulus.git
synced 2025-05-15 00:47:11 +02:00
Reverse dependency between /flows/ and /batch/
Certain flows need to launch batched jobs. Logically this would mean that flows depend on batch. However, the current state of dependency was the other way around, and the reason for that was ResourceFlowUtils.java that had in it some utility functions that weren't used in the flows and were needed in the batch jobs. This CL removes these utility functions from the /flows/ directory, letting us reverse the dependency edge between flows/ and batch/ Part of this was moving the flows/async/ code into batch/ - which also makes sense because flows/async/ just "enqueued" tasks that would then be run by actions in batch/ It makes sense that the code that enqueues the tasks and the code that dequeues the tasks sit in the same library. ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=228698761
This commit is contained in:
parent
d0c73efac0
commit
898448b8a0
42 changed files with 419 additions and 423 deletions
|
@ -12,7 +12,7 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package google.registry.flows.async;
|
||||
package google.registry.batch;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static google.registry.util.DateTimeUtils.isBeforeOrAt;
|
||||
|
@ -39,7 +39,7 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.Duration;
|
||||
|
||||
/** Helper class to enqueue tasks for handling asynchronous operations in flows. */
|
||||
public final class AsyncFlowEnqueuer {
|
||||
public final class AsyncTaskEnqueuer {
|
||||
|
||||
/** The HTTP parameter names used by async flows. */
|
||||
public static final String PARAM_RESOURCE_KEY = "resourceKey";
|
||||
|
@ -70,7 +70,7 @@ public final class AsyncFlowEnqueuer {
|
|||
|
||||
@VisibleForTesting
|
||||
@Inject
|
||||
public AsyncFlowEnqueuer(
|
||||
public AsyncTaskEnqueuer(
|
||||
@Named(QUEUE_ASYNC_ACTIONS) Queue asyncActionsPushQueue,
|
||||
@Named(QUEUE_ASYNC_DELETE) Queue asyncDeletePullQueue,
|
||||
@Named(QUEUE_ASYNC_HOST_RENAME) Queue asyncDnsRefreshPullQueue,
|
|
@ -12,12 +12,12 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package google.registry.flows.async;
|
||||
package google.registry.batch;
|
||||
|
||||
import static com.google.appengine.api.taskqueue.QueueConstants.maxLeaseCount;
|
||||
import static com.google.monitoring.metrics.EventMetric.DEFAULT_FITTER;
|
||||
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.CONTACT_AND_HOST_DELETE;
|
||||
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.DNS_REFRESH;
|
||||
import static google.registry.batch.AsyncTaskMetrics.OperationType.CONTACT_AND_HOST_DELETE;
|
||||
import static google.registry.batch.AsyncTaskMetrics.OperationType.DNS_REFRESH;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
@ -36,16 +36,16 @@ import org.joda.time.Duration;
|
|||
/**
|
||||
* Instrumentation for async flows (contact/host deletion and DNS refreshes).
|
||||
*
|
||||
* @see AsyncFlowEnqueuer
|
||||
* @see AsyncTaskEnqueuer
|
||||
*/
|
||||
public class AsyncFlowMetrics {
|
||||
public class AsyncTaskMetrics {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
private final Clock clock;
|
||||
|
||||
@Inject
|
||||
public AsyncFlowMetrics(Clock clock) {
|
||||
public AsyncTaskMetrics(Clock clock) {
|
||||
this.clock = clock;
|
||||
}
|
||||
|
|
@ -8,21 +8,16 @@ java_library(
|
|||
name = "batch",
|
||||
srcs = glob(["*.java"]),
|
||||
deps = [
|
||||
"//java/google/registry/bigquery",
|
||||
"//java/google/registry/config",
|
||||
"//java/google/registry/dns",
|
||||
"//java/google/registry/flows",
|
||||
"//java/google/registry/mapreduce",
|
||||
"//java/google/registry/mapreduce/inputs",
|
||||
"//java/google/registry/model",
|
||||
"//java/google/registry/monitoring/whitebox",
|
||||
"//java/google/registry/pricing",
|
||||
"//java/google/registry/request",
|
||||
"//java/google/registry/request:modules",
|
||||
"//java/google/registry/request/auth",
|
||||
"//java/google/registry/util",
|
||||
"//third_party/objectify:objectify-v4_1",
|
||||
"@com_google_apis_google_api_services_bigquery",
|
||||
"@com_google_appengine_api_1_0_sdk",
|
||||
"@com_google_appengine_tools_appengine_gcs_client",
|
||||
"@com_google_appengine_tools_appengine_mapreduce",
|
||||
|
@ -35,6 +30,7 @@ java_library(
|
|||
"@com_google_flogger_system_backend",
|
||||
"@com_google_guava",
|
||||
"@com_google_http_client",
|
||||
"@com_google_monitoring_client_metrics",
|
||||
"@javax_servlet_api",
|
||||
"@joda_time",
|
||||
"@org_joda_money",
|
||||
|
|
|
@ -14,9 +14,13 @@
|
|||
|
||||
package google.registry.batch;
|
||||
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESAVE_TIMES;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESAVE_TIMES;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.request.RequestParameters.extractOptionalBooleanParameter;
|
||||
import static google.registry.request.RequestParameters.extractOptionalIntParameter;
|
||||
import static google.registry.request.RequestParameters.extractOptionalParameter;
|
||||
|
@ -24,6 +28,7 @@ import static google.registry.request.RequestParameters.extractRequiredDatetimeP
|
|||
import static google.registry.request.RequestParameters.extractRequiredParameter;
|
||||
import static google.registry.request.RequestParameters.extractSetOfDatetimeParameters;
|
||||
|
||||
import com.google.appengine.api.taskqueue.Queue;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import dagger.Module;
|
||||
|
@ -31,6 +36,7 @@ import dagger.Provides;
|
|||
import google.registry.model.ImmutableObject;
|
||||
import google.registry.request.Parameter;
|
||||
import java.util.Optional;
|
||||
import javax.inject.Named;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
|
@ -87,4 +93,22 @@ public class BatchModule {
|
|||
static ImmutableSet<DateTime> provideResaveTimes(HttpServletRequest req) {
|
||||
return extractSetOfDatetimeParameters(req, PARAM_RESAVE_TIMES);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Named(QUEUE_ASYNC_ACTIONS)
|
||||
static Queue provideAsyncActionsPushQueue() {
|
||||
return getQueue(QUEUE_ASYNC_ACTIONS);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Named(QUEUE_ASYNC_DELETE)
|
||||
static Queue provideAsyncDeletePullQueue() {
|
||||
return getQueue(QUEUE_ASYNC_DELETE);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Named(QUEUE_ASYNC_HOST_RENAME)
|
||||
static Queue provideAsyncHostRenamePullQueue() {
|
||||
return getQueue(QUEUE_ASYNC_HOST_RENAME);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,18 +21,18 @@ import static com.google.common.base.Preconditions.checkState;
|
|||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.google.common.math.IntMath.divide;
|
||||
import static com.googlecode.objectify.Key.getKind;
|
||||
import static google.registry.flows.ResourceFlowUtils.denyPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.handlePendingTransferOnDelete;
|
||||
import static google.registry.flows.ResourceFlowUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_CLIENT_TRANSACTION_ID;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_IS_SUPERUSER;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTING_CLIENT_ID;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_SERVER_TRANSACTION_ID;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_CLIENT_TRANSACTION_ID;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_IS_SUPERUSER;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTING_CLIENT_ID;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_SERVER_TRANSACTION_ID;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.model.EppResourceUtils.isActive;
|
||||
import static google.registry.model.EppResourceUtils.isDeleted;
|
||||
import static google.registry.model.ResourceTransferUtils.denyPendingTransfer;
|
||||
import static google.registry.model.ResourceTransferUtils.handlePendingTransferOnDelete;
|
||||
import static google.registry.model.ResourceTransferUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.model.eppcommon.StatusValue.PENDING_DELETE;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.model.reporting.HistoryEntry.Type.CONTACT_DELETE;
|
||||
|
@ -63,11 +63,10 @@ import com.google.common.collect.Iterators;
|
|||
import com.google.common.collect.Multiset;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskMetrics.OperationResult;
|
||||
import google.registry.batch.AsyncTaskMetrics.OperationType;
|
||||
import google.registry.batch.DeleteContactsAndHostsAction.DeletionResult.Type;
|
||||
import google.registry.dns.DnsQueue;
|
||||
import google.registry.flows.async.AsyncFlowMetrics;
|
||||
import google.registry.flows.async.AsyncFlowMetrics.OperationResult;
|
||||
import google.registry.flows.async.AsyncFlowMetrics.OperationType;
|
||||
import google.registry.mapreduce.MapreduceRunner;
|
||||
import google.registry.mapreduce.UnlockerOutput;
|
||||
import google.registry.mapreduce.inputs.EppResourceInputs;
|
||||
|
@ -124,7 +123,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
private static final int MAX_REDUCE_SHARDS = 50;
|
||||
private static final int DELETES_PER_SHARD = 5;
|
||||
|
||||
@Inject AsyncFlowMetrics asyncFlowMetrics;
|
||||
@Inject AsyncTaskMetrics asyncTaskMetrics;
|
||||
@Inject Clock clock;
|
||||
@Inject MapreduceRunner mrRunner;
|
||||
@Inject @Named(QUEUE_ASYNC_DELETE) Queue queue;
|
||||
|
@ -154,7 +153,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
LeaseOptions.Builder.withCountLimit(maxLeaseCount())
|
||||
.leasePeriod(LEASE_LENGTH.getStandardSeconds(), SECONDS);
|
||||
List<TaskHandle> tasks = queue.leaseTasks(options);
|
||||
asyncFlowMetrics.recordContactHostDeletionBatchSize(tasks.size());
|
||||
asyncTaskMetrics.recordContactHostDeletionBatchSize(tasks.size());
|
||||
|
||||
// Check if there are no tasks to process, and if so, return early.
|
||||
if (tasks.isEmpty()) {
|
||||
|
@ -210,7 +209,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
retrier.callWithRetry(() -> queue.deleteTask(tasks), TransientFailureException.class);
|
||||
deletionRequests.forEach(
|
||||
deletionRequest ->
|
||||
asyncFlowMetrics.recordAsyncFlowResult(
|
||||
asyncTaskMetrics.recordAsyncFlowResult(
|
||||
deletionRequest.getMetricOperationType(),
|
||||
OperationResult.STALE,
|
||||
deletionRequest.requestedTime()));
|
||||
|
@ -306,7 +305,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
private static final DnsQueue dnsQueue = DnsQueue.create();
|
||||
|
||||
@NonFinalForTesting
|
||||
private static AsyncFlowMetrics asyncFlowMetrics = new AsyncFlowMetrics(new SystemClock());
|
||||
private static AsyncTaskMetrics asyncTaskMetrics = new AsyncTaskMetrics(new SystemClock());
|
||||
|
||||
@Override
|
||||
public void reduce(final DeletionRequest deletionRequest, ReducerInput<Boolean> values) {
|
||||
|
@ -321,7 +320,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
getQueue(QUEUE_ASYNC_DELETE).deleteTask(deletionRequest.task());
|
||||
return deletionResult;
|
||||
});
|
||||
asyncFlowMetrics.recordAsyncFlowResult(
|
||||
asyncTaskMetrics.recordAsyncFlowResult(
|
||||
deletionRequest.getMetricOperationType(),
|
||||
result.getMetricOperationResult(),
|
||||
deletionRequest.requestedTime());
|
||||
|
|
|
@ -17,8 +17,8 @@ package google.registry.batch;
|
|||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static com.google.common.base.Preconditions.checkState;
|
||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
import static google.registry.flows.ResourceFlowUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.mapreduce.MapreduceRunner.PARAM_DRY_RUN;
|
||||
import static google.registry.model.ResourceTransferUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.model.registry.Registries.getTldsOfType;
|
||||
import static google.registry.model.reporting.HistoryEntry.Type.DOMAIN_DELETE;
|
||||
|
|
|
@ -18,10 +18,10 @@ 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.flows.async.AsyncFlowEnqueuer.PARAM_HOST_KEY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.DNS_REFRESH;
|
||||
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.isActive;
|
||||
import static google.registry.model.EppResourceUtils.isDeleted;
|
||||
|
@ -46,9 +46,8 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskMetrics.OperationResult;
|
||||
import google.registry.dns.DnsQueue;
|
||||
import google.registry.flows.async.AsyncFlowMetrics;
|
||||
import google.registry.flows.async.AsyncFlowMetrics.OperationResult;
|
||||
import google.registry.mapreduce.MapreduceRunner;
|
||||
import google.registry.mapreduce.inputs.NullInput;
|
||||
import google.registry.model.domain.DomainResource;
|
||||
|
@ -83,7 +82,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
private static final Duration LEASE_LENGTH = standardHours(4);
|
||||
|
||||
@Inject AsyncFlowMetrics asyncFlowMetrics;
|
||||
@Inject AsyncTaskMetrics asyncTaskMetrics;
|
||||
@Inject Clock clock;
|
||||
@Inject MapreduceRunner mrRunner;
|
||||
@Inject @Named(QUEUE_ASYNC_HOST_RENAME) Queue pullQueue;
|
||||
|
@ -114,7 +113,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
LeaseOptions.Builder.withCountLimit(maxLeaseCount())
|
||||
.leasePeriod(LEASE_LENGTH.getStandardSeconds(), SECONDS);
|
||||
List<TaskHandle> tasks = pullQueue.leaseTasks(options);
|
||||
asyncFlowMetrics.recordDnsRefreshBatchSize(tasks.size());
|
||||
asyncTaskMetrics.recordDnsRefreshBatchSize(tasks.size());
|
||||
|
||||
// Check if there are no tasks to process, and if so, return early.
|
||||
if (tasks.isEmpty()) {
|
||||
|
@ -146,7 +145,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
}
|
||||
|
||||
deleteTasksWithRetry(
|
||||
requestsToDelete, pullQueue, asyncFlowMetrics, retrier, OperationResult.STALE);
|
||||
requestsToDelete, pullQueue, asyncTaskMetrics, retrier, OperationResult.STALE);
|
||||
ImmutableList<DnsRefreshRequest> refreshRequests = requestsBuilder.build();
|
||||
if (refreshRequests.isEmpty()) {
|
||||
logRespondAndUnlock(
|
||||
|
@ -245,7 +244,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
private static final long serialVersionUID = 9077366205249562118L;
|
||||
|
||||
@NonFinalForTesting
|
||||
private static AsyncFlowMetrics asyncFlowMetrics = new AsyncFlowMetrics(new SystemClock());
|
||||
private static AsyncTaskMetrics asyncTaskMetrics = new AsyncTaskMetrics(new SystemClock());
|
||||
|
||||
private final Lock lock;
|
||||
private final Retrier retrier;
|
||||
|
@ -265,7 +264,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
deleteTasksWithRetry(
|
||||
refreshRequests,
|
||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||
asyncFlowMetrics,
|
||||
asyncTaskMetrics,
|
||||
retrier,
|
||||
OperationResult.SUCCESS);
|
||||
|
||||
|
@ -277,7 +276,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
private static void deleteTasksWithRetry(
|
||||
final List<DnsRefreshRequest> refreshRequests,
|
||||
final Queue queue,
|
||||
AsyncFlowMetrics asyncFlowMetrics,
|
||||
AsyncTaskMetrics asyncTaskMetrics,
|
||||
Retrier retrier,
|
||||
OperationResult result) {
|
||||
if (refreshRequests.isEmpty()) {
|
||||
|
@ -287,7 +286,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
refreshRequests.stream().map(DnsRefreshRequest::task).collect(toImmutableList());
|
||||
retrier.callWithRetry(() -> queue.deleteTask(tasks), TransientFailureException.class);
|
||||
refreshRequests.forEach(
|
||||
r -> asyncFlowMetrics.recordAsyncFlowResult(DNS_REFRESH, result, r.requestedTime()));
|
||||
r -> asyncTaskMetrics.recordAsyncFlowResult(DNS_REFRESH, result, r.requestedTime()));
|
||||
}
|
||||
|
||||
/** A class that encapsulates the values of a request to refresh DNS for a renamed host. */
|
||||
|
|
|
@ -14,16 +14,15 @@
|
|||
|
||||
package google.registry.batch;
|
||||
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESAVE_TIMES;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESAVE_TIMES;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.ImmutableObject;
|
||||
import google.registry.request.Action;
|
||||
|
@ -39,15 +38,17 @@ import org.joda.time.DateTime;
|
|||
*
|
||||
* <p>{@link EppResource}s will be projected forward to the current time.
|
||||
*/
|
||||
@Action(path = "/_dr/task/resaveEntity", auth = Auth.AUTH_INTERNAL_OR_ADMIN, method = Method.POST)
|
||||
@Action(path = ResaveEntityAction.PATH, auth = Auth.AUTH_INTERNAL_OR_ADMIN, method = Method.POST)
|
||||
public class ResaveEntityAction implements Runnable {
|
||||
|
||||
public static final String PATH = "/_dr/task/resaveEntity";
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
private final Key<ImmutableObject> resourceKey;
|
||||
private final DateTime requestedTime;
|
||||
private final ImmutableSortedSet<DateTime> resaveTimes;
|
||||
private final AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
private final AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
private final Response response;
|
||||
|
||||
@Inject
|
||||
|
@ -55,12 +56,12 @@ public class ResaveEntityAction implements Runnable {
|
|||
@Parameter(PARAM_RESOURCE_KEY) Key<ImmutableObject> resourceKey,
|
||||
@Parameter(PARAM_REQUESTED_TIME) DateTime requestedTime,
|
||||
@Parameter(PARAM_RESAVE_TIMES) ImmutableSet<DateTime> resaveTimes,
|
||||
AsyncFlowEnqueuer asyncFlowEnqueuer,
|
||||
AsyncTaskEnqueuer asyncTaskEnqueuer,
|
||||
Response response) {
|
||||
this.resourceKey = resourceKey;
|
||||
this.requestedTime = requestedTime;
|
||||
this.resaveTimes = ImmutableSortedSet.copyOf(resaveTimes);
|
||||
this.asyncFlowEnqueuer = asyncFlowEnqueuer;
|
||||
this.asyncTaskEnqueuer = asyncTaskEnqueuer;
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
|
@ -75,7 +76,7 @@ public class ResaveEntityAction implements Runnable {
|
|||
? ((EppResource) entity).cloneProjectedAtTime(ofy().getTransactionTime()) : entity
|
||||
);
|
||||
if (!resaveTimes.isEmpty()) {
|
||||
asyncFlowEnqueuer.enqueueAsyncResave(entity, requestedTime, resaveTimes);
|
||||
asyncTaskEnqueuer.enqueueAsyncResave(entity, requestedTime, resaveTimes);
|
||||
}
|
||||
});
|
||||
response.setPayload("Entity re-saved.");
|
||||
|
|
|
@ -1076,7 +1076,7 @@ public final class RegistryConfig {
|
|||
* duration should also be longer than that cache duration (eppResourceCachingSeconds).
|
||||
*
|
||||
* @see google.registry.config.RegistryConfigSettings.Caching
|
||||
* @see google.registry.flows.async.AsyncFlowEnqueuer
|
||||
* @see google.registry.batch.AsyncTaskEnqueuer
|
||||
*/
|
||||
@Provides
|
||||
@Config("asyncDeleteFlowMapreduceDelay")
|
||||
|
|
|
@ -23,6 +23,7 @@ java_library(
|
|||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
":soy_java_wrappers",
|
||||
"//java/google/registry/batch",
|
||||
"//java/google/registry/config",
|
||||
"//java/google/registry/dns",
|
||||
"//java/google/registry/model",
|
||||
|
|
|
@ -30,12 +30,14 @@ import javax.servlet.http.HttpServletRequest;
|
|||
|
||||
/** Runs EPP commands directly without logging in, verifying an XSRF token from the tool. */
|
||||
@Action(
|
||||
path = "/_dr/epptool",
|
||||
path = EppToolAction.PATH,
|
||||
method = Method.POST,
|
||||
auth = Auth.AUTH_INTERNAL_OR_ADMIN
|
||||
)
|
||||
public class EppToolAction implements Runnable {
|
||||
|
||||
public static final String PATH = "/_dr/epptool";
|
||||
|
||||
@Inject @Parameter("clientId") String clientId;
|
||||
@Inject @Parameter("superuser") boolean isSuperuser;
|
||||
@Inject @Parameter("dryRun") boolean isDryRun;
|
||||
|
|
|
@ -17,8 +17,8 @@ package google.registry.flows;
|
|||
import dagger.Module;
|
||||
import dagger.Provides;
|
||||
import dagger.Subcomponent;
|
||||
import google.registry.batch.BatchModule;
|
||||
import google.registry.dns.DnsModule;
|
||||
import google.registry.flows.async.AsyncFlowsModule;
|
||||
import google.registry.flows.contact.ContactCheckFlow;
|
||||
import google.registry.flows.contact.ContactCreateFlow;
|
||||
import google.registry.flows.contact.ContactDeleteFlow;
|
||||
|
@ -65,7 +65,7 @@ import google.registry.model.eppcommon.Trid;
|
|||
@FlowScope
|
||||
@Subcomponent(modules = {
|
||||
AllocationTokenModule.class,
|
||||
AsyncFlowsModule.class,
|
||||
BatchModule.class,
|
||||
CustomLogicModule.class,
|
||||
DnsModule.class,
|
||||
FlowModule.class,
|
||||
|
|
|
@ -14,17 +14,12 @@
|
|||
|
||||
package google.registry.flows;
|
||||
|
||||
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 com.google.common.collect.Sets.intersection;
|
||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||
import static google.registry.model.EppResourceUtils.queryForLinkedDomains;
|
||||
import static google.registry.model.domain.DomainResource.extendRegistrationWithCap;
|
||||
import static google.registry.model.index.ForeignKeyIndex.loadAndGetKey;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.googlecode.objectify.Key;
|
||||
|
@ -41,26 +36,13 @@ import google.registry.flows.exceptions.ResourceStatusProhibitsOperationExceptio
|
|||
import google.registry.flows.exceptions.ResourceToDeleteIsReferencedException;
|
||||
import google.registry.flows.exceptions.TooManyResourceChecksException;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.EppResource.Builder;
|
||||
import google.registry.model.EppResource.BuilderWithTransferData;
|
||||
import google.registry.model.EppResource.ForeignKeyedEppResource;
|
||||
import google.registry.model.EppResource.ResourceWithTransferData;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.domain.DomainResource;
|
||||
import google.registry.model.eppcommon.AuthInfo;
|
||||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.model.eppcommon.Trid;
|
||||
import google.registry.model.index.ForeignKeyIndex;
|
||||
import google.registry.model.poll.PendingActionNotificationResponse;
|
||||
import google.registry.model.poll.PendingActionNotificationResponse.ContactPendingActionNotificationResponse;
|
||||
import google.registry.model.poll.PendingActionNotificationResponse.DomainPendingActionNotificationResponse;
|
||||
import google.registry.model.poll.PollMessage;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.model.transfer.TransferData;
|
||||
import google.registry.model.transfer.TransferResponse;
|
||||
import google.registry.model.transfer.TransferResponse.ContactTransferResponse;
|
||||
import google.registry.model.transfer.TransferResponse.DomainTransferResponse;
|
||||
import google.registry.model.transfer.TransferStatus;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
@ -74,65 +56,9 @@ public final class ResourceFlowUtils {
|
|||
|
||||
private ResourceFlowUtils() {}
|
||||
|
||||
/** Statuses for which an exDate should be added to transfer responses. */
|
||||
private static final ImmutableSet<TransferStatus> ADD_EXDATE_STATUSES = Sets.immutableEnumSet(
|
||||
TransferStatus.PENDING, TransferStatus.CLIENT_APPROVED, TransferStatus.SERVER_APPROVED);
|
||||
|
||||
/** In {@link #failfastForAsyncDelete}, check this (arbitrary) number of query results. */
|
||||
private static final int FAILFAST_CHECK_COUNT = 5;
|
||||
|
||||
/**
|
||||
* Create a transfer response using the id and type of this resource and the specified
|
||||
* {@link TransferData}.
|
||||
*/
|
||||
public static TransferResponse createTransferResponse(
|
||||
EppResource eppResource, TransferData transferData, DateTime now) {
|
||||
assertIsContactOrDomain(eppResource);
|
||||
TransferResponse.Builder<? extends TransferResponse, ?> builder;
|
||||
if (eppResource instanceof ContactResource) {
|
||||
builder = new ContactTransferResponse.Builder().setContactId(eppResource.getForeignKey());
|
||||
} else {
|
||||
DomainResource domain = (DomainResource) eppResource;
|
||||
builder =
|
||||
new DomainTransferResponse.Builder()
|
||||
.setFullyQualifiedDomainName(eppResource.getForeignKey())
|
||||
// TODO(b/25084229): fix exDate computation logic.
|
||||
.setExtendedRegistrationExpirationTime(
|
||||
ADD_EXDATE_STATUSES.contains(transferData.getTransferStatus())
|
||||
? extendRegistrationWithCap(now, domain.getRegistrationExpirationTime(), 1)
|
||||
: null);
|
||||
}
|
||||
builder.setGainingClientId(transferData.getGainingClientId())
|
||||
.setLosingClientId(transferData.getLosingClientId())
|
||||
.setPendingTransferExpirationTime(transferData.getPendingTransferExpirationTime())
|
||||
.setTransferRequestTime(transferData.getTransferRequestTime())
|
||||
.setTransferStatus(transferData.getTransferStatus());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a pending action notification response indicating the resolution of a transfer.
|
||||
*
|
||||
* <p>The returned object will use the id and type of this resource, the trid of the resource's
|
||||
* last transfer request, and the specified status and date.
|
||||
*/
|
||||
public static PendingActionNotificationResponse createPendingTransferNotificationResponse(
|
||||
EppResource eppResource,
|
||||
Trid transferRequestTrid,
|
||||
boolean actionResult,
|
||||
DateTime processedDate) {
|
||||
assertIsContactOrDomain(eppResource);
|
||||
return eppResource instanceof ContactResource
|
||||
? ContactPendingActionNotificationResponse.create(
|
||||
eppResource.getForeignKey(), actionResult, transferRequestTrid, processedDate)
|
||||
: DomainPendingActionNotificationResponse.create(
|
||||
eppResource.getForeignKey(), actionResult, transferRequestTrid, processedDate);
|
||||
}
|
||||
|
||||
private static void assertIsContactOrDomain(EppResource eppResource) {
|
||||
checkState(eppResource instanceof ContactResource || eppResource instanceof DomainResource);
|
||||
}
|
||||
|
||||
/** Check that the given clientId corresponds to the owner of given resource. */
|
||||
public static void verifyResourceOwnership(String myClientId, EppResource resource)
|
||||
throws EppException {
|
||||
|
@ -141,33 +67,6 @@ public final class ResourceFlowUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/** Update the relevant {@link ForeignKeyIndex} to cache the new deletion time. */
|
||||
public static <R extends EppResource> void updateForeignKeyIndexDeletionTime(R resource) {
|
||||
if (resource instanceof ForeignKeyedEppResource) {
|
||||
ofy().save().entity(ForeignKeyIndex.create(resource, resource.getDeletionTime()));
|
||||
}
|
||||
}
|
||||
|
||||
/** If there is a transfer out, delete the server-approve entities and enqueue a poll message. */
|
||||
public static <R extends EppResource & ResourceWithTransferData>
|
||||
void handlePendingTransferOnDelete(
|
||||
R resource, R newResource, DateTime now, HistoryEntry historyEntry) {
|
||||
if (resource.getStatusValues().contains(StatusValue.PENDING_TRANSFER)) {
|
||||
TransferData oldTransferData = resource.getTransferData();
|
||||
ofy().delete().keys(oldTransferData.getServerApproveEntities());
|
||||
ofy().save().entity(new PollMessage.OneTime.Builder()
|
||||
.setClientId(oldTransferData.getGainingClientId())
|
||||
.setEventTime(now)
|
||||
.setMsg(TransferStatus.SERVER_CANCELLED.getMessage())
|
||||
.setResponseData(ImmutableList.of(
|
||||
createTransferResponse(newResource, newResource.getTransferData(), now),
|
||||
createPendingTransferNotificationResponse(
|
||||
resource, oldTransferData.getTransferRequestTrid(), false, now)))
|
||||
.setParent(historyEntry)
|
||||
.build());
|
||||
}
|
||||
}
|
||||
|
||||
/** Check whether an asynchronous delete would obviously fail, and throw an exception if so. */
|
||||
public static <R extends EppResource> void failfastForAsyncDelete(
|
||||
final String targetId,
|
||||
|
@ -206,71 +105,6 @@ public final class ResourceFlowUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Turn a resource into a builder with its pending transfer resolved.
|
||||
*
|
||||
* <p>This removes the {@link StatusValue#PENDING_TRANSFER} status, sets the {@link
|
||||
* TransferStatus}, clears all the server-approve fields on the {@link TransferData}, and sets the
|
||||
* expiration time of the last pending transfer to now.
|
||||
*/
|
||||
private static <
|
||||
R extends EppResource & ResourceWithTransferData,
|
||||
B extends EppResource.Builder<R, B> & BuilderWithTransferData<B>>
|
||||
B resolvePendingTransfer(R resource, TransferStatus transferStatus, DateTime now) {
|
||||
checkArgument(
|
||||
resource.getStatusValues().contains(StatusValue.PENDING_TRANSFER),
|
||||
"Resource is not in pending transfer status.");
|
||||
checkArgument(
|
||||
!TransferData.EMPTY.equals(resource.getTransferData()),
|
||||
"No old transfer data to resolve.");
|
||||
@SuppressWarnings("unchecked")
|
||||
B builder = (B) resource.asBuilder();
|
||||
return builder
|
||||
.removeStatusValue(StatusValue.PENDING_TRANSFER)
|
||||
.setTransferData(
|
||||
resource.getTransferData().copyConstantFieldsToBuilder()
|
||||
.setTransferStatus(transferStatus)
|
||||
.setPendingTransferExpirationTime(checkNotNull(now))
|
||||
.build());
|
||||
}
|
||||
|
||||
/**
|
||||
* Resolve a pending transfer by awarding it to the gaining client.
|
||||
*
|
||||
* <p>This removes the {@link StatusValue#PENDING_TRANSFER} status, sets the {@link
|
||||
* TransferStatus}, clears all the server-approve fields on the {@link TransferData}, sets the new
|
||||
* client id, and sets the last transfer time and the expiration time of the last pending transfer
|
||||
* to now.
|
||||
*/
|
||||
public static <
|
||||
R extends EppResource & ResourceWithTransferData,
|
||||
B extends Builder<R, B> & BuilderWithTransferData<B>>
|
||||
R approvePendingTransfer(R resource, TransferStatus transferStatus, DateTime now) {
|
||||
checkArgument(transferStatus.isApproved(), "Not an approval transfer status");
|
||||
B builder = ResourceFlowUtils.resolvePendingTransfer(resource, transferStatus, now);
|
||||
return builder
|
||||
.setLastTransferTime(now)
|
||||
.setPersistedCurrentSponsorClientId(resource.getTransferData().getGainingClientId())
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Resolve a pending transfer by denying it.
|
||||
*
|
||||
* <p>This removes the {@link StatusValue#PENDING_TRANSFER} status, sets the {@link
|
||||
* TransferStatus}, clears all the server-approve fields on the {@link TransferData}, sets the
|
||||
* expiration time of the last pending transfer to now, sets the last EPP update time to now, and
|
||||
* sets the last EPP update client id to the given client id.
|
||||
*/
|
||||
public static <R extends EppResource & ResourceWithTransferData> R denyPendingTransfer(
|
||||
R resource, TransferStatus transferStatus, DateTime now, String lastEppUpdateClientId) {
|
||||
checkArgument(transferStatus.isDenied(), "Not a denial transfer status");
|
||||
return resolvePendingTransfer(resource, transferStatus, now)
|
||||
.setLastEppUpdateTime(now)
|
||||
.setLastEppUpdateClientId(lastEppUpdateClientId)
|
||||
.build();
|
||||
}
|
||||
|
||||
public static <R extends EppResource & ResourceWithTransferData> void verifyHasPendingTransfer(
|
||||
R resource) throws NotPendingTransferException {
|
||||
if (resource.getTransferData().getTransferStatus() != TransferStatus.PENDING) {
|
||||
|
|
|
@ -1,48 +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.flows.async;
|
||||
|
||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
|
||||
import com.google.appengine.api.taskqueue.Queue;
|
||||
import dagger.Module;
|
||||
import dagger.Provides;
|
||||
import javax.inject.Named;
|
||||
|
||||
/** Dagger module for the async flows package. */
|
||||
@Module
|
||||
public final class AsyncFlowsModule {
|
||||
|
||||
@Provides
|
||||
@Named(QUEUE_ASYNC_ACTIONS)
|
||||
static Queue provideAsyncActionsPushQueue() {
|
||||
return getQueue(QUEUE_ASYNC_ACTIONS);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Named(QUEUE_ASYNC_DELETE)
|
||||
static Queue provideAsyncDeletePullQueue() {
|
||||
return getQueue(QUEUE_ASYNC_DELETE);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Named(QUEUE_ASYNC_HOST_RENAME)
|
||||
static Queue provideAsyncHostRenamePullQueue() {
|
||||
return getQueue(QUEUE_ASYNC_HOST_RENAME);
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import static google.registry.model.ofy.ObjectifyService.ofy;
|
|||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskEnqueuer;
|
||||
import google.registry.flows.EppException;
|
||||
import google.registry.flows.ExtensionManager;
|
||||
import google.registry.flows.FlowModule.ClientId;
|
||||
|
@ -32,7 +33,6 @@ import google.registry.flows.FlowModule.Superuser;
|
|||
import google.registry.flows.FlowModule.TargetId;
|
||||
import google.registry.flows.TransactionalFlow;
|
||||
import google.registry.flows.annotations.ReportingSpec;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.domain.metadata.MetadataExtension;
|
||||
|
@ -75,7 +75,7 @@ public final class ContactDeleteFlow implements TransactionalFlow {
|
|||
@Inject @Superuser boolean isSuperuser;
|
||||
@Inject Optional<AuthInfo> authInfo;
|
||||
@Inject HistoryEntry.Builder historyBuilder;
|
||||
@Inject AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
@Inject EppResponse.Builder responseBuilder;
|
||||
@Inject ContactDeleteFlow() {}
|
||||
|
||||
|
@ -92,7 +92,7 @@ public final class ContactDeleteFlow implements TransactionalFlow {
|
|||
if (!isSuperuser) {
|
||||
verifyResourceOwnership(clientId, existingContact);
|
||||
}
|
||||
asyncFlowEnqueuer.enqueueAsyncDelete(
|
||||
asyncTaskEnqueuer.enqueueAsyncDelete(
|
||||
existingContact, ofy().getTransactionTime(), clientId, trid, isSuperuser);
|
||||
ContactResource newContact =
|
||||
existingContact.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
|
||||
|
|
|
@ -15,13 +15,13 @@
|
|||
package google.registry.flows.contact;
|
||||
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.approvePendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.loadAndVerifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyHasPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyResourceOwnership;
|
||||
import static google.registry.flows.contact.ContactFlowUtils.createGainingTransferPollMessage;
|
||||
import static google.registry.flows.contact.ContactFlowUtils.createTransferResponse;
|
||||
import static google.registry.model.ResourceTransferUtils.approvePendingTransfer;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
||||
import com.googlecode.objectify.Key;
|
||||
|
|
|
@ -15,13 +15,13 @@
|
|||
package google.registry.flows.contact;
|
||||
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.denyPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.loadAndVerifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyHasPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyTransferInitiator;
|
||||
import static google.registry.flows.contact.ContactFlowUtils.createLosingTransferPollMessage;
|
||||
import static google.registry.flows.contact.ContactFlowUtils.createTransferResponse;
|
||||
import static google.registry.model.ResourceTransferUtils.denyPendingTransfer;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
||||
import com.googlecode.objectify.Key;
|
||||
|
|
|
@ -15,13 +15,13 @@
|
|||
package google.registry.flows.contact;
|
||||
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.denyPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.loadAndVerifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyHasPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyResourceOwnership;
|
||||
import static google.registry.flows.contact.ContactFlowUtils.createGainingTransferPollMessage;
|
||||
import static google.registry.flows.contact.ContactFlowUtils.createTransferResponse;
|
||||
import static google.registry.model.ResourceTransferUtils.denyPendingTransfer;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
||||
import com.googlecode.objectify.Key;
|
||||
|
|
|
@ -15,7 +15,6 @@
|
|||
package google.registry.flows.domain;
|
||||
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyResourceOwnership;
|
||||
|
@ -24,6 +23,7 @@ import static google.registry.flows.domain.DomainFlowUtils.verifyApplicationDoma
|
|||
import static google.registry.flows.domain.DomainFlowUtils.verifyLaunchPhaseMatchesRegistryPhase;
|
||||
import static google.registry.flows.domain.DomainFlowUtils.verifyRegistryStateAllowsApplicationFlows;
|
||||
import static google.registry.model.EppResourceUtils.loadDomainApplication;
|
||||
import static google.registry.model.ResourceTransferUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
||||
import com.googlecode.objectify.Key;
|
||||
|
|
|
@ -17,10 +17,7 @@ package google.registry.flows.domain;
|
|||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
import static google.registry.flows.FlowUtils.persistEntityChanges;
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.denyPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.handlePendingTransferOnDelete;
|
||||
import static google.registry.flows.ResourceFlowUtils.loadAndVerifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyNoDisallowedStatuses;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyResourceOwnership;
|
||||
|
@ -28,6 +25,9 @@ import static google.registry.flows.domain.DomainFlowUtils.checkAllowedAccessToT
|
|||
import static google.registry.flows.domain.DomainFlowUtils.createCancelingRecords;
|
||||
import static google.registry.flows.domain.DomainFlowUtils.updateAutorenewRecurrenceEndTime;
|
||||
import static google.registry.flows.domain.DomainFlowUtils.verifyNotInPredelegation;
|
||||
import static google.registry.model.ResourceTransferUtils.denyPendingTransfer;
|
||||
import static google.registry.model.ResourceTransferUtils.handlePendingTransferOnDelete;
|
||||
import static google.registry.model.ResourceTransferUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.model.eppoutput.Result.Code.SUCCESS;
|
||||
import static google.registry.model.eppoutput.Result.Code.SUCCESS_WITH_ACTION_PENDING;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
@ -42,6 +42,7 @@ import com.google.common.collect.ImmutableSet;
|
|||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskEnqueuer;
|
||||
import google.registry.dns.DnsQueue;
|
||||
import google.registry.flows.EppException;
|
||||
import google.registry.flows.EppException.AssociationProhibitsOperationException;
|
||||
|
@ -52,7 +53,6 @@ import google.registry.flows.FlowModule.TargetId;
|
|||
import google.registry.flows.SessionMetadata;
|
||||
import google.registry.flows.TransactionalFlow;
|
||||
import google.registry.flows.annotations.ReportingSpec;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.flows.custom.DomainDeleteFlowCustomLogic;
|
||||
import google.registry.flows.custom.DomainDeleteFlowCustomLogic.AfterValidationParameters;
|
||||
import google.registry.flows.custom.DomainDeleteFlowCustomLogic.BeforeResponseParameters;
|
||||
|
@ -129,7 +129,7 @@ public final class DomainDeleteFlow implements TransactionalFlow {
|
|||
@Inject HistoryEntry.Builder historyBuilder;
|
||||
@Inject DnsQueue dnsQueue;
|
||||
@Inject Trid trid;
|
||||
@Inject AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
@Inject EppResponse.Builder responseBuilder;
|
||||
@Inject DomainDeleteFlowCustomLogic flowCustomLogic;
|
||||
@Inject DomainDeleteFlow() {}
|
||||
|
@ -188,7 +188,7 @@ public final class DomainDeleteFlow implements TransactionalFlow {
|
|||
PollMessage.OneTime deletePollMessage =
|
||||
createDeletePollMessage(existingDomain, historyEntry, deletionTime);
|
||||
entitiesToSave.add(deletePollMessage);
|
||||
asyncFlowEnqueuer.enqueueAsyncResave(
|
||||
asyncTaskEnqueuer.enqueueAsyncResave(
|
||||
existingDomain, now, ImmutableSortedSet.of(redemptionTime, deletionTime));
|
||||
builder.setDeletionTime(deletionTime)
|
||||
.setStatusValues(ImmutableSet.of(StatusValue.PENDING_DELETE))
|
||||
|
|
|
@ -16,7 +16,6 @@ package google.registry.flows.domain;
|
|||
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.loadAndVerifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyResourceOwnership;
|
||||
import static google.registry.flows.domain.DomainFlowUtils.checkAllowedAccessToTld;
|
||||
|
@ -26,6 +25,7 @@ import static google.registry.flows.domain.DomainFlowUtils.validateFeeChallenge;
|
|||
import static google.registry.flows.domain.DomainFlowUtils.verifyNotReserved;
|
||||
import static google.registry.flows.domain.DomainFlowUtils.verifyPremiumNameIsNotBlocked;
|
||||
import static google.registry.flows.domain.DomainFlowUtils.verifyRegistrarIsActive;
|
||||
import static google.registry.model.ResourceTransferUtils.updateForeignKeyIndexDeletionTime;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||
|
||||
|
|
|
@ -16,7 +16,6 @@ package google.registry.flows.domain;
|
|||
|
||||
import static com.google.common.collect.Iterables.getOnlyElement;
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.approvePendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.loadAndVerifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyHasPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
|
@ -26,6 +25,7 @@ import static google.registry.flows.domain.DomainFlowUtils.createCancelingRecord
|
|||
import static google.registry.flows.domain.DomainFlowUtils.updateAutorenewRecurrenceEndTime;
|
||||
import static google.registry.flows.domain.DomainTransferUtils.createGainingTransferPollMessage;
|
||||
import static google.registry.flows.domain.DomainTransferUtils.createTransferResponse;
|
||||
import static google.registry.model.ResourceTransferUtils.approvePendingTransfer;
|
||||
import static google.registry.model.domain.DomainResource.extendRegistrationWithCap;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.model.reporting.DomainTransactionRecord.TransactionReportField.TRANSFER_SUCCESSFUL;
|
||||
|
|
|
@ -15,7 +15,6 @@
|
|||
package google.registry.flows.domain;
|
||||
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.denyPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.loadAndVerifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyHasPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
|
@ -25,6 +24,7 @@ import static google.registry.flows.domain.DomainFlowUtils.createCancelingRecord
|
|||
import static google.registry.flows.domain.DomainFlowUtils.updateAutorenewRecurrenceEndTime;
|
||||
import static google.registry.flows.domain.DomainTransferUtils.createLosingTransferPollMessage;
|
||||
import static google.registry.flows.domain.DomainTransferUtils.createTransferResponse;
|
||||
import static google.registry.model.ResourceTransferUtils.denyPendingTransfer;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.model.reporting.DomainTransactionRecord.TransactionReportField.TRANSFER_SUCCESSFUL;
|
||||
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||
|
|
|
@ -15,7 +15,6 @@
|
|||
package google.registry.flows.domain;
|
||||
|
||||
import static google.registry.flows.FlowUtils.validateClientIsLoggedIn;
|
||||
import static google.registry.flows.ResourceFlowUtils.denyPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.loadAndVerifyExistence;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyHasPendingTransfer;
|
||||
import static google.registry.flows.ResourceFlowUtils.verifyOptionalAuthInfo;
|
||||
|
@ -25,6 +24,7 @@ import static google.registry.flows.domain.DomainFlowUtils.createCancelingRecord
|
|||
import static google.registry.flows.domain.DomainFlowUtils.updateAutorenewRecurrenceEndTime;
|
||||
import static google.registry.flows.domain.DomainTransferUtils.createGainingTransferPollMessage;
|
||||
import static google.registry.flows.domain.DomainTransferUtils.createTransferResponse;
|
||||
import static google.registry.model.ResourceTransferUtils.denyPendingTransfer;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.model.reporting.DomainTransactionRecord.TransactionReportField.TRANSFER_NACKED;
|
||||
import static google.registry.model.reporting.DomainTransactionRecord.TransactionReportField.TRANSFER_SUCCESSFUL;
|
||||
|
|
|
@ -36,6 +36,7 @@ import static google.registry.model.ofy.ObjectifyService.ofy;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskEnqueuer;
|
||||
import google.registry.flows.EppException;
|
||||
import google.registry.flows.ExtensionManager;
|
||||
import google.registry.flows.FlowModule.ClientId;
|
||||
|
@ -43,7 +44,6 @@ import google.registry.flows.FlowModule.Superuser;
|
|||
import google.registry.flows.FlowModule.TargetId;
|
||||
import google.registry.flows.TransactionalFlow;
|
||||
import google.registry.flows.annotations.ReportingSpec;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.flows.exceptions.AlreadyPendingTransferException;
|
||||
import google.registry.flows.exceptions.InvalidTransferPeriodValueException;
|
||||
import google.registry.flows.exceptions.ObjectAlreadySponsoredException;
|
||||
|
@ -128,7 +128,7 @@ public final class DomainTransferRequestFlow implements TransactionalFlow {
|
|||
@Inject @Superuser boolean isSuperuser;
|
||||
@Inject HistoryEntry.Builder historyBuilder;
|
||||
@Inject Trid trid;
|
||||
@Inject AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
@Inject EppResponse.Builder responseBuilder;
|
||||
@Inject DomainPricingLogic pricingLogic;
|
||||
@Inject DomainTransferRequestFlow() {}
|
||||
|
@ -233,7 +233,7 @@ public final class DomainTransferRequestFlow implements TransactionalFlow {
|
|||
.setLastEppUpdateTime(now)
|
||||
.setLastEppUpdateClientId(gainingClientId)
|
||||
.build();
|
||||
asyncFlowEnqueuer.enqueueAsyncResave(newDomain, now, automaticTransferTime);
|
||||
asyncTaskEnqueuer.enqueueAsyncResave(newDomain, now, automaticTransferTime);
|
||||
ofy().save()
|
||||
.entities(new ImmutableSet.Builder<>()
|
||||
.add(newDomain, historyEntry, requestPollMessage)
|
||||
|
|
|
@ -25,6 +25,7 @@ import static google.registry.model.ofy.ObjectifyService.ofy;
|
|||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskEnqueuer;
|
||||
import google.registry.flows.EppException;
|
||||
import google.registry.flows.ExtensionManager;
|
||||
import google.registry.flows.FlowModule.ClientId;
|
||||
|
@ -32,7 +33,6 @@ import google.registry.flows.FlowModule.Superuser;
|
|||
import google.registry.flows.FlowModule.TargetId;
|
||||
import google.registry.flows.TransactionalFlow;
|
||||
import google.registry.flows.annotations.ReportingSpec;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.domain.DomainBase;
|
||||
import google.registry.model.domain.metadata.MetadataExtension;
|
||||
|
@ -76,7 +76,7 @@ public final class HostDeleteFlow implements TransactionalFlow {
|
|||
@Inject Trid trid;
|
||||
@Inject @Superuser boolean isSuperuser;
|
||||
@Inject HistoryEntry.Builder historyBuilder;
|
||||
@Inject AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
@Inject EppResponse.Builder responseBuilder;
|
||||
@Inject HostDeleteFlow() {}
|
||||
|
||||
|
@ -100,7 +100,7 @@ public final class HostDeleteFlow implements TransactionalFlow {
|
|||
: existingHost;
|
||||
verifyResourceOwnership(clientId, owningResource);
|
||||
}
|
||||
asyncFlowEnqueuer.enqueueAsyncDelete(
|
||||
asyncTaskEnqueuer.enqueueAsyncDelete(
|
||||
existingHost, ofy().getTransactionTime(), clientId, trid, isSuperuser);
|
||||
HostResource newHost =
|
||||
existingHost.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
|
||||
|
|
|
@ -32,6 +32,7 @@ import static google.registry.util.CollectionUtils.isNullOrEmpty;
|
|||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskEnqueuer;
|
||||
import google.registry.dns.DnsQueue;
|
||||
import google.registry.flows.EppException;
|
||||
import google.registry.flows.EppException.ObjectAlreadyExistsException;
|
||||
|
@ -43,7 +44,6 @@ import google.registry.flows.FlowModule.Superuser;
|
|||
import google.registry.flows.FlowModule.TargetId;
|
||||
import google.registry.flows.TransactionalFlow;
|
||||
import google.registry.flows.annotations.ReportingSpec;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.flows.exceptions.ResourceHasClientUpdateProhibitedException;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.ImmutableObject;
|
||||
|
@ -116,7 +116,7 @@ public final class HostUpdateFlow implements TransactionalFlow {
|
|||
@Inject @TargetId String targetId;
|
||||
@Inject @Superuser boolean isSuperuser;
|
||||
@Inject HistoryEntry.Builder historyBuilder;
|
||||
@Inject AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
@Inject DnsQueue dnsQueue;
|
||||
@Inject EppResponse.Builder responseBuilder;
|
||||
@Inject HostUpdateFlow() {}
|
||||
|
@ -271,7 +271,7 @@ public final class HostUpdateFlow implements TransactionalFlow {
|
|||
}
|
||||
// We must also enqueue updates for all domains that use this host as their nameserver so
|
||||
// that their NS records can be updated to point at the new name.
|
||||
asyncFlowEnqueuer.enqueueAsyncDnsRefresh(existingHost, ofy().getTransactionTime());
|
||||
asyncTaskEnqueuer.enqueueAsyncDnsRefresh(existingHost, ofy().getTransactionTime());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
199
java/google/registry/model/ResourceTransferUtils.java
Normal file
199
java/google/registry/model/ResourceTransferUtils.java
Normal file
|
@ -0,0 +1,199 @@
|
|||
// 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.model;
|
||||
|
||||
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.model.domain.DomainResource.extendRegistrationWithCap;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import google.registry.model.EppResource.Builder;
|
||||
import google.registry.model.EppResource.BuilderWithTransferData;
|
||||
import google.registry.model.EppResource.ForeignKeyedEppResource;
|
||||
import google.registry.model.EppResource.ResourceWithTransferData;
|
||||
import google.registry.model.contact.ContactResource;
|
||||
import google.registry.model.domain.DomainResource;
|
||||
import google.registry.model.eppcommon.StatusValue;
|
||||
import google.registry.model.eppcommon.Trid;
|
||||
import google.registry.model.index.ForeignKeyIndex;
|
||||
import google.registry.model.poll.PendingActionNotificationResponse;
|
||||
import google.registry.model.poll.PendingActionNotificationResponse.ContactPendingActionNotificationResponse;
|
||||
import google.registry.model.poll.PendingActionNotificationResponse.DomainPendingActionNotificationResponse;
|
||||
import google.registry.model.poll.PollMessage;
|
||||
import google.registry.model.reporting.HistoryEntry;
|
||||
import google.registry.model.transfer.TransferData;
|
||||
import google.registry.model.transfer.TransferResponse;
|
||||
import google.registry.model.transfer.TransferResponse.ContactTransferResponse;
|
||||
import google.registry.model.transfer.TransferResponse.DomainTransferResponse;
|
||||
import google.registry.model.transfer.TransferStatus;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Static utility functions for resource transfers. */
|
||||
public final class ResourceTransferUtils {
|
||||
|
||||
private ResourceTransferUtils() {}
|
||||
|
||||
/** Statuses for which an exDate should be added to transfer responses. */
|
||||
private static final ImmutableSet<TransferStatus> ADD_EXDATE_STATUSES = Sets.immutableEnumSet(
|
||||
TransferStatus.PENDING, TransferStatus.CLIENT_APPROVED, TransferStatus.SERVER_APPROVED);
|
||||
|
||||
/**
|
||||
* Create a transfer response using the id and type of this resource and the specified
|
||||
* {@link TransferData}.
|
||||
*/
|
||||
public static TransferResponse createTransferResponse(
|
||||
EppResource eppResource, TransferData transferData, DateTime now) {
|
||||
assertIsContactOrDomain(eppResource);
|
||||
TransferResponse.Builder<? extends TransferResponse, ?> builder;
|
||||
if (eppResource instanceof ContactResource) {
|
||||
builder = new ContactTransferResponse.Builder().setContactId(eppResource.getForeignKey());
|
||||
} else {
|
||||
DomainResource domain = (DomainResource) eppResource;
|
||||
builder =
|
||||
new DomainTransferResponse.Builder()
|
||||
.setFullyQualifiedDomainName(eppResource.getForeignKey())
|
||||
// TODO(b/25084229): fix exDate computation logic.
|
||||
.setExtendedRegistrationExpirationTime(
|
||||
ADD_EXDATE_STATUSES.contains(transferData.getTransferStatus())
|
||||
? extendRegistrationWithCap(now, domain.getRegistrationExpirationTime(), 1)
|
||||
: null);
|
||||
}
|
||||
builder.setGainingClientId(transferData.getGainingClientId())
|
||||
.setLosingClientId(transferData.getLosingClientId())
|
||||
.setPendingTransferExpirationTime(transferData.getPendingTransferExpirationTime())
|
||||
.setTransferRequestTime(transferData.getTransferRequestTime())
|
||||
.setTransferStatus(transferData.getTransferStatus());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a pending action notification response indicating the resolution of a transfer.
|
||||
*
|
||||
* <p>The returned object will use the id and type of this resource, the trid of the resource's
|
||||
* last transfer request, and the specified status and date.
|
||||
*/
|
||||
public static PendingActionNotificationResponse createPendingTransferNotificationResponse(
|
||||
EppResource eppResource,
|
||||
Trid transferRequestTrid,
|
||||
boolean actionResult,
|
||||
DateTime processedDate) {
|
||||
assertIsContactOrDomain(eppResource);
|
||||
return eppResource instanceof ContactResource
|
||||
? ContactPendingActionNotificationResponse.create(
|
||||
eppResource.getForeignKey(), actionResult, transferRequestTrid, processedDate)
|
||||
: DomainPendingActionNotificationResponse.create(
|
||||
eppResource.getForeignKey(), actionResult, transferRequestTrid, processedDate);
|
||||
}
|
||||
|
||||
private static void assertIsContactOrDomain(EppResource eppResource) {
|
||||
checkState(eppResource instanceof ContactResource || eppResource instanceof DomainResource);
|
||||
}
|
||||
|
||||
/** Update the relevant {@link ForeignKeyIndex} to cache the new deletion time. */
|
||||
public static <R extends EppResource> void updateForeignKeyIndexDeletionTime(R resource) {
|
||||
if (resource instanceof ForeignKeyedEppResource) {
|
||||
ofy().save().entity(ForeignKeyIndex.create(resource, resource.getDeletionTime()));
|
||||
}
|
||||
}
|
||||
|
||||
/** If there is a transfer out, delete the server-approve entities and enqueue a poll message. */
|
||||
public static <R extends EppResource & ResourceWithTransferData>
|
||||
void handlePendingTransferOnDelete(
|
||||
R resource, R newResource, DateTime now, HistoryEntry historyEntry) {
|
||||
if (resource.getStatusValues().contains(StatusValue.PENDING_TRANSFER)) {
|
||||
TransferData oldTransferData = resource.getTransferData();
|
||||
ofy().delete().keys(oldTransferData.getServerApproveEntities());
|
||||
ofy().save().entity(new PollMessage.OneTime.Builder()
|
||||
.setClientId(oldTransferData.getGainingClientId())
|
||||
.setEventTime(now)
|
||||
.setMsg(TransferStatus.SERVER_CANCELLED.getMessage())
|
||||
.setResponseData(ImmutableList.of(
|
||||
createTransferResponse(newResource, newResource.getTransferData(), now),
|
||||
createPendingTransferNotificationResponse(
|
||||
resource, oldTransferData.getTransferRequestTrid(), false, now)))
|
||||
.setParent(historyEntry)
|
||||
.build());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Turn a resource into a builder with its pending transfer resolved.
|
||||
*
|
||||
* <p>This removes the {@link StatusValue#PENDING_TRANSFER} status, sets the {@link
|
||||
* TransferStatus}, clears all the server-approve fields on the {@link TransferData}, and sets the
|
||||
* expiration time of the last pending transfer to now.
|
||||
*/
|
||||
private static <
|
||||
R extends EppResource & ResourceWithTransferData,
|
||||
B extends EppResource.Builder<R, B> & BuilderWithTransferData<B>>
|
||||
B resolvePendingTransfer(R resource, TransferStatus transferStatus, DateTime now) {
|
||||
checkArgument(
|
||||
resource.getStatusValues().contains(StatusValue.PENDING_TRANSFER),
|
||||
"Resource is not in pending transfer status.");
|
||||
checkArgument(
|
||||
!TransferData.EMPTY.equals(resource.getTransferData()),
|
||||
"No old transfer data to resolve.");
|
||||
@SuppressWarnings("unchecked")
|
||||
B builder = (B) resource.asBuilder();
|
||||
return builder
|
||||
.removeStatusValue(StatusValue.PENDING_TRANSFER)
|
||||
.setTransferData(
|
||||
resource.getTransferData().copyConstantFieldsToBuilder()
|
||||
.setTransferStatus(transferStatus)
|
||||
.setPendingTransferExpirationTime(checkNotNull(now))
|
||||
.build());
|
||||
}
|
||||
|
||||
/**
|
||||
* Resolve a pending transfer by awarding it to the gaining client.
|
||||
*
|
||||
* <p>This removes the {@link StatusValue#PENDING_TRANSFER} status, sets the {@link
|
||||
* TransferStatus}, clears all the server-approve fields on the {@link TransferData}, sets the new
|
||||
* client id, and sets the last transfer time and the expiration time of the last pending transfer
|
||||
* to now.
|
||||
*/
|
||||
public static <
|
||||
R extends EppResource & ResourceWithTransferData,
|
||||
B extends Builder<R, B> & BuilderWithTransferData<B>>
|
||||
R approvePendingTransfer(R resource, TransferStatus transferStatus, DateTime now) {
|
||||
checkArgument(transferStatus.isApproved(), "Not an approval transfer status");
|
||||
B builder = resolvePendingTransfer(resource, transferStatus, now);
|
||||
return builder
|
||||
.setLastTransferTime(now)
|
||||
.setPersistedCurrentSponsorClientId(resource.getTransferData().getGainingClientId())
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Resolve a pending transfer by denying it.
|
||||
*
|
||||
* <p>This removes the {@link StatusValue#PENDING_TRANSFER} status, sets the {@link
|
||||
* TransferStatus}, clears all the server-approve fields on the {@link TransferData}, sets the
|
||||
* expiration time of the last pending transfer to now, sets the last EPP update time to now, and
|
||||
* sets the last EPP update client id to the given client id.
|
||||
*/
|
||||
public static <R extends EppResource & ResourceWithTransferData> R denyPendingTransfer(
|
||||
R resource, TransferStatus transferStatus, DateTime now, String lastEppUpdateClientId) {
|
||||
checkArgument(transferStatus.isDenied(), "Not a denial transfer status");
|
||||
return resolvePendingTransfer(resource, transferStatus, now)
|
||||
.setLastEppUpdateTime(now)
|
||||
.setLastEppUpdateClientId(lastEppUpdateClientId)
|
||||
.build();
|
||||
}
|
||||
}
|
|
@ -20,7 +20,6 @@ import static google.registry.util.CollectionUtils.nullToEmpty;
|
|||
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.googlecode.objectify.Key;
|
||||
import com.googlecode.objectify.annotation.Entity;
|
||||
|
@ -121,7 +120,6 @@ public abstract class PollMessage extends ImmutableObject
|
|||
public abstract Builder<?, ?> asBuilder();
|
||||
|
||||
/** Builder for {@link PollMessage} because it is immutable. */
|
||||
@VisibleForTesting
|
||||
public abstract static class Builder<T extends PollMessage, B extends Builder<?, ?>>
|
||||
extends GenericBuilder<T, B> {
|
||||
|
||||
|
@ -131,7 +129,6 @@ public abstract class PollMessage extends ImmutableObject
|
|||
super(instance);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public B setId(Long id) {
|
||||
getInstance().id = id;
|
||||
return thisCastToDerived();
|
||||
|
@ -221,7 +218,6 @@ public abstract class PollMessage extends ImmutableObject
|
|||
}
|
||||
|
||||
/** A builder for {@link OneTime} since it is immutable. */
|
||||
@VisibleForTesting
|
||||
public static class Builder extends PollMessage.Builder<OneTime, Builder> {
|
||||
|
||||
public Builder() {}
|
||||
|
@ -327,7 +323,6 @@ public abstract class PollMessage extends ImmutableObject
|
|||
}
|
||||
|
||||
/** A builder for {@link Autorenew} since it is immutable. */
|
||||
@VisibleForTesting
|
||||
public static class Builder extends PollMessage.Builder<Autorenew, Builder> {
|
||||
|
||||
public Builder() {}
|
||||
|
|
|
@ -54,7 +54,6 @@ import google.registry.export.UpdateSnapshotViewAction;
|
|||
import google.registry.export.UploadDatastoreBackupAction;
|
||||
import google.registry.export.sheet.SheetModule;
|
||||
import google.registry.export.sheet.SyncRegistrarsSheetAction;
|
||||
import google.registry.flows.async.AsyncFlowsModule;
|
||||
import google.registry.mapreduce.MapreduceModule;
|
||||
import google.registry.monitoring.whitebox.WhiteboxModule;
|
||||
import google.registry.rde.BrdaCopyAction;
|
||||
|
@ -94,7 +93,6 @@ import google.registry.tmch.TmchSmdrlAction;
|
|||
@RequestScope
|
||||
@Subcomponent(
|
||||
modules = {
|
||||
AsyncFlowsModule.class,
|
||||
BackendModule.class,
|
||||
BackupModule.class,
|
||||
BatchModule.class,
|
||||
|
|
|
@ -12,16 +12,16 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package google.registry.flows.async;
|
||||
package google.registry.batch;
|
||||
|
||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESAVE_TIMES;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PATH_RESAVE_ENTITY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESAVE_TIMES;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PATH_RESAVE_ENTITY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.testing.DatastoreHelper.persistActiveContact;
|
||||
import static google.registry.testing.TaskQueueHelper.assertNoTasksEnqueued;
|
||||
import static google.registry.testing.TaskQueueHelper.assertTasksEnqueued;
|
||||
|
@ -54,9 +54,9 @@ import org.junit.runner.RunWith;
|
|||
import org.junit.runners.JUnit4;
|
||||
import org.mockito.Mock;
|
||||
|
||||
/** Unit tests for {@link AsyncFlowEnqueuer}. */
|
||||
/** Unit tests for {@link AsyncTaskEnqueuer}. */
|
||||
@RunWith(JUnit4.class)
|
||||
public class AsyncFlowEnqueuerTest extends ShardableTestCase {
|
||||
public class AsyncTaskEnqueuerTest extends ShardableTestCase {
|
||||
|
||||
@Rule
|
||||
public final AppEngineRule appEngine =
|
||||
|
@ -68,16 +68,16 @@ public class AsyncFlowEnqueuerTest extends ShardableTestCase {
|
|||
|
||||
@Mock private AppEngineServiceUtils appEngineServiceUtils;
|
||||
|
||||
private AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
private AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
private final CapturingLogHandler logHandler = new CapturingLogHandler();
|
||||
private final FakeClock clock = new FakeClock(DateTime.parse("2015-05-18T12:34:56Z"));
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
LoggerConfig.getConfig(AsyncFlowEnqueuer.class).addHandler(logHandler);
|
||||
LoggerConfig.getConfig(AsyncTaskEnqueuer.class).addHandler(logHandler);
|
||||
when(appEngineServiceUtils.getServiceHostname("backend")).thenReturn("backend.hostname.fake");
|
||||
asyncFlowEnqueuer =
|
||||
new AsyncFlowEnqueuer(
|
||||
asyncTaskEnqueuer =
|
||||
new AsyncTaskEnqueuer(
|
||||
getQueue(QUEUE_ASYNC_ACTIONS),
|
||||
getQueue(QUEUE_ASYNC_DELETE),
|
||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||
|
@ -89,7 +89,7 @@ public class AsyncFlowEnqueuerTest extends ShardableTestCase {
|
|||
@Test
|
||||
public void test_enqueueAsyncResave_success() {
|
||||
ContactResource contact = persistActiveContact("jd23456");
|
||||
asyncFlowEnqueuer.enqueueAsyncResave(contact, clock.nowUtc(), clock.nowUtc().plusDays(5));
|
||||
asyncTaskEnqueuer.enqueueAsyncResave(contact, clock.nowUtc(), clock.nowUtc().plusDays(5));
|
||||
assertTasksEnqueued(
|
||||
QUEUE_ASYNC_ACTIONS,
|
||||
new TaskMatcher()
|
||||
|
@ -108,7 +108,7 @@ public class AsyncFlowEnqueuerTest extends ShardableTestCase {
|
|||
public void test_enqueueAsyncResave_multipleResaves() {
|
||||
ContactResource contact = persistActiveContact("jd23456");
|
||||
DateTime now = clock.nowUtc();
|
||||
asyncFlowEnqueuer.enqueueAsyncResave(
|
||||
asyncTaskEnqueuer.enqueueAsyncResave(
|
||||
contact,
|
||||
now,
|
||||
ImmutableSortedSet.of(now.plusHours(24), now.plusHours(50), now.plusHours(75)));
|
||||
|
@ -132,7 +132,7 @@ public class AsyncFlowEnqueuerTest extends ShardableTestCase {
|
|||
@Test
|
||||
public void test_enqueueAsyncResave_ignoresTasksTooFarIntoFuture() throws Exception {
|
||||
ContactResource contact = persistActiveContact("jd23456");
|
||||
asyncFlowEnqueuer.enqueueAsyncResave(contact, clock.nowUtc(), clock.nowUtc().plusDays(31));
|
||||
asyncTaskEnqueuer.enqueueAsyncResave(contact, clock.nowUtc(), clock.nowUtc().plusDays(31));
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_ACTIONS);
|
||||
assertLogMessage(logHandler, Level.INFO, "Ignoring async re-save");
|
||||
}
|
|
@ -12,12 +12,12 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package google.registry.flows.async;
|
||||
package google.registry.batch;
|
||||
|
||||
import static com.google.monitoring.metrics.contrib.DistributionMetricSubject.assertThat;
|
||||
import static com.google.monitoring.metrics.contrib.LongMetricSubject.assertThat;
|
||||
import static google.registry.flows.async.AsyncFlowMetrics.OperationResult.SUCCESS;
|
||||
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.CONTACT_AND_HOST_DELETE;
|
||||
import static google.registry.batch.AsyncTaskMetrics.OperationResult.SUCCESS;
|
||||
import static google.registry.batch.AsyncTaskMetrics.OperationType.CONTACT_AND_HOST_DELETE;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.testing.FakeClock;
|
||||
|
@ -26,24 +26,24 @@ import org.junit.Test;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.JUnit4;
|
||||
|
||||
/** Unit tests for {@link AsyncFlowMetrics}. */
|
||||
/** Unit tests for {@link AsyncTaskMetrics}. */
|
||||
@RunWith(JUnit4.class)
|
||||
public class AsyncFlowMetricsTest extends ShardableTestCase {
|
||||
public class AsyncTaskMetricsTest extends ShardableTestCase {
|
||||
|
||||
private final FakeClock clock = new FakeClock();
|
||||
private final AsyncFlowMetrics asyncFlowMetrics = new AsyncFlowMetrics(clock);
|
||||
private final AsyncTaskMetrics asyncTaskMetrics = new AsyncTaskMetrics(clock);
|
||||
|
||||
@Test
|
||||
public void testRecordAsyncFlowResult_calculatesDurationMillisCorrectly() {
|
||||
asyncFlowMetrics.recordAsyncFlowResult(
|
||||
asyncTaskMetrics.recordAsyncFlowResult(
|
||||
CONTACT_AND_HOST_DELETE,
|
||||
SUCCESS,
|
||||
clock.nowUtc().minusMinutes(10).minusSeconds(5).minusMillis(566));
|
||||
assertThat(AsyncFlowMetrics.asyncFlowOperationCounts)
|
||||
assertThat(AsyncTaskMetrics.asyncFlowOperationCounts)
|
||||
.hasValueForLabels(1, "contactAndHostDelete", "success")
|
||||
.and()
|
||||
.hasNoOtherValues();
|
||||
assertThat(AsyncFlowMetrics.asyncFlowOperationProcessingTime)
|
||||
assertThat(AsyncTaskMetrics.asyncFlowOperationProcessingTime)
|
||||
.hasDataSetForLabels(ImmutableSet.of(605566.0), "contactAndHostDelete", "success")
|
||||
.and()
|
||||
.hasNoOtherValues();
|
|
@ -12,17 +12,13 @@ java_library(
|
|||
srcs = glob(["*.java"]),
|
||||
deps = [
|
||||
"//java/google/registry/batch",
|
||||
"//java/google/registry/bigquery",
|
||||
"//java/google/registry/config",
|
||||
"//java/google/registry/flows",
|
||||
"//java/google/registry/mapreduce",
|
||||
"//java/google/registry/model",
|
||||
"//java/google/registry/request",
|
||||
"//java/google/registry/util",
|
||||
"//javatests/google/registry/testing",
|
||||
"//javatests/google/registry/testing/mapreduce",
|
||||
"//third_party/objectify:objectify-v4_1",
|
||||
"@com_google_apis_google_api_services_bigquery",
|
||||
"@com_google_appengine_api_1_0_sdk",
|
||||
"@com_google_appengine_api_stubs",
|
||||
"@com_google_appengine_tools_appengine_gcs_client",
|
||||
|
@ -30,8 +26,12 @@ java_library(
|
|||
"@com_google_appengine_tools_appengine_pipeline",
|
||||
"@com_google_code_findbugs_jsr305",
|
||||
"@com_google_dagger",
|
||||
"@com_google_flogger",
|
||||
"@com_google_flogger_system_backend",
|
||||
"@com_google_guava",
|
||||
"@com_google_http_client",
|
||||
"@com_google_monitoring_client_contrib",
|
||||
"@com_google_monitoring_client_metrics",
|
||||
"@com_google_truth",
|
||||
"@com_google_truth_extensions_truth_java8_extension",
|
||||
"@javax_servlet_api",
|
||||
|
|
|
@ -18,10 +18,10 @@ import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
|||
import static com.google.common.collect.MoreCollectors.onlyElement;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.flows.async.AsyncFlowMetrics.OperationResult.STALE;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.batch.AsyncTaskMetrics.OperationResult.STALE;
|
||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||
import static google.registry.model.eppcommon.StatusValue.PENDING_DELETE;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
@ -67,11 +67,9 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskMetrics.OperationResult;
|
||||
import google.registry.batch.AsyncTaskMetrics.OperationType;
|
||||
import google.registry.batch.DeleteContactsAndHostsAction.DeleteEppResourceReducer;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.flows.async.AsyncFlowMetrics;
|
||||
import google.registry.flows.async.AsyncFlowMetrics.OperationResult;
|
||||
import google.registry.flows.async.AsyncFlowMetrics.OperationType;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.contact.ContactAddress;
|
||||
import google.registry.model.contact.ContactPhoneNumber;
|
||||
|
@ -124,7 +122,7 @@ public class DeleteContactsAndHostsActionTest
|
|||
@Rule public final InjectRule inject = new InjectRule();
|
||||
@Rule public final MockitoJUnitRule mocks = MockitoJUnitRule.create();
|
||||
|
||||
private AsyncFlowEnqueuer enqueuer;
|
||||
private AsyncTaskEnqueuer enqueuer;
|
||||
private final FakeClock clock = new FakeClock(DateTime.parse("2015-01-15T11:22:33Z"));
|
||||
private final FakeResponse fakeResponse = new FakeResponse();
|
||||
@Mock private RequestStatusChecker requestStatusChecker;
|
||||
|
@ -155,17 +153,17 @@ public class DeleteContactsAndHostsActionTest
|
|||
public void setup() {
|
||||
inject.setStaticField(Ofy.class, "clock", clock);
|
||||
enqueuer =
|
||||
new AsyncFlowEnqueuer(
|
||||
new AsyncTaskEnqueuer(
|
||||
getQueue(QUEUE_ASYNC_ACTIONS),
|
||||
getQueue(QUEUE_ASYNC_DELETE),
|
||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||
Duration.ZERO,
|
||||
mock(AppEngineServiceUtils.class),
|
||||
new Retrier(new FakeSleeper(clock), 1));
|
||||
AsyncFlowMetrics asyncFlowMetricsMock = mock(AsyncFlowMetrics.class);
|
||||
AsyncTaskMetrics asyncTaskMetricsMock = mock(AsyncTaskMetrics.class);
|
||||
action = new DeleteContactsAndHostsAction();
|
||||
action.asyncFlowMetrics = asyncFlowMetricsMock;
|
||||
inject.setStaticField(DeleteEppResourceReducer.class, "asyncFlowMetrics", asyncFlowMetricsMock);
|
||||
action.asyncTaskMetrics = asyncTaskMetricsMock;
|
||||
inject.setStaticField(DeleteEppResourceReducer.class, "asyncTaskMetrics", asyncTaskMetricsMock);
|
||||
action.clock = clock;
|
||||
action.mrRunner = makeDefaultRunner();
|
||||
action.requestStatusChecker = requestStatusChecker;
|
||||
|
@ -212,10 +210,10 @@ public class DeleteContactsAndHostsActionTest
|
|||
contact,
|
||||
Optional.of("fakeClientTrid"));
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(OperationType.CONTACT_DELETE, OperationResult.FAILURE, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -293,10 +291,10 @@ public class DeleteContactsAndHostsActionTest
|
|||
assertPollMessageFor(
|
||||
historyEntry, "TheRegistrar", "Deleted contact jim919.", true, contact, clientTrid);
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(OperationType.CONTACT_DELETE, OperationResult.SUCCESS, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
|
||||
}
|
||||
|
||||
|
@ -537,8 +535,8 @@ public class DeleteContactsAndHostsActionTest
|
|||
new TaskMatcher()
|
||||
.payload("gobbledygook=kljhadfgsd9f7gsdfh")
|
||||
.etaDelta(standardHours(23), standardHours(25)));
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verify(action.asyncTaskMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
assertThat(acquireLock()).isPresent();
|
||||
}
|
||||
|
||||
|
@ -565,12 +563,12 @@ public class DeleteContactsAndHostsActionTest
|
|||
assertThat(loadByForeignKey(HostResource.class, "rustles.your.jimmies", clock.nowUtc()))
|
||||
.hasValue(host);
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(2L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics).recordContactHostDeletionBatchSize(2L);
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(OperationType.CONTACT_DELETE, STALE, timeEnqueued);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(OperationType.HOST_DELETE, STALE, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
assertThat(acquireLock()).isPresent();
|
||||
}
|
||||
|
||||
|
@ -628,10 +626,10 @@ public class DeleteContactsAndHostsActionTest
|
|||
host,
|
||||
Optional.of("fakeClientTrid"));
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(OperationType.HOST_DELETE, OperationResult.FAILURE, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -676,10 +674,10 @@ public class DeleteContactsAndHostsActionTest
|
|||
host,
|
||||
clientTrid);
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(OperationType.HOST_DELETE, OperationResult.SUCCESS, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -17,10 +17,10 @@ package google.registry.batch;
|
|||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.DNS_REFRESH;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.batch.AsyncTaskMetrics.OperationType.DNS_REFRESH;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.testing.DatastoreHelper.createTld;
|
||||
import static google.registry.testing.DatastoreHelper.newDomainApplication;
|
||||
|
@ -47,10 +47,8 @@ import static org.mockito.Mockito.when;
|
|||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.AsyncTaskMetrics.OperationResult;
|
||||
import google.registry.batch.RefreshDnsOnHostRenameAction.RefreshDnsOnHostRenameReducer;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.flows.async.AsyncFlowMetrics;
|
||||
import google.registry.flows.async.AsyncFlowMetrics.OperationResult;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.model.server.Lock;
|
||||
import google.registry.testing.FakeClock;
|
||||
|
@ -83,7 +81,7 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
@Rule public final InjectRule inject = new InjectRule();
|
||||
@Rule public final MockitoJUnitRule mocks = MockitoJUnitRule.create();
|
||||
|
||||
private AsyncFlowEnqueuer enqueuer;
|
||||
private AsyncTaskEnqueuer enqueuer;
|
||||
private final FakeClock clock = new FakeClock(DateTime.parse("2015-01-15T11:22:33Z"));
|
||||
private final FakeResponse fakeResponse = new FakeResponse();
|
||||
@Mock private RequestStatusChecker requestStatusChecker;
|
||||
|
@ -92,18 +90,18 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
public void setup() {
|
||||
createTld("tld");
|
||||
enqueuer =
|
||||
new AsyncFlowEnqueuer(
|
||||
new AsyncTaskEnqueuer(
|
||||
getQueue(QUEUE_ASYNC_ACTIONS),
|
||||
getQueue(QUEUE_ASYNC_DELETE),
|
||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||
Duration.ZERO,
|
||||
mock(AppEngineServiceUtils.class),
|
||||
new Retrier(new FakeSleeper(clock), 1));
|
||||
AsyncFlowMetrics asyncFlowMetricsMock = mock(AsyncFlowMetrics.class);
|
||||
AsyncTaskMetrics asyncTaskMetricsMock = mock(AsyncTaskMetrics.class);
|
||||
action = new RefreshDnsOnHostRenameAction();
|
||||
action.asyncFlowMetrics = asyncFlowMetricsMock;
|
||||
action.asyncTaskMetrics = asyncTaskMetricsMock;
|
||||
inject.setStaticField(
|
||||
RefreshDnsOnHostRenameReducer.class, "asyncFlowMetrics", asyncFlowMetricsMock);
|
||||
RefreshDnsOnHostRenameReducer.class, "asyncTaskMetrics", asyncTaskMetricsMock);
|
||||
action.clock = clock;
|
||||
action.mrRunner = makeDefaultRunner();
|
||||
action.pullQueue = getQueue(QUEUE_ASYNC_HOST_RENAME);
|
||||
|
@ -152,10 +150,10 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
runMapreduce();
|
||||
assertDnsTasksEnqueued("example.tld", "otherexample.tld");
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
verify(action.asyncFlowMetrics).recordDnsRefreshBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics).recordDnsRefreshBatchSize(1L);
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.SUCCESS, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -174,12 +172,12 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
runMapreduce();
|
||||
assertDnsTasksEnqueued("example1.tld", "example2.tld", "example3.tld");
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
verify(action.asyncFlowMetrics).recordDnsRefreshBatchSize(3L);
|
||||
verify(action.asyncFlowMetrics, times(2))
|
||||
verify(action.asyncTaskMetrics).recordDnsRefreshBatchSize(3L);
|
||||
verify(action.asyncTaskMetrics, times(2))
|
||||
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.SUCCESS, timeEnqueued);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.SUCCESS, laterTimeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -191,10 +189,10 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
runMapreduce();
|
||||
assertNoDnsTasksEnqueued();
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
verify(action.asyncFlowMetrics).recordDnsRefreshBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
verify(action.asyncTaskMetrics).recordDnsRefreshBatchSize(1L);
|
||||
verify(action.asyncTaskMetrics)
|
||||
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.STALE, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
verifyNoMoreInteractions(action.asyncTaskMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -16,12 +16,12 @@ package google.registry.batch;
|
|||
|
||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PATH_RESAVE_ENTITY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PATH_RESAVE_ENTITY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.testing.DatastoreHelper.createTld;
|
||||
import static google.registry.testing.DatastoreHelper.newDomainResource;
|
||||
|
@ -38,7 +38,6 @@ import static org.mockito.Mockito.when;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.model.ImmutableObject;
|
||||
import google.registry.model.domain.DomainResource;
|
||||
import google.registry.model.domain.GracePeriod;
|
||||
|
@ -78,14 +77,14 @@ public class ResaveEntityActionTest extends ShardableTestCase {
|
|||
@Mock private AppEngineServiceUtils appEngineServiceUtils;
|
||||
@Mock private Response response;
|
||||
private final FakeClock clock = new FakeClock(DateTime.parse("2016-02-11T10:00:00Z"));
|
||||
private AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
private AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
inject.setStaticField(Ofy.class, "clock", clock);
|
||||
when(appEngineServiceUtils.getServiceHostname("backend")).thenReturn("backend.hostname.fake");
|
||||
asyncFlowEnqueuer =
|
||||
new AsyncFlowEnqueuer(
|
||||
asyncTaskEnqueuer =
|
||||
new AsyncTaskEnqueuer(
|
||||
getQueue(QUEUE_ASYNC_ACTIONS),
|
||||
getQueue(QUEUE_ASYNC_DELETE),
|
||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||
|
@ -101,7 +100,7 @@ public class ResaveEntityActionTest extends ShardableTestCase {
|
|||
ImmutableSortedSet<DateTime> resaveTimes) {
|
||||
ResaveEntityAction action =
|
||||
new ResaveEntityAction(
|
||||
resourceKey, requestedTime, resaveTimes, asyncFlowEnqueuer, response);
|
||||
resourceKey, requestedTime, resaveTimes, asyncTaskEnqueuer, response);
|
||||
action.run();
|
||||
}
|
||||
|
||||
|
|
|
@ -24,6 +24,7 @@ java_library(
|
|||
]),
|
||||
resources = glob(["**/testdata/*.xml"]),
|
||||
deps = [
|
||||
"//java/google/registry/batch",
|
||||
"//java/google/registry/config",
|
||||
"//java/google/registry/dns",
|
||||
"//java/google/registry/flows",
|
||||
|
|
|
@ -15,9 +15,9 @@
|
|||
package google.registry.flows;
|
||||
|
||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
|
@ -25,10 +25,10 @@ import dagger.Component;
|
|||
import dagger.Module;
|
||||
import dagger.Provides;
|
||||
import dagger.Subcomponent;
|
||||
import google.registry.batch.AsyncTaskEnqueuer;
|
||||
import google.registry.config.RegistryConfig.ConfigModule;
|
||||
import google.registry.config.RegistryConfig.ConfigModule.TmchCaMode;
|
||||
import google.registry.dns.DnsQueue;
|
||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||
import google.registry.flows.custom.CustomLogicFactory;
|
||||
import google.registry.flows.custom.TestCustomLogicFactory;
|
||||
import google.registry.flows.domain.DomainFlowTmchUtils;
|
||||
|
@ -62,7 +62,7 @@ interface EppTestComponent {
|
|||
@Module
|
||||
class FakesAndMocksModule {
|
||||
|
||||
private AsyncFlowEnqueuer asyncFlowEnqueuer;
|
||||
private AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||
private DnsQueue dnsQueue;
|
||||
private DomainFlowTmchUtils domainFlowTmchUtils;
|
||||
private EppMetric.Builder metricBuilder;
|
||||
|
@ -90,8 +90,8 @@ interface EppTestComponent {
|
|||
FakesAndMocksModule instance = new FakesAndMocksModule();
|
||||
AppEngineServiceUtils appEngineServiceUtils = mock(AppEngineServiceUtils.class);
|
||||
when(appEngineServiceUtils.getServiceHostname("backend")).thenReturn("backend.hostname.fake");
|
||||
instance.asyncFlowEnqueuer =
|
||||
new AsyncFlowEnqueuer(
|
||||
instance.asyncTaskEnqueuer =
|
||||
new AsyncTaskEnqueuer(
|
||||
getQueue(QUEUE_ASYNC_ACTIONS),
|
||||
getQueue(QUEUE_ASYNC_DELETE),
|
||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||
|
@ -109,8 +109,8 @@ interface EppTestComponent {
|
|||
}
|
||||
|
||||
@Provides
|
||||
AsyncFlowEnqueuer provideAsyncFlowEnqueuer() {
|
||||
return asyncFlowEnqueuer;
|
||||
AsyncTaskEnqueuer provideAsyncTaskEnqueuer() {
|
||||
return asyncTaskEnqueuer;
|
||||
}
|
||||
|
||||
@Provides
|
||||
|
|
|
@ -16,11 +16,11 @@ package google.registry.flows.domain;
|
|||
|
||||
import static com.google.common.collect.MoreCollectors.onlyElement;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESAVE_TIMES;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PATH_RESAVE_ENTITY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESAVE_TIMES;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PATH_RESAVE_ENTITY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.flows.domain.DomainTransferFlowTestCase.persistWithPendingTransfer;
|
||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
|
|
|
@ -18,10 +18,10 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
|||
import static com.google.common.collect.MoreCollectors.onlyElement;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PATH_RESAVE_ENTITY;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_REQUESTED_TIME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.PATH_RESAVE_ENTITY;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_ACTIONS;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.model.reporting.DomainTransactionRecord.TransactionReportField.TRANSFER_SUCCESSFUL;
|
||||
import static google.registry.model.reporting.HistoryEntry.Type.DOMAIN_CREATE;
|
||||
|
|
|
@ -16,7 +16,7 @@ package google.registry.flows.host;
|
|||
|
||||
import static com.google.common.base.Strings.nullToEmpty;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.batch.AsyncTaskEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.testing.DatastoreHelper.assertNoBillingEvents;
|
||||
|
|
|
@ -24,9 +24,9 @@ import static com.google.common.truth.Truth.assertThat;
|
|||
import static com.google.common.truth.Truth.assertWithMessage;
|
||||
import static google.registry.config.RegistryConfig.getContactAndHostRoidSuffix;
|
||||
import static google.registry.config.RegistryConfig.getContactAutomaticTransferLength;
|
||||
import static google.registry.flows.ResourceFlowUtils.createTransferResponse;
|
||||
import static google.registry.model.EppResourceUtils.createDomainRepoId;
|
||||
import static google.registry.model.EppResourceUtils.createRepoId;
|
||||
import static google.registry.model.ResourceTransferUtils.createTransferResponse;
|
||||
import static google.registry.model.domain.launch.ApplicationStatus.VALIDATED;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.model.registry.label.PremiumListUtils.parentPremiumListEntriesOnRevision;
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue