mirror of
https://github.com/google/nomulus.git
synced 2025-07-21 10:16:07 +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
|
@ -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,169 +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.common.base.Preconditions.checkArgument;
|
||||
import static google.registry.util.DateTimeUtils.isBeforeOrAt;
|
||||
|
||||
import com.google.appengine.api.taskqueue.Queue;
|
||||
import com.google.appengine.api.taskqueue.TaskOptions;
|
||||
import com.google.appengine.api.taskqueue.TaskOptions.Method;
|
||||
import com.google.appengine.api.taskqueue.TransientFailureException;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.model.EppResource;
|
||||
import google.registry.model.ImmutableObject;
|
||||
import google.registry.model.eppcommon.Trid;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.util.AppEngineServiceUtils;
|
||||
import google.registry.util.Retrier;
|
||||
import javax.inject.Inject;
|
||||
import javax.inject.Named;
|
||||
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 {
|
||||
|
||||
/** The HTTP parameter names used by async flows. */
|
||||
public static final String PARAM_RESOURCE_KEY = "resourceKey";
|
||||
public static final String PARAM_REQUESTING_CLIENT_ID = "requestingClientId";
|
||||
public static final String PARAM_CLIENT_TRANSACTION_ID = "clientTransactionId";
|
||||
public static final String PARAM_SERVER_TRANSACTION_ID = "serverTransactionId";
|
||||
public static final String PARAM_IS_SUPERUSER = "isSuperuser";
|
||||
public static final String PARAM_HOST_KEY = "hostKey";
|
||||
public static final String PARAM_REQUESTED_TIME = "requestedTime";
|
||||
public static final String PARAM_RESAVE_TIMES = "resaveTimes";
|
||||
|
||||
/** The task queue names used by async flows. */
|
||||
public static final String QUEUE_ASYNC_ACTIONS = "async-actions";
|
||||
public static final String QUEUE_ASYNC_DELETE = "async-delete-pull";
|
||||
public static final String QUEUE_ASYNC_HOST_RENAME = "async-host-rename-pull";
|
||||
|
||||
public static final String PATH_RESAVE_ENTITY = "/_dr/task/resaveEntity";
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
private static final Duration MAX_ASYNC_ETA = Duration.standardDays(30);
|
||||
|
||||
private final Duration asyncDeleteDelay;
|
||||
private final Queue asyncActionsPushQueue;
|
||||
private final Queue asyncDeletePullQueue;
|
||||
private final Queue asyncDnsRefreshPullQueue;
|
||||
private final AppEngineServiceUtils appEngineServiceUtils;
|
||||
private final Retrier retrier;
|
||||
|
||||
@VisibleForTesting
|
||||
@Inject
|
||||
public AsyncFlowEnqueuer(
|
||||
@Named(QUEUE_ASYNC_ACTIONS) Queue asyncActionsPushQueue,
|
||||
@Named(QUEUE_ASYNC_DELETE) Queue asyncDeletePullQueue,
|
||||
@Named(QUEUE_ASYNC_HOST_RENAME) Queue asyncDnsRefreshPullQueue,
|
||||
@Config("asyncDeleteFlowMapreduceDelay") Duration asyncDeleteDelay,
|
||||
AppEngineServiceUtils appEngineServiceUtils,
|
||||
Retrier retrier) {
|
||||
this.asyncActionsPushQueue = asyncActionsPushQueue;
|
||||
this.asyncDeletePullQueue = asyncDeletePullQueue;
|
||||
this.asyncDnsRefreshPullQueue = asyncDnsRefreshPullQueue;
|
||||
this.asyncDeleteDelay = asyncDeleteDelay;
|
||||
this.appEngineServiceUtils = appEngineServiceUtils;
|
||||
this.retrier = retrier;
|
||||
}
|
||||
|
||||
/** Enqueues a task to asynchronously re-save an entity at some point in the future. */
|
||||
public void enqueueAsyncResave(
|
||||
ImmutableObject entityToResave, DateTime now, DateTime whenToResave) {
|
||||
enqueueAsyncResave(entityToResave, now, ImmutableSortedSet.of(whenToResave));
|
||||
}
|
||||
|
||||
/**
|
||||
* Enqueues a task to asynchronously re-save an entity at some point(s) in the future.
|
||||
*
|
||||
* <p>Multiple re-save times are chained one after the other, i.e. any given run will re-enqueue
|
||||
* itself to run at the next time if there are remaining re-saves scheduled.
|
||||
*/
|
||||
public void enqueueAsyncResave(
|
||||
ImmutableObject entityToResave, DateTime now, ImmutableSortedSet<DateTime> whenToResave) {
|
||||
DateTime firstResave = whenToResave.first();
|
||||
checkArgument(isBeforeOrAt(now, firstResave), "Can't enqueue a resave to run in the past");
|
||||
Key<ImmutableObject> entityKey = Key.create(entityToResave);
|
||||
Duration etaDuration = new Duration(now, firstResave);
|
||||
if (etaDuration.isLongerThan(MAX_ASYNC_ETA)) {
|
||||
logger.atInfo().log(
|
||||
"Ignoring async re-save of %s; %s is past the ETA threshold of %s.",
|
||||
entityKey, firstResave, MAX_ASYNC_ETA);
|
||||
return;
|
||||
}
|
||||
logger.atInfo().log("Enqueuing async re-save of %s to run at %s.", entityKey, whenToResave);
|
||||
String backendHostname = appEngineServiceUtils.getServiceHostname("backend");
|
||||
TaskOptions task =
|
||||
TaskOptions.Builder.withUrl(PATH_RESAVE_ENTITY)
|
||||
.method(Method.POST)
|
||||
.header("Host", backendHostname)
|
||||
.countdownMillis(etaDuration.getMillis())
|
||||
.param(PARAM_RESOURCE_KEY, entityKey.getString())
|
||||
.param(PARAM_REQUESTED_TIME, now.toString());
|
||||
if (whenToResave.size() > 1) {
|
||||
task.param(PARAM_RESAVE_TIMES, Joiner.on(',').join(whenToResave.tailSet(firstResave, false)));
|
||||
}
|
||||
addTaskToQueueWithRetry(asyncActionsPushQueue, task);
|
||||
}
|
||||
|
||||
/** Enqueues a task to asynchronously delete a contact or host, by key. */
|
||||
public void enqueueAsyncDelete(
|
||||
EppResource resourceToDelete,
|
||||
DateTime now,
|
||||
String requestingClientId,
|
||||
Trid trid,
|
||||
boolean isSuperuser) {
|
||||
Key<EppResource> resourceKey = Key.create(resourceToDelete);
|
||||
logger.atInfo().log(
|
||||
"Enqueuing async deletion of %s on behalf of registrar %s.",
|
||||
resourceKey, requestingClientId);
|
||||
TaskOptions task =
|
||||
TaskOptions.Builder.withMethod(Method.PULL)
|
||||
.countdownMillis(asyncDeleteDelay.getMillis())
|
||||
.param(PARAM_RESOURCE_KEY, resourceKey.getString())
|
||||
.param(PARAM_REQUESTING_CLIENT_ID, requestingClientId)
|
||||
.param(PARAM_SERVER_TRANSACTION_ID, trid.getServerTransactionId())
|
||||
.param(PARAM_IS_SUPERUSER, Boolean.toString(isSuperuser))
|
||||
.param(PARAM_REQUESTED_TIME, now.toString());
|
||||
trid.getClientTransactionId()
|
||||
.ifPresent(clTrid -> task.param(PARAM_CLIENT_TRANSACTION_ID, clTrid));
|
||||
addTaskToQueueWithRetry(asyncDeletePullQueue, task);
|
||||
}
|
||||
|
||||
/** Enqueues a task to asynchronously refresh DNS for a renamed host. */
|
||||
public void enqueueAsyncDnsRefresh(HostResource host, DateTime now) {
|
||||
Key<HostResource> hostKey = Key.create(host);
|
||||
logger.atInfo().log("Enqueuing async DNS refresh for renamed host %s.", hostKey);
|
||||
addTaskToQueueWithRetry(
|
||||
asyncDnsRefreshPullQueue,
|
||||
TaskOptions.Builder.withMethod(Method.PULL)
|
||||
.param(PARAM_HOST_KEY, hostKey.getString())
|
||||
.param(PARAM_REQUESTED_TIME, now.toString()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a task to a queue with retrying, to avoid aborting the entire flow over a transient issue
|
||||
* enqueuing a task.
|
||||
*/
|
||||
private void addTaskToQueueWithRetry(final Queue queue, final TaskOptions task) {
|
||||
retrier.callWithRetry(() -> queue.add(task), TransientFailureException.class);
|
||||
}
|
||||
}
|
|
@ -1,165 +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.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 com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.google.monitoring.metrics.DistributionFitter;
|
||||
import com.google.monitoring.metrics.EventMetric;
|
||||
import com.google.monitoring.metrics.FibonacciFitter;
|
||||
import com.google.monitoring.metrics.IncrementableMetric;
|
||||
import com.google.monitoring.metrics.LabelDescriptor;
|
||||
import com.google.monitoring.metrics.MetricRegistryImpl;
|
||||
import google.registry.util.Clock;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/**
|
||||
* Instrumentation for async flows (contact/host deletion and DNS refreshes).
|
||||
*
|
||||
* @see AsyncFlowEnqueuer
|
||||
*/
|
||||
public class AsyncFlowMetrics {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
private final Clock clock;
|
||||
|
||||
@Inject
|
||||
public AsyncFlowMetrics(Clock clock) {
|
||||
this.clock = clock;
|
||||
}
|
||||
|
||||
/**
|
||||
* A Fibonacci fitter used for bucketing the batch count.
|
||||
*
|
||||
* <p>We use a Fibonacci filter because it provides better resolution at the low end than an
|
||||
* exponential fitter, which is important because most batch sizes are likely to be very low,
|
||||
* despite going up to 1,000 on the high end. Also, the precision is better, as batch size is
|
||||
* inherently an integer, whereas an exponential fitter with an exponent base less than 2 would
|
||||
* have unintuitive boundaries.
|
||||
*/
|
||||
private static final DistributionFitter FITTER_BATCH_SIZE =
|
||||
FibonacciFitter.create(maxLeaseCount());
|
||||
|
||||
private static final ImmutableSet<LabelDescriptor> LABEL_DESCRIPTORS =
|
||||
ImmutableSet.of(
|
||||
LabelDescriptor.create("operation_type", "The type of async flow operation."),
|
||||
LabelDescriptor.create("result", "The result of the async flow operation."));
|
||||
|
||||
@VisibleForTesting
|
||||
static final IncrementableMetric asyncFlowOperationCounts =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newIncrementableMetric(
|
||||
"/async_flows/operations",
|
||||
"Count of Async Flow Operations",
|
||||
"count",
|
||||
LABEL_DESCRIPTORS);
|
||||
|
||||
@VisibleForTesting
|
||||
static final EventMetric asyncFlowOperationProcessingTime =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newEventMetric(
|
||||
"/async_flows/processing_time",
|
||||
"Async Flow Processing Time",
|
||||
"milliseconds",
|
||||
LABEL_DESCRIPTORS,
|
||||
DEFAULT_FITTER);
|
||||
|
||||
@VisibleForTesting
|
||||
static final EventMetric asyncFlowBatchSize =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newEventMetric(
|
||||
"/async_flows/batch_size",
|
||||
"Async Operation Batch Size",
|
||||
"batch size",
|
||||
ImmutableSet.of(
|
||||
LabelDescriptor.create("operation_type", "The type of async flow operation.")),
|
||||
FITTER_BATCH_SIZE);
|
||||
|
||||
/** The type of asynchronous operation. */
|
||||
public enum OperationType {
|
||||
CONTACT_DELETE("contactDelete"),
|
||||
HOST_DELETE("hostDelete"),
|
||||
CONTACT_AND_HOST_DELETE("contactAndHostDelete"),
|
||||
DNS_REFRESH("dnsRefresh");
|
||||
|
||||
private final String metricLabelValue;
|
||||
|
||||
OperationType(String metricLabelValue) {
|
||||
this.metricLabelValue = metricLabelValue;
|
||||
}
|
||||
|
||||
String getMetricLabelValue() {
|
||||
return metricLabelValue;
|
||||
}
|
||||
}
|
||||
|
||||
/** The result of an asynchronous operation. */
|
||||
public enum OperationResult {
|
||||
/** The operation processed correctly and the result was success. */
|
||||
SUCCESS("success"),
|
||||
|
||||
/** The operation processed correctly and the result was failure. */
|
||||
FAILURE("failure"),
|
||||
|
||||
/** The operation did not process correctly due to some unexpected error. */
|
||||
ERROR("error"),
|
||||
|
||||
/** The operation was skipped because the request is now stale. */
|
||||
STALE("stale");
|
||||
|
||||
private final String metricLabelValue;
|
||||
|
||||
OperationResult(String metricLabelValue) {
|
||||
this.metricLabelValue = metricLabelValue;
|
||||
}
|
||||
|
||||
String getMetricLabelValue() {
|
||||
return metricLabelValue;
|
||||
}
|
||||
}
|
||||
|
||||
public void recordAsyncFlowResult(
|
||||
OperationType operationType, OperationResult operationResult, DateTime whenEnqueued) {
|
||||
asyncFlowOperationCounts.increment(
|
||||
operationType.getMetricLabelValue(), operationResult.getMetricLabelValue());
|
||||
long processingMillis = new Duration(whenEnqueued, clock.nowUtc()).getMillis();
|
||||
asyncFlowOperationProcessingTime.record(
|
||||
processingMillis,
|
||||
operationType.getMetricLabelValue(),
|
||||
operationResult.getMetricLabelValue());
|
||||
logger.atInfo().log(
|
||||
"Asynchronous %s operation took %d ms to process, yielding result: %s.",
|
||||
operationType.getMetricLabelValue(),
|
||||
processingMillis,
|
||||
operationResult.getMetricLabelValue());
|
||||
}
|
||||
|
||||
public void recordContactHostDeletionBatchSize(long batchSize) {
|
||||
asyncFlowBatchSize.record(batchSize, CONTACT_AND_HOST_DELETE.getMetricLabelValue());
|
||||
}
|
||||
|
||||
public void recordDnsRefreshBatchSize(long batchSize) {
|
||||
asyncFlowBatchSize.record(batchSize, DNS_REFRESH.getMetricLabelValue());
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue