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:
guyben 2019-01-10 06:19:17 -08:00 committed by Ben McIlwain
parent d0c73efac0
commit 898448b8a0
42 changed files with 419 additions and 423 deletions

View file

@ -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,

View file

@ -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;
}

View file

@ -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",

View file

@ -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);
}
}

View file

@ -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());

View file

@ -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;

View file

@ -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. */

View file

@ -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.");

View file

@ -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")

View file

@ -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",

View file

@ -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;

View file

@ -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,

View file

@ -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) {

View file

@ -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);
}
}

View file

@ -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();

View file

@ -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;

View file

@ -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;

View file

@ -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;

View file

@ -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;

View file

@ -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))

View file

@ -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;

View file

@ -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;

View file

@ -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;

View file

@ -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;

View file

@ -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)

View file

@ -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();

View file

@ -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());
}
}

View 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();
}
}

View file

@ -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() {}

View file

@ -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,

View file

@ -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");
}

View file

@ -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();

View file

@ -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",

View file

@ -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

View file

@ -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

View file

@ -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();
}

View file

@ -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",

View file

@ -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

View file

@ -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;

View file

@ -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;

View file

@ -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;

View file

@ -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;