diff --git a/java/google/registry/config/ConfigModule.java b/java/google/registry/config/ConfigModule.java
index 8a5d242a8..8711c1554 100644
--- a/java/google/registry/config/ConfigModule.java
+++ b/java/google/registry/config/ConfigModule.java
@@ -665,15 +665,33 @@ public final class ConfigModule {
return config.getContactAutomaticTransferLength();
}
- @Provides
- @Config("asyncDeleteFlowMapreduceDelay")
- public static Duration provideAsyncDeleteFlowMapreduceDelay(RegistryConfig config) {
- return config.getAsyncDeleteFlowMapreduceDelay();
- }
-
@Provides
@Config("maxChecks")
public static int provideMaxChecks(RegistryConfig config) {
return config.getMaxChecks();
}
+
+ /**
+ * Returns the delay before executing async delete flow mapreduces.
+ *
+ *
This delay should be sufficiently longer than a transaction, to solve the following problem:
+ *
+ * - a domain mutation flow starts a transaction
+ *
- the domain flow non-transactionally reads a resource and sees that it's not in
+ * PENDING_DELETE
+ *
- the domain flow creates a new reference to this resource
+ *
- a contact/host delete flow runs and marks the resource PENDING_DELETE and commits
+ *
- the domain flow commits
+ *
+ *
+ * Although we try not to add references to a PENDING_DELETE resource, strictly speaking that
+ * is ok as long as the mapreduce eventually sees the new reference (and therefore asynchronously
+ * fails the delete). Without this delay, the mapreduce might have started before the domain flow
+ * committed, and could potentially miss the reference.
+ */
+ @Provides
+ @Config("asyncDeleteFlowMapreduceDelay")
+ public static Duration getAsyncDeleteFlowMapreduceDelay() {
+ return Duration.standardSeconds(90);
+ }
}
diff --git a/java/google/registry/config/RegistryConfig.java b/java/google/registry/config/RegistryConfig.java
index c66ab572e..7a13d2237 100644
--- a/java/google/registry/config/RegistryConfig.java
+++ b/java/google/registry/config/RegistryConfig.java
@@ -229,6 +229,7 @@ public interface RegistryConfig {
* fails the delete). Without this delay, the mapreduce might have started before the domain flow
* committed, and could potentially miss the reference.
*/
+ // TODO(b/26140521): Remove this configuration option along with non-batched async operations.
public Duration getAsyncDeleteFlowMapreduceDelay();
/**
@@ -237,6 +238,7 @@ public interface RegistryConfig {
* This should be ~orders of magnitude larger than the rate on the queue, in order to prevent
* the logs from filling up with unnecessarily failures.
*/
+ // TODO(b/26140521): Remove this configuration option along with non-batched async operations.
public Duration getAsyncFlowFailureBackoff();
// XXX: Please consider using ConfigModule instead of adding new methods to this file.
diff --git a/java/google/registry/env/alpha/default/WEB-INF/cron.xml b/java/google/registry/env/alpha/default/WEB-INF/cron.xml
index 5164d34ea..abf4e9212 100644
--- a/java/google/registry/env/alpha/default/WEB-INF/cron.xml
+++ b/java/google/registry/env/alpha/default/WEB-INF/cron.xml
@@ -122,6 +122,17 @@
backend
+
+
+
+ This job runs a mapreduce that processes batch asynchronous deletions of
+ contact and host resources by mapping over all EppResources and checking
+ for any references to the contacts/hosts in pending deletion.
+
+ every 5 minutes synchronized
+ backend
+
+
diff --git a/java/google/registry/env/common/backend/WEB-INF/web.xml b/java/google/registry/env/common/backend/WEB-INF/web.xml
index 8e02c28f2..51c8816dd 100644
--- a/java/google/registry/env/common/backend/WEB-INF/web.xml
+++ b/java/google/registry/env/common/backend/WEB-INF/web.xml
@@ -241,17 +241,28 @@
+
backend-servlet
/_dr/task/deleteContactResource
+
backend-servlet
/_dr/task/deleteHostResource
+
+
+ backend-servlet
+ /_dr/task/deleteContactsAndHosts
+
+
backend-servlet
diff --git a/java/google/registry/env/common/default/WEB-INF/queue.xml b/java/google/registry/env/common/default/WEB-INF/queue.xml
index 891e35428..de6fb23ac 100644
--- a/java/google/registry/env/common/default/WEB-INF/queue.xml
+++ b/java/google/registry/env/common/default/WEB-INF/queue.xml
@@ -12,20 +12,23 @@
pull
+
+
+
+ flows-async
+ 1/m
+ backend
+
+
dns-publish
100/s
100
-
- flows-async
-
- 1/m
-
- backend
+ async-delete-pull
+ pull
diff --git a/java/google/registry/env/crash/default/WEB-INF/cron.xml b/java/google/registry/env/crash/default/WEB-INF/cron.xml
index 2570890dd..0fccefc78 100644
--- a/java/google/registry/env/crash/default/WEB-INF/cron.xml
+++ b/java/google/registry/env/crash/default/WEB-INF/cron.xml
@@ -102,6 +102,17 @@
backend
+
+
+
+ This job runs a mapreduce that processes batch asynchronous deletions of
+ contact and host resources by mapping over all EppResources and checking
+ for any references to the contacts/hosts in pending deletion.
+
+ every 5 minutes synchronized
+ backend
+
+
diff --git a/java/google/registry/env/production/default/WEB-INF/cron.xml b/java/google/registry/env/production/default/WEB-INF/cron.xml
index d9c7487fc..df0b7da98 100644
--- a/java/google/registry/env/production/default/WEB-INF/cron.xml
+++ b/java/google/registry/env/production/default/WEB-INF/cron.xml
@@ -133,6 +133,17 @@
backend
+
+
+
+ This job runs a mapreduce that processes batch asynchronous deletions of
+ contact and host resources by mapping over all EppResources and checking
+ for any references to the contacts/hosts in pending deletion.
+
+ every 5 minutes synchronized
+ backend
+
+
diff --git a/java/google/registry/env/sandbox/default/WEB-INF/cron.xml b/java/google/registry/env/sandbox/default/WEB-INF/cron.xml
index e39b71ccc..9eb3a45cc 100644
--- a/java/google/registry/env/sandbox/default/WEB-INF/cron.xml
+++ b/java/google/registry/env/sandbox/default/WEB-INF/cron.xml
@@ -40,6 +40,17 @@
backend
+
+
+
+ This job runs a mapreduce that processes batch asynchronous deletions of
+ contact and host resources by mapping over all EppResources and checking
+ for any references to the contacts/hosts in pending deletion.
+
+ every 5 minutes synchronized
+ backend
+
+
diff --git a/java/google/registry/flows/BUILD b/java/google/registry/flows/BUILD
index 0ddfb72dd..30ed58145 100644
--- a/java/google/registry/flows/BUILD
+++ b/java/google/registry/flows/BUILD
@@ -31,6 +31,7 @@ java_library(
"//java/com/google/common/net",
"//third_party/java/appengine:appengine-api",
"//third_party/java/appengine_mapreduce2:appengine_mapreduce",
+ "//third_party/java/auto:auto_value",
"//third_party/java/dagger",
"//third_party/java/joda_money",
"//third_party/java/joda_time",
diff --git a/java/google/registry/flows/FlowComponent.java b/java/google/registry/flows/FlowComponent.java
index 515a9c398..b4fbc447c 100644
--- a/java/google/registry/flows/FlowComponent.java
+++ b/java/google/registry/flows/FlowComponent.java
@@ -17,6 +17,8 @@ package google.registry.flows;
import dagger.Module;
import dagger.Provides;
import dagger.Subcomponent;
+import google.registry.config.ConfigModule;
+import google.registry.flows.async.AsyncFlowsModule;
import google.registry.flows.contact.ContactCheckFlow;
import google.registry.flows.contact.ContactCreateFlow;
import google.registry.flows.contact.ContactDeleteFlow;
@@ -56,12 +58,16 @@ import google.registry.flows.session.HelloFlow;
import google.registry.flows.session.LoginFlow;
import google.registry.flows.session.LogoutFlow;
import google.registry.model.eppcommon.Trid;
+import google.registry.util.SystemSleeper.SystemSleeperModule;
/** Dagger component for flow classes. */
@FlowScope
@Subcomponent(modules = {
+ AsyncFlowsModule.class,
+ ConfigModule.class,
FlowModule.class,
- FlowComponent.FlowComponentModule.class})
+ FlowComponent.FlowComponentModule.class,
+ SystemSleeperModule.class})
public interface FlowComponent {
Trid trid();
diff --git a/java/google/registry/flows/async/AsyncFlowEnqueuer.java b/java/google/registry/flows/async/AsyncFlowEnqueuer.java
new file mode 100644
index 000000000..c8c09c158
--- /dev/null
+++ b/java/google/registry/flows/async/AsyncFlowEnqueuer.java
@@ -0,0 +1,71 @@
+// Copyright 2016 The Domain Registry 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 google.registry.flows.async.DeleteContactsAndHostsAction.PARAM_IS_SUPERUSER;
+import static google.registry.flows.async.DeleteContactsAndHostsAction.PARAM_REQUESTING_CLIENT_ID;
+import static google.registry.flows.async.DeleteContactsAndHostsAction.PARAM_RESOURCE_KEY;
+
+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.googlecode.objectify.Key;
+import google.registry.config.ConfigModule.Config;
+import google.registry.model.EppResource;
+import google.registry.util.FormattingLogger;
+import google.registry.util.Retrier;
+import java.util.concurrent.Callable;
+import javax.inject.Inject;
+import javax.inject.Named;
+import org.joda.time.Duration;
+
+/** Helper class to enqueue tasks for handling asynchronous deletions to pull queues. */
+public final class AsyncFlowEnqueuer {
+
+ private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
+
+ @Inject @Config("asyncDeleteFlowMapreduceDelay") Duration asyncDeleteDelay;
+ @Inject @Named("async-delete-pull") Queue queue;
+ @Inject Retrier retrier;
+ @Inject AsyncFlowEnqueuer() {}
+
+ /**
+ * Enqueues a task to asynchronously delete a contact or host, by key.
+ *
+ * Note that the clientId is of the logged-in registrar that is requesting the deletion, not
+ * necessarily the current owner of the resource.
+ */
+ public void enqueueAsyncDelete(
+ EppResource resourceToDelete, String clientId, boolean isSuperuser) {
+ Key resourceKey = Key.create(resourceToDelete);
+ logger.infofmt(
+ "Enqueueing async action to delete %s on behalf of registrar %s.", resourceKey, clientId);
+ final TaskOptions options =
+ TaskOptions.Builder
+ .withMethod(Method.PULL)
+ .countdownMillis(asyncDeleteDelay.getMillis())
+ .param(PARAM_RESOURCE_KEY, resourceKey.getString())
+ .param(PARAM_REQUESTING_CLIENT_ID, clientId)
+ .param(PARAM_IS_SUPERUSER, Boolean.toString(isSuperuser));
+ // Retry on transient failure exceptions so that the entire flow isn't aborted unnecessarily.
+ retrier.callWithRetry(new Callable() {
+ @Override
+ public Void call() throws Exception {
+ queue.add(options);
+ return null;
+ }}, TransientFailureException.class);
+ }
+}
diff --git a/java/google/registry/flows/async/AsyncFlowUtils.java b/java/google/registry/flows/async/AsyncFlowUtils.java
index 42efb7d16..ab5fe027c 100644
--- a/java/google/registry/flows/async/AsyncFlowUtils.java
+++ b/java/google/registry/flows/async/AsyncFlowUtils.java
@@ -30,6 +30,7 @@ import java.util.Map.Entry;
import org.joda.time.Duration;
/** Utility methods specific to async flows. */
+// TODO(b/26140521): Delete this class once non-batched async operations are deleted.
public final class AsyncFlowUtils {
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
diff --git a/java/google/registry/flows/async/AsyncFlowsModule.java b/java/google/registry/flows/async/AsyncFlowsModule.java
index 7ec5431b6..9dcfa52cd 100644
--- a/java/google/registry/flows/async/AsyncFlowsModule.java
+++ b/java/google/registry/flows/async/AsyncFlowsModule.java
@@ -13,7 +13,7 @@
// limitations under the License.
package google.registry.flows.async;
-
+import static google.registry.flows.async.DeleteContactsAndHostsAction.QUEUE_ASYNC_DELETE;
import static google.registry.flows.async.DeleteEppResourceAction.PARAM_IS_SUPERUSER;
import static google.registry.flows.async.DeleteEppResourceAction.PARAM_REQUESTING_CLIENT_ID;
import static google.registry.flows.async.DeleteEppResourceAction.PARAM_RESOURCE_KEY;
@@ -21,29 +21,41 @@ import static google.registry.flows.async.DnsRefreshForHostRenameAction.PARAM_HO
import static google.registry.request.RequestParameters.extractBooleanParameter;
import static google.registry.request.RequestParameters.extractRequiredParameter;
+import com.google.appengine.api.taskqueue.Queue;
+import com.google.appengine.api.taskqueue.QueueFactory;
import dagger.Module;
import dagger.Provides;
import google.registry.request.Parameter;
+import javax.inject.Named;
import javax.servlet.http.HttpServletRequest;
/** Dagger module for the async flows package. */
@Module
public final class AsyncFlowsModule {
+ @Provides
+ @Named(QUEUE_ASYNC_DELETE)
+ static Queue provideAsyncDeletePullQueue() {
+ return QueueFactory.getQueue(QUEUE_ASYNC_DELETE);
+ }
+
@Provides
@Parameter(PARAM_IS_SUPERUSER)
+ //TODO(b/26140521): Delete this method once non-batched async operations are deleted.
static boolean provideIsSuperuser(HttpServletRequest req) {
return extractBooleanParameter(req, PARAM_IS_SUPERUSER);
}
@Provides
@Parameter(PARAM_REQUESTING_CLIENT_ID)
+ //TODO(b/26140521): Delete this method once non-batched async operations are deleted.
static String provideRequestingClientId(HttpServletRequest req) {
return extractRequiredParameter(req, PARAM_REQUESTING_CLIENT_ID);
}
@Provides
@Parameter(PARAM_RESOURCE_KEY)
+ //TODO(b/26140521): Delete this method once non-batched async operations are deleted.
static String provideResourceKey(HttpServletRequest req) {
return extractRequiredParameter(req, PARAM_RESOURCE_KEY);
}
diff --git a/java/google/registry/flows/async/DeleteContactResourceAction.java b/java/google/registry/flows/async/DeleteContactResourceAction.java
index f129407ea..1f044c9ae 100644
--- a/java/google/registry/flows/async/DeleteContactResourceAction.java
+++ b/java/google/registry/flows/async/DeleteContactResourceAction.java
@@ -30,6 +30,7 @@ import org.joda.time.DateTime;
* existing DomainBase entity.
*/
@Action(path = "/_dr/task/deleteContactResource")
+// TODO(b/26140521): Delete this class once non-batched async operations are deleted.
public class DeleteContactResourceAction extends DeleteEppResourceAction {
@Inject
diff --git a/java/google/registry/flows/async/DeleteContactsAndHostsAction.java b/java/google/registry/flows/async/DeleteContactsAndHostsAction.java
new file mode 100644
index 000000000..edf28f276
--- /dev/null
+++ b/java/google/registry/flows/async/DeleteContactsAndHostsAction.java
@@ -0,0 +1,394 @@
+// Copyright 2016 The Domain Registry 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.appengine.api.taskqueue.QueueFactory.getQueue;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+import static com.googlecode.objectify.Key.getKind;
+import static google.registry.flows.ResourceFlowUtils.handlePendingTransferOnDelete;
+import static google.registry.flows.ResourceFlowUtils.prepareDeletedResourceAsBuilder;
+import static google.registry.flows.ResourceFlowUtils.updateForeignKeyIndexDeletionTime;
+import static google.registry.model.EppResourceUtils.isActive;
+import static google.registry.model.EppResourceUtils.isDeleted;
+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;
+import static google.registry.model.reporting.HistoryEntry.Type.CONTACT_DELETE_FAILURE;
+import static google.registry.model.reporting.HistoryEntry.Type.HOST_DELETE;
+import static google.registry.model.reporting.HistoryEntry.Type.HOST_DELETE_FAILURE;
+import static google.registry.util.FormattingLogger.getLoggerForCallerClass;
+import static google.registry.util.PipelineUtils.createJobPath;
+import static java.util.concurrent.TimeUnit.DAYS;
+import static java.util.concurrent.TimeUnit.MINUTES;
+
+import com.google.appengine.api.taskqueue.LeaseOptions;
+import com.google.appengine.api.taskqueue.Queue;
+import com.google.appengine.api.taskqueue.TaskHandle;
+import com.google.appengine.tools.mapreduce.Mapper;
+import com.google.appengine.tools.mapreduce.Reducer;
+import com.google.appengine.tools.mapreduce.ReducerInput;
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterators;
+import com.googlecode.objectify.Key;
+import com.googlecode.objectify.Work;
+import google.registry.dns.DnsQueue;
+import google.registry.flows.async.DeleteContactsAndHostsAction.DeletionResult.Type;
+import google.registry.mapreduce.MapreduceRunner;
+import google.registry.mapreduce.inputs.EppResourceInputs;
+import google.registry.mapreduce.inputs.NullInput;
+import google.registry.model.EppResource;
+import google.registry.model.ImmutableObject;
+import google.registry.model.annotations.ExternalMessagingName;
+import google.registry.model.contact.ContactResource;
+import google.registry.model.domain.DomainBase;
+import google.registry.model.host.HostResource;
+import google.registry.model.poll.PollMessage;
+import google.registry.model.reporting.HistoryEntry;
+import google.registry.request.Action;
+import google.registry.request.Response;
+import google.registry.util.Clock;
+import google.registry.util.FormattingLogger;
+import java.io.Serializable;
+import java.util.List;
+import javax.inject.Inject;
+import javax.inject.Named;
+import org.joda.time.DateTime;
+
+/**
+ * A mapreduce that processes batch asynchronous deletions of contact and host resources by mapping
+ * over all domains and domain applications and checking for any references to the contacts/hosts in
+ * pending deletion.
+ */
+@Action(path = "/_dr/task/deleteContactsAndHosts")
+public class DeleteContactsAndHostsAction implements Runnable {
+
+ /** The HTTP parameter name used to specify the websafe key of the resource to delete. */
+ public static final String PARAM_RESOURCE_KEY = "resourceKey";
+ public static final String PARAM_REQUESTING_CLIENT_ID = "requestingClientId";
+ public static final String PARAM_IS_SUPERUSER = "isSuperuser";
+ public static final String QUEUE_ASYNC_DELETE = "async-delete-pull";
+
+ static final String KIND_CONTACT = getKind(ContactResource.class);
+ static final String KIND_HOST = getKind(HostResource.class);
+
+ private static final long LEASE_MINUTES = 20;
+ private static final FormattingLogger logger = getLoggerForCallerClass();
+
+ @Inject Clock clock;
+ @Inject MapreduceRunner mrRunner;
+ @Inject @Named(QUEUE_ASYNC_DELETE) Queue queue;
+ @Inject Response response;
+ @Inject DeleteContactsAndHostsAction() {}
+
+ @Override
+ public void run() {
+ LeaseOptions options =
+ LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(LEASE_MINUTES, MINUTES);
+ List tasks = queue.leaseTasks(options);
+ if (tasks.isEmpty()) {
+ return;
+ }
+ ImmutableList.Builder builder = new ImmutableList.Builder<>();
+ for (TaskHandle task : tasks) {
+ try {
+ builder.add(DeletionRequest.createFromTask(task, clock.nowUtc()));
+ } catch (Exception e) {
+ logger.severefmt(
+ e, "Could not parse async deletion request, delaying task for a day: %s", task);
+ // Grab the lease for a whole day, so that it won't continue throwing errors every five
+ // minutes.
+ queue.modifyTaskLease(task, 1L, DAYS);
+ }
+ }
+ ImmutableList deletionRequests = builder.build();
+ logger.infofmt(
+ "Processing asynchronous deletion of %d contacts and hosts.", deletionRequests.size());
+ runMapreduce(deletionRequests);
+ }
+
+ private void runMapreduce(ImmutableList deletionRequests) {
+ try {
+ response.sendJavaScriptRedirect(createJobPath(mrRunner
+ .setJobName("Check for EPP resource references and then delete")
+ .setModuleName("backend")
+ .runMapreduce(
+ new DeleteContactsAndHostsMapper(deletionRequests),
+ new DeleteEppResourceReducer(),
+ ImmutableList.of(
+ // Add an extra shard that maps over a null domain. See the mapper code for why.
+ new NullInput(),
+ EppResourceInputs.createEntityInput(DomainBase.class)))));
+ } catch (Throwable t) {
+ logger.severefmt(t, "Error while kicking off mapreduce to delete contacts/hosts");
+ }
+ }
+
+ /**
+ * A mapper that iterates over all {@link DomainBase} entities.
+ *
+ * It emits the target key and {@code true} for domains referencing the target resource. For
+ * the special input of {@code null} it emits the target key and {@code false}.
+ */
+ public static class DeleteContactsAndHostsMapper
+ extends Mapper {
+
+ private static final long serialVersionUID = -253652818502690537L;
+
+ private final ImmutableList deletionRequests;
+
+ DeleteContactsAndHostsMapper(ImmutableList resourcesToDelete) {
+ this.deletionRequests = resourcesToDelete;
+ }
+
+ @Override
+ public void map(DomainBase domain) {
+ for (DeletionRequest deletionRequest : deletionRequests) {
+ if (domain == null) {
+ // The reducer only runs if at least one value is emitted. We add a null input to the
+ // mapreduce and emit one 'false' for each deletion request so that the reducer always
+ // runs for each requested deletion (so that it can finish up tasks if nothing else).
+ emit(deletionRequest, false);
+ } else if (isActive(domain, deletionRequest.lastUpdateTime())
+ && isLinked(domain, deletionRequest.key())) {
+ emit(deletionRequest, true);
+ getContext()
+ .incrementCounter(
+ String.format("active Domain-%s links found", deletionRequest.key().getKind()));
+ }
+ }
+ if (domain != null) {
+ getContext().incrementCounter("domains processed");
+ }
+ }
+
+ /** Determine whether the target resource is a linked resource on the domain. */
+ private boolean isLinked(DomainBase domain, Key extends EppResource> resourceKey) {
+ if (resourceKey.getKind().equals(KIND_CONTACT)) {
+ return domain.getReferencedContacts().contains(resourceKey);
+ } else if (resourceKey.getKind().equals(KIND_HOST)) {
+ return domain.getNameservers().contains(resourceKey);
+ } else {
+ throw new IllegalStateException("EPP resource key of unknown type: " + resourceKey);
+ }
+ }
+ }
+
+ /**
+ * A reducer that checks if the EPP resource to be deleted is referenced anywhere, and then
+ * deletes it if not and unmarks it for deletion if so.
+ */
+ public static class DeleteEppResourceReducer
+ extends Reducer {
+
+ private static final long serialVersionUID = 6569363449285506326L;
+
+ @Override
+ public void reduce(final DeletionRequest deletionRequest, ReducerInput values) {
+ final boolean hasNoActiveReferences = !Iterators.contains(values, true);
+ logger.infofmt("Processing async deletion request for %s", deletionRequest.key());
+ DeletionResult result = ofy().transactNew(new Work() {
+ @Override
+ @SuppressWarnings("unchecked")
+ public DeletionResult run() {
+ DeletionResult deletionResult =
+ attemptToDeleteResource(deletionRequest, hasNoActiveReferences);
+ getQueue(QUEUE_ASYNC_DELETE).deleteTask(deletionRequest.task());
+ return deletionResult;
+ }});
+ String resourceNamePlural = deletionRequest.key().getKind() + "s";
+ getContext().incrementCounter(result.type().renderCounterText(resourceNamePlural));
+ logger.infofmt(
+ "Result of async deletion for resource %s: %s",
+ deletionRequest.key(), result.pollMessageText());
+ }
+
+ private DeletionResult attemptToDeleteResource(
+ DeletionRequest deletionRequest, boolean hasNoActiveReferences) {
+ DateTime now = ofy().getTransactionTime();
+ EppResource resource =
+ ofy().load().key(deletionRequest.key()).now().cloneProjectedAtTime(now);
+ // Double-check transactionally that the resource is still active and in PENDING_DELETE.
+ try {
+ checkResourceStateAllowsDeletion(resource, now);
+ } catch (IllegalStateException e) {
+ logger.severefmt(e, "State of %s does not allow async deletion", deletionRequest.key());
+ return DeletionResult.create(Type.ERRORED, "");
+ }
+
+ boolean requestedByCurrentOwner =
+ resource.getCurrentSponsorClientId().equals(deletionRequest.requestingClientId());
+ boolean deleteAllowed =
+ hasNoActiveReferences && (requestedByCurrentOwner || deletionRequest.isSuperuser());
+
+ String resourceTypeName =
+ resource.getClass().getAnnotation(ExternalMessagingName.class).value();
+ String pollMessageText =
+ deleteAllowed
+ ? String.format("Deleted %s %s.", resourceTypeName, resource.getForeignKey())
+ : String.format(
+ "Can't delete %s %s because %s.",
+ resourceTypeName,
+ resource.getForeignKey(),
+ requestedByCurrentOwner
+ ? "it is referenced by a domain"
+ : "it was transferred prior to deletion");
+
+ HistoryEntry historyEntry =
+ new HistoryEntry.Builder()
+ .setClientId(deletionRequest.requestingClientId())
+ .setModificationTime(now)
+ .setType(getHistoryEntryType(resource, deleteAllowed))
+ .setParent(deletionRequest.key())
+ .build();
+
+ PollMessage.OneTime pollMessage =
+ new PollMessage.OneTime.Builder()
+ .setClientId(deletionRequest.requestingClientId())
+ .setMsg(pollMessageText)
+ .setParent(historyEntry)
+ .setEventTime(now)
+ .build();
+
+ EppResource resourceToSave;
+ if (deleteAllowed) {
+ resourceToSave = prepareDeletedResourceAsBuilder(resource, now).build();
+ performDeleteTasks(resource, resourceToSave, now, historyEntry);
+ updateForeignKeyIndexDeletionTime(resourceToSave);
+ } else {
+ resourceToSave = resource.asBuilder().removeStatusValue(PENDING_DELETE).build();
+ }
+ ofy().save().entities(resourceToSave, historyEntry, pollMessage);
+ return DeletionResult.create(
+ deleteAllowed ? Type.DELETED : Type.NOT_DELETED, pollMessageText);
+ }
+
+ /**
+ * Determine the proper history entry type for the delete operation, as a function of
+ * whether or not the delete was successful.
+ */
+ private HistoryEntry.Type getHistoryEntryType(EppResource resource, boolean successfulDelete) {
+ if (resource instanceof ContactResource) {
+ return successfulDelete ? CONTACT_DELETE : CONTACT_DELETE_FAILURE;
+ } else if (resource instanceof HostResource) {
+ return successfulDelete ? HOST_DELETE : HOST_DELETE_FAILURE;
+ } else {
+ throw new IllegalStateException("EPP resource of unknown type: " + Key.create(resource));
+ }
+ }
+
+ /** Perform any type-specific tasks on the resource to be deleted (and/or its dependencies). */
+ private void performDeleteTasks(
+ EppResource existingResource,
+ EppResource deletedResource,
+ DateTime deletionTime,
+ HistoryEntry historyEntryForDelete) {
+ if (existingResource instanceof ContactResource) {
+ handlePendingTransferOnDelete(
+ existingResource, deletedResource, deletionTime, historyEntryForDelete);
+ } else if (existingResource instanceof HostResource) {
+ HostResource host = (HostResource) existingResource;
+ if (host.getSuperordinateDomain() != null) {
+ DnsQueue.create().addHostRefreshTask(host.getFullyQualifiedHostName());
+ ofy().save().entity(
+ ofy().load().key(host.getSuperordinateDomain()).now().asBuilder()
+ .removeSubordinateHost(host.getFullyQualifiedHostName())
+ .build());
+ }
+ } else {
+ throw new IllegalStateException(
+ "EPP resource of unknown type: " + Key.create(existingResource));
+ }
+ }
+ }
+
+ /** A class that encapsulates the values of a request to delete a contact or host. */
+ @AutoValue
+ abstract static class DeletionRequest implements Serializable {
+
+ private static final long serialVersionUID = 5782119100274089088L;
+
+ abstract Key extends EppResource> key();
+ abstract DateTime lastUpdateTime();
+ /**
+ * The client id of the registrar that requested this deletion (which might NOT be the same as
+ * the actual current owner of the resource).
+ */
+ abstract String requestingClientId();
+ abstract boolean isSuperuser();
+ abstract TaskHandle task();
+
+ static DeletionRequest createFromTask(TaskHandle task, DateTime now) throws Exception {
+ ImmutableMap params = ImmutableMap.copyOf(task.extractParams());
+ Key resourceKey = Key.create(
+ checkNotNull(params.get(PARAM_RESOURCE_KEY), "Resource to delete not specified"));
+ EppResource resource = checkNotNull(
+ ofy().load().key(resourceKey).now(), "Resource to delete doesn't exist");
+ checkState(
+ resource instanceof ContactResource || resource instanceof HostResource,
+ "Cannot delete a %s via this action",
+ resource.getClass().getSimpleName());
+ checkResourceStateAllowsDeletion(resource, now);
+ return new AutoValue_DeleteContactsAndHostsAction_DeletionRequest(
+ resourceKey,
+ resource.getUpdateAutoTimestamp().getTimestamp(),
+ checkNotNull(
+ params.get(PARAM_REQUESTING_CLIENT_ID), "Requesting client id not specified"),
+ Boolean.valueOf(
+ checkNotNull(params.get(PARAM_IS_SUPERUSER), "Is superuser not specified")),
+ task);
+ }
+ }
+
+ /** A class that encapsulates the values resulting from attempted contact/host deletion. */
+ @AutoValue
+ abstract static class DeletionResult {
+
+ enum Type {
+ DELETED("%s deleted"),
+ NOT_DELETED("%s not deleted"),
+ ERRORED("%s errored out during deletion");
+
+ private final String counterFormat;
+
+ private Type(String counterFormat) {
+ this.counterFormat = counterFormat;
+ }
+
+ String renderCounterText(String resourceName) {
+ return String.format(counterFormat, resourceName);
+ }
+ }
+
+ abstract Type type();
+ abstract String pollMessageText();
+
+ static DeletionResult create(Type type, String pollMessageText) {
+ return new AutoValue_DeleteContactsAndHostsAction_DeletionResult(type, pollMessageText);
+ }
+ }
+
+ static void checkResourceStateAllowsDeletion(EppResource resource, DateTime now) {
+ Key key = Key.create(resource);
+ checkState(!isDeleted(resource, now), "Resource %s is already deleted", key);
+ checkState(
+ resource.getStatusValues().contains(PENDING_DELETE),
+ "Resource %s is not set as PENDING_DELETE",
+ key);
+ }
+}
diff --git a/java/google/registry/flows/async/DeleteEppResourceAction.java b/java/google/registry/flows/async/DeleteEppResourceAction.java
index e7624c1f1..945571f11 100644
--- a/java/google/registry/flows/async/DeleteEppResourceAction.java
+++ b/java/google/registry/flows/async/DeleteEppResourceAction.java
@@ -53,6 +53,7 @@ import org.joda.time.DateTime;
* A mapreduce to delete the specified EPP resource, but ONLY if it is not referred to by any
* existing DomainBase entity.
*/
+// TODO(b/26140521): Delete this class once non-batched async operations are deleted.
public abstract class DeleteEppResourceAction implements Runnable {
/** The HTTP parameter name used to specify the websafe key of the resource to delete. */
diff --git a/java/google/registry/flows/async/DeleteHostResourceAction.java b/java/google/registry/flows/async/DeleteHostResourceAction.java
index 494fb65b8..21dd07df2 100644
--- a/java/google/registry/flows/async/DeleteHostResourceAction.java
+++ b/java/google/registry/flows/async/DeleteHostResourceAction.java
@@ -31,6 +31,7 @@ import org.joda.time.DateTime;
* existing DomainBase entity.
*/
@Action(path = "/_dr/task/deleteHostResource")
+// TODO(b/26140521): Delete this class once non-batched async operations are deleted.
public class DeleteHostResourceAction extends DeleteEppResourceAction {
@Inject
diff --git a/java/google/registry/flows/contact/ContactDeleteFlow.java b/java/google/registry/flows/contact/ContactDeleteFlow.java
index 3c49fda71..578386d84 100644
--- a/java/google/registry/flows/contact/ContactDeleteFlow.java
+++ b/java/google/registry/flows/contact/ContactDeleteFlow.java
@@ -33,6 +33,7 @@ import google.registry.flows.FlowModule.ClientId;
import google.registry.flows.FlowModule.TargetId;
import google.registry.flows.LoggedInFlow;
import google.registry.flows.TransactionalFlow;
+import google.registry.flows.async.AsyncFlowEnqueuer;
import google.registry.flows.async.AsyncFlowUtils;
import google.registry.flows.async.DeleteContactResourceAction;
import google.registry.flows.async.DeleteEppResourceAction;
@@ -63,6 +64,7 @@ public class ContactDeleteFlow extends LoggedInFlow implements TransactionalFlow
StatusValue.PENDING_DELETE,
StatusValue.SERVER_DELETE_PROHIBITED);
+ @Inject AsyncFlowEnqueuer asyncFlowEnqueuer;
@Inject @ClientId String clientId;
@Inject @TargetId String targetId;
@Inject Optional authInfo;
@@ -105,6 +107,8 @@ public class ContactDeleteFlow extends LoggedInFlow implements TransactionalFlow
DeleteEppResourceAction.PARAM_IS_SUPERUSER,
Boolean.toString(isSuperuser)),
mapreduceDelay);
+ // TODO(b/26140521): Switch over to batch async operations as follows:
+ // asyncFlowEnqueuer.enqueueAsyncDelete(existingResource, getClientId(), isSuperuser);
ContactResource newResource =
existingResource.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
historyBuilder
diff --git a/java/google/registry/flows/host/HostDeleteFlow.java b/java/google/registry/flows/host/HostDeleteFlow.java
index db5fe17e3..3b6bac841 100644
--- a/java/google/registry/flows/host/HostDeleteFlow.java
+++ b/java/google/registry/flows/host/HostDeleteFlow.java
@@ -24,6 +24,7 @@ import com.googlecode.objectify.Key;
import google.registry.config.RegistryEnvironment;
import google.registry.flows.EppException;
import google.registry.flows.ResourceAsyncDeleteFlow;
+import google.registry.flows.async.AsyncFlowEnqueuer;
import google.registry.flows.async.AsyncFlowUtils;
import google.registry.flows.async.DeleteEppResourceAction;
import google.registry.flows.async.DeleteHostResourceAction;
@@ -47,6 +48,7 @@ public class HostDeleteFlow extends ResourceAsyncDeleteFlow {
+
+ @Rule
+ public final ExceptionRule thrown = new ExceptionRule();
+
+ @Rule
+ public final InjectRule inject = new InjectRule();
+
+ AsyncFlowEnqueuer enqueuer;
+ FakeClock clock = new FakeClock(DateTime.parse("2015-01-15T11:22:33Z"));
+
+ private void runMapreduce() throws Exception {
+ clock.advanceBy(standardSeconds(5));
+ // Apologies for the hard sleeps. Without them, the tests can be flaky because the tasks aren't
+ // quite fully enqueued by the time the tests attempt to lease from the queue.
+ Sleeper sleeper = new SystemSleeper();
+ sleeper.sleep(millis(50));
+ action.run();
+ sleeper.sleep(millis(50));
+ executeTasksUntilEmpty("mapreduce", clock);
+ sleeper.sleep(millis(50));
+ clock.advanceBy(standardSeconds(5));
+ ofy().clearSessionCache();
+ }
+
+ @Before
+ public void setup() throws Exception {
+ enqueuer = new AsyncFlowEnqueuer();
+ enqueuer.asyncDeleteDelay = Duration.ZERO;
+ enqueuer.queue = QueueFactory.getQueue(QUEUE_ASYNC_DELETE);
+ enqueuer.retrier = new Retrier(new FakeSleeper(clock), 1);
+
+ action = new DeleteContactsAndHostsAction();
+ action.clock = clock;
+ action.mrRunner = new MapreduceRunner(Optional.of(5), Optional.of(2));
+ action.response = new FakeResponse();
+ action.queue = getQueue(QUEUE_ASYNC_DELETE);
+ inject.setStaticField(Ofy.class, "clock", clock);
+
+ createTld("tld");
+ clock.advanceOneMilli();
+ }
+
+ @After
+ public void after() throws Exception {
+ LeaseOptions options =
+ LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(20, TimeUnit.MINUTES);
+ assertThat(getQueue(QUEUE_ASYNC_DELETE).leaseTasks(options)).isEmpty();
+ }
+
+ @Test
+ public void testSuccess_contact_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
+ ContactResource contact = persistContactPendingDelete("blah8221");
+ persistResource(newDomainResource("example.tld", contact));
+ enqueuer.enqueueAsyncDelete(contact, "TheRegistrar", false);
+ runMapreduce();
+ ContactResource contactUpdated =
+ loadByUniqueId(ContactResource.class, "blah8221", clock.nowUtc());
+ assertAboutContacts()
+ .that(contactUpdated)
+ .doesNotHaveStatusValue(PENDING_DELETE)
+ .and()
+ .hasDeletionTime(END_OF_TIME);
+ DomainResource domainReloaded =
+ loadByUniqueId(DomainResource.class, "example.tld", clock.nowUtc());
+ assertThat(domainReloaded.getReferencedContacts()).contains(Key.create(contactUpdated));
+ HistoryEntry historyEntry =
+ getOnlyHistoryEntryOfType(contactUpdated, HistoryEntry.Type.CONTACT_DELETE_FAILURE);
+ assertPollMessageFor(
+ historyEntry,
+ "TheRegistrar",
+ "Can't delete contact blah8221 because it is referenced by a domain.");
+ }
+
+ @Test
+ public void testSuccess_contact_notReferenced_getsDeleted_andPiiWipedOut() throws Exception {
+ ContactResource contact = persistContactWithPii("jim919");
+ enqueuer.enqueueAsyncDelete(contact, "TheRegistrar", false);
+ runMapreduce();
+ assertThat(loadByUniqueId(ContactResource.class, "jim919", clock.nowUtc())).isNull();
+ ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
+ assertAboutContacts()
+ .that(contactAfterDeletion)
+ .isNotActiveAt(clock.nowUtc())
+ // Note that there will be another history entry of CONTACT_PENDING_DELETE, but this is
+ // added by the flow and not the mapreduce itself.
+ .and()
+ .hasOnlyOneHistoryEntryWhich()
+ .hasType(CONTACT_DELETE);
+ assertAboutContacts()
+ .that(contactAfterDeletion)
+ .hasNullLocalizedPostalInfo()
+ .and()
+ .hasNullInternationalizedPostalInfo()
+ .and()
+ .hasNullEmailAddress()
+ .and()
+ .hasNullVoiceNumber()
+ .and()
+ .hasNullFaxNumber();
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
+ assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact jim919.");
+ }
+
+ @Test
+ public void testSuccess_contactWithPendingTransfer_getsDeleted() throws Exception {
+ DateTime transferRequestTime = clock.nowUtc().minusDays(3);
+ ContactResource contact =
+ persistContactWithPendingTransfer(
+ newContactResource("sh8013").asBuilder().addStatusValue(PENDING_DELETE).build(),
+ transferRequestTime,
+ transferRequestTime.plus(Registry.DEFAULT_TRANSFER_GRACE_PERIOD),
+ clock.nowUtc());
+ enqueuer.enqueueAsyncDelete(contact, "TheRegistrar", false);
+ runMapreduce();
+ // Check that the contact is deleted as of now.
+ assertThat(loadByUniqueId(ContactResource.class, "sh8013", clock.nowUtc())).isNull();
+ // Check that it's still there (it wasn't deleted yesterday) and that it has history.
+ assertAboutContacts()
+ .that(loadByUniqueId(ContactResource.class, "sh8013", clock.nowUtc().minusDays(1)))
+ .hasOneHistoryEntryEachOfTypes(CONTACT_TRANSFER_REQUEST, CONTACT_DELETE);
+ assertNoBillingEvents();
+ PollMessage deletePollMessage =
+ Iterables.getOnlyElement(getPollMessages("TheRegistrar", clock.nowUtc().plusMonths(1)));
+ assertThat(deletePollMessage.getMsg()).isEqualTo("Deleted contact sh8013.");
+ // The poll message in the future to the gaining registrar should be gone too, but there
+ // should be one at the current time to the gaining registrar.
+ PollMessage gainingPollMessage =
+ Iterables.getOnlyElement(getPollMessages("NewRegistrar", clock.nowUtc()));
+ assertThat(gainingPollMessage.getEventTime()).isLessThan(clock.nowUtc());
+ assertThat(
+ Iterables.getOnlyElement(
+ FluentIterable.from(gainingPollMessage.getResponseData())
+ .filter(TransferResponse.class))
+ .getTransferStatus())
+ .isEqualTo(SERVER_CANCELLED);
+ PendingActionNotificationResponse panData =
+ getOnlyElement(
+ FluentIterable.from(gainingPollMessage.getResponseData())
+ .filter(PendingActionNotificationResponse.class));
+ assertThat(panData.getTrid())
+ .isEqualTo(Trid.create("transferClient-trid", "transferServer-trid"));
+ assertThat(panData.getActionResult()).isFalse();
+ }
+
+ @Test
+ public void testSuccess_contact_referencedByDeletedDomain_getsDeleted() throws Exception {
+ ContactResource contactUsed = persistContactPendingDelete("blah1234");
+ persistResource(
+ newDomainResource("example.tld", contactUsed)
+ .asBuilder()
+ .setDeletionTime(clock.nowUtc().minusDays(3))
+ .build());
+ enqueuer.enqueueAsyncDelete(contactUsed, "TheRegistrar", false);
+ runMapreduce();
+ assertThat(loadByUniqueId(ContactResource.class, "blah1234", clock.nowUtc())).isNull();
+ ContactResource contactBeforeDeletion =
+ loadByUniqueId(ContactResource.class, "blah1234", clock.nowUtc().minusDays(1));
+ assertAboutContacts()
+ .that(contactBeforeDeletion)
+ .isNotActiveAt(clock.nowUtc())
+ .and()
+ .hasExactlyStatusValues(StatusValue.OK)
+ // Note that there will be another history entry of CONTACT_PENDING_DELETE, but this is
+ // added by the flow and not the mapreduce itself.
+ .and()
+ .hasOnlyOneHistoryEntryWhich()
+ .hasType(CONTACT_DELETE);
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactBeforeDeletion, CONTACT_DELETE);
+ assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact blah1234.");
+ }
+
+ @Test
+ public void testFailure_notInPendingDelete() throws Exception {
+ ContactResource contact = persistActiveContact("blah2222");
+ HostResource host = persistActiveHost("rustles.your.jimmies");
+ enqueuer.enqueueAsyncDelete(contact, "TheRegistrar", false);
+ enqueuer.enqueueAsyncDelete(host, "TheRegistrar", false);
+ runMapreduce();
+ assertThat(loadByUniqueId(ContactResource.class, "blah2222", clock.nowUtc()))
+ .isEqualTo(contact);
+ assertThat(loadByUniqueId(HostResource.class, "rustles.your.jimmies", clock.nowUtc()))
+ .isEqualTo(host);
+ }
+
+ @Test
+ public void testSuccess_contact_notRequestedByOwner_doesNotGetDeleted() throws Exception {
+ ContactResource contact = persistContactPendingDelete("jane0991");
+ enqueuer.enqueueAsyncDelete(contact, "OtherRegistrar", false);
+ runMapreduce();
+ ContactResource contactAfter =
+ loadByUniqueId(ContactResource.class, "jane0991", clock.nowUtc());
+ assertAboutContacts()
+ .that(contactAfter)
+ .doesNotHaveStatusValue(PENDING_DELETE)
+ .and()
+ .hasDeletionTime(END_OF_TIME);
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfter, CONTACT_DELETE_FAILURE);
+ assertPollMessageFor(
+ historyEntry,
+ "OtherRegistrar",
+ "Can't delete contact jane0991 because it was transferred prior to deletion.");
+ }
+
+ @Test
+ public void testSuccess_contact_notRequestedByOwner_isSuperuser_getsDeleted() throws Exception {
+ ContactResource contact = persistContactWithPii("nate007");
+ enqueuer.enqueueAsyncDelete(contact, "OtherRegistrar", true);
+ runMapreduce();
+ assertThat(loadByUniqueId(ContactResource.class, "nate007", clock.nowUtc())).isNull();
+ ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
+ assertAboutContacts()
+ .that(contactAfterDeletion)
+ .isNotActiveAt(clock.nowUtc())
+ // Note that there will be another history entry of CONTACT_PENDING_DELETE, but this is
+ // added by the flow and not the mapreduce itself.
+ .and()
+ .hasOnlyOneHistoryEntryWhich()
+ .hasType(CONTACT_DELETE);
+ assertAboutContacts()
+ .that(contactAfterDeletion)
+ .hasNullLocalizedPostalInfo()
+ .and()
+ .hasNullInternationalizedPostalInfo()
+ .and()
+ .hasNullEmailAddress()
+ .and()
+ .hasNullVoiceNumber()
+ .and()
+ .hasNullFaxNumber();
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
+ assertPollMessageFor(historyEntry, "OtherRegistrar", "Deleted contact nate007.");
+ }
+
+ @Test
+ public void testFailure_targetResourcesDontExist() throws Exception {
+ ContactResource contactNotSaved = newContactResource("somecontact");
+ HostResource hostNotSaved = newHostResource("a11.blah.foo");
+ enqueuer.enqueueAsyncDelete(contactNotSaved, "TheRegistrar", false);
+ enqueuer.enqueueAsyncDelete(hostNotSaved, "TheRegistrar", false);
+ runMapreduce();
+ }
+
+ @Test
+ public void testFailure_alreadyDeleted() throws Exception {
+ ContactResource contactDeleted = persistDeletedContact("blah1236", clock.nowUtc().minusDays(1));
+ HostResource hostDeleted = persistDeletedHost("a.lim.lop", clock.nowUtc().minusDays(3));
+ enqueuer.enqueueAsyncDelete(contactDeleted, "TheRegistrar", false);
+ enqueuer.enqueueAsyncDelete(hostDeleted, "TheRegistrar", false);
+ runMapreduce();
+ assertThat(ofy().load().entity(contactDeleted).now()).isEqualTo(contactDeleted);
+ assertThat(ofy().load().entity(hostDeleted).now()).isEqualTo(hostDeleted);
+ }
+
+ @Test
+ public void testSuccess_host_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
+ HostResource host = persistHostPendingDelete("ns1.example.tld");
+ persistUsedDomain("example.tld", persistActiveContact("abc456"), host);
+ enqueuer.enqueueAsyncDelete(host, "TheRegistrar", false);
+ runMapreduce();
+ HostResource hostAfter = loadByUniqueId(HostResource.class, "ns1.example.tld", clock.nowUtc());
+ assertAboutHosts()
+ .that(hostAfter)
+ .doesNotHaveStatusValue(PENDING_DELETE)
+ .and()
+ .hasDeletionTime(END_OF_TIME);
+ DomainResource domain = loadByUniqueId(DomainResource.class, "example.tld", clock.nowUtc());
+ assertThat(domain.getNameservers()).contains(Key.create(hostAfter));
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostAfter, HOST_DELETE_FAILURE);
+ assertPollMessageFor(
+ historyEntry,
+ "TheRegistrar",
+ "Can't delete host ns1.example.tld because it is referenced by a domain.");
+ }
+
+ @Test
+ public void testSuccess_host_notReferenced_getsDeleted() throws Exception {
+ HostResource host = persistHostPendingDelete("ns2.example.tld");
+ enqueuer.enqueueAsyncDelete(host, "TheRegistrar", false);
+ runMapreduce();
+ assertThat(loadByUniqueId(HostResource.class, "ns2.example.tld", clock.nowUtc())).isNull();
+ HostResource hostBeforeDeletion =
+ loadByUniqueId(HostResource.class, "ns2.example.tld", clock.nowUtc().minusDays(1));
+ assertAboutHosts()
+ .that(hostBeforeDeletion)
+ .isNotActiveAt(clock.nowUtc())
+ .and()
+ .hasExactlyStatusValues(StatusValue.OK)
+ // Note that there will be another history entry of HOST_PENDING_DELETE, but this is
+ // added by the flow and not the mapreduce itself.
+ .and()
+ .hasOnlyOneHistoryEntryWhich()
+ .hasType(HOST_DELETE);
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
+ assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.");
+ }
+
+ @Test
+ public void testSuccess_host_referencedByDeletedDomain_getsDeleted() throws Exception {
+ HostResource host = persistHostPendingDelete("ns1.example.tld");
+ persistResource(
+ newDomainResource("example.tld")
+ .asBuilder()
+ .setNameservers(ImmutableSet.of(Key.create(host)))
+ .setDeletionTime(clock.nowUtc().minusDays(5))
+ .build());
+ enqueuer.enqueueAsyncDelete(host, "TheRegistrar", false);
+ runMapreduce();
+ assertThat(loadByUniqueId(HostResource.class, "ns1.example.tld", clock.nowUtc())).isNull();
+ HostResource hostBeforeDeletion =
+ loadByUniqueId(HostResource.class, "ns1.example.tld", clock.nowUtc().minusDays(1));
+ assertAboutHosts()
+ .that(hostBeforeDeletion)
+ .isNotActiveAt(clock.nowUtc())
+ .and()
+ .hasExactlyStatusValues(StatusValue.OK)
+ // Note that there will be another history entry of HOST_PENDING_DELETE, but this is
+ // added by the flow and not the mapreduce itself.
+ .and()
+ .hasOnlyOneHistoryEntryWhich()
+ .hasType(HOST_DELETE);
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
+ assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns1.example.tld.");
+ }
+
+ @Test
+ public void testSuccess_subordinateHost_getsDeleted() throws Exception {
+ DomainResource domain =
+ persistResource(
+ newDomainResource("example.tld")
+ .asBuilder()
+ .setSubordinateHosts(ImmutableSet.of("ns2.example.tld"))
+ .build());
+ HostResource host =
+ persistResource(
+ persistHostPendingDelete("ns2.example.tld")
+ .asBuilder()
+ .setSuperordinateDomain(Key.create(domain))
+ .build());
+ enqueuer.enqueueAsyncDelete(host, "TheRegistrar", false);
+ runMapreduce();
+ // Check that the host is deleted as of now.
+ assertThat(loadByUniqueId(HostResource.class, "ns2.example.tld", clock.nowUtc())).isNull();
+ assertNoBillingEvents();
+ assertThat(
+ loadByUniqueId(DomainResource.class, "example.tld", clock.nowUtc())
+ .getSubordinateHosts())
+ .isEmpty();
+ assertDnsTasksEnqueued("ns2.example.tld");
+ HostResource hostBeforeDeletion =
+ loadByUniqueId(HostResource.class, "ns2.example.tld", clock.nowUtc().minusDays(1));
+ assertAboutHosts()
+ .that(hostBeforeDeletion)
+ .isNotActiveAt(clock.nowUtc())
+ .and()
+ .hasExactlyStatusValues(StatusValue.OK)
+ .and()
+ .hasOnlyOneHistoryEntryWhich()
+ .hasType(HOST_DELETE);
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
+ assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.");
+ }
+
+ @Test
+ public void testSuccess_host_notRequestedByOwner_doesNotGetDeleted() throws Exception {
+ HostResource host = persistHostPendingDelete("ns2.example.tld");
+ enqueuer.enqueueAsyncDelete(host, "OtherRegistrar", false);
+ runMapreduce();
+ HostResource hostAfter = loadByUniqueId(HostResource.class, "ns2.example.tld", clock.nowUtc());
+ assertAboutHosts()
+ .that(hostAfter)
+ .doesNotHaveStatusValue(PENDING_DELETE)
+ .and()
+ .hasDeletionTime(END_OF_TIME);
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(host, HOST_DELETE_FAILURE);
+ assertPollMessageFor(
+ historyEntry,
+ "OtherRegistrar",
+ "Can't delete host ns2.example.tld because it was transferred prior to deletion.");
+ }
+
+ @Test
+ public void testSuccess_host_notRequestedByOwner_isSuperuser_getsDeleted() throws Exception {
+ HostResource host = persistHostPendingDelete("ns66.example.tld");
+ enqueuer.enqueueAsyncDelete(host, "OtherRegistrar", true);
+ runMapreduce();
+ assertThat(loadByUniqueId(HostResource.class, "ns66.example.tld", clock.nowUtc())).isNull();
+ HostResource hostBeforeDeletion =
+ loadByUniqueId(HostResource.class, "ns66.example.tld", clock.nowUtc().minusDays(1));
+ assertAboutHosts()
+ .that(hostBeforeDeletion)
+ .isNotActiveAt(clock.nowUtc())
+ .and()
+ .hasExactlyStatusValues(StatusValue.OK)
+ // Note that there will be another history entry of HOST_PENDING_DELETE, but this is
+ // added by the flow and not the mapreduce itself.
+ .and()
+ .hasOnlyOneHistoryEntryWhich()
+ .hasType(HOST_DELETE);
+ HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
+ assertPollMessageFor(historyEntry, "OtherRegistrar", "Deleted host ns66.example.tld.");
+ }
+
+ @Test
+ public void testSuccess_deleteABunchOfContactsAndHosts_butNotSome() throws Exception {
+ ContactResource c1 = persistContactPendingDelete("nsaid54");
+ ContactResource c2 = persistContactPendingDelete("nsaid55");
+ ContactResource c3 = persistContactPendingDelete("nsaid57");
+ HostResource h1 = persistHostPendingDelete("nn5.example.tld");
+ HostResource h2 = persistHostPendingDelete("no.foos.ball");
+ HostResource h3 = persistHostPendingDelete("slime.wars.fun");
+ ContactResource c4 = persistContactPendingDelete("iaminuse6");
+ HostResource h4 = persistHostPendingDelete("used.host.com");
+ persistUsedDomain("usescontactandhost.tld", c4, h4);
+ for (EppResource resource : ImmutableList.of(c1, c2, c3, c4, h1, h2, h3, h4)) {
+ enqueuer.enqueueAsyncDelete(resource, "TheRegistrar", false);
+ }
+ runMapreduce();
+ for (EppResource resource : ImmutableList.of(c1, c2, c3, h1, h2, h3)) {
+ EppResource loaded = ofy().load().entity(resource).now();
+ assertThat(loaded.getDeletionTime()).isLessThan(DateTime.now(UTC));
+ assertThat(loaded.getStatusValues()).doesNotContain(PENDING_DELETE);
+ }
+ for (EppResource resource : ImmutableList.of(c4, h4)) {
+ EppResource loaded = ofy().load().entity(resource).now();
+ assertThat(loaded.getDeletionTime()).isEqualTo(END_OF_TIME);
+ assertThat(loaded.getStatusValues()).doesNotContain(PENDING_DELETE);
+ }
+ }
+
+ private static ContactResource persistContactWithPii(String contactId) {
+ return persistResource(
+ newContactResource(contactId)
+ .asBuilder()
+ .setLocalizedPostalInfo(
+ new PostalInfo.Builder()
+ .setType(PostalInfo.Type.LOCALIZED)
+ .setAddress(
+ new ContactAddress.Builder()
+ .setStreet(ImmutableList.of("123 Grand Ave"))
+ .build())
+ .build())
+ .setInternationalizedPostalInfo(
+ new PostalInfo.Builder()
+ .setType(PostalInfo.Type.INTERNATIONALIZED)
+ .setAddress(
+ new ContactAddress.Builder()
+ .setStreet(ImmutableList.of("123 Avenida Grande"))
+ .build())
+ .build())
+ .setEmailAddress("bob@bob.com")
+ .setVoiceNumber(new ContactPhoneNumber.Builder().setPhoneNumber("555-1212").build())
+ .setFaxNumber(new ContactPhoneNumber.Builder().setPhoneNumber("555-1212").build())
+ .addStatusValue(PENDING_DELETE)
+ .build());
+ }
+
+ /**
+ * Helper method to check that one poll message exists with a given history entry, resource,
+ * client id, and message.
+ */
+ private static void assertPollMessageFor(HistoryEntry historyEntry, String clientId, String msg) {
+ PollMessage.OneTime pollMessage = (OneTime) getOnlyPollMessageForHistoryEntry(historyEntry);
+ assertThat(msg).isEqualTo(pollMessage.getMsg());
+ assertThat(clientId).isEqualTo(pollMessage.getClientId());
+ assertThat(pollMessage.getClientId()).isEqualTo(clientId);
+ }
+
+ private static ContactResource persistContactPendingDelete(String contactId) {
+ return persistResource(
+ newContactResource(contactId).asBuilder().addStatusValue(PENDING_DELETE).build());
+ }
+
+ private static HostResource persistHostPendingDelete(String hostName) {
+ return persistResource(
+ newHostResource(hostName).asBuilder().addStatusValue(PENDING_DELETE).build());
+ }
+
+ private static DomainResource persistUsedDomain(
+ String domainName, ContactResource contact, HostResource host) {
+ return persistResource(
+ newDomainResource(domainName, contact)
+ .asBuilder()
+ .setNameservers(ImmutableSet.of(Key.create(host)))
+ .build());
+ }
+}
diff --git a/javatests/google/registry/testing/AbstractEppResourceSubject.java b/javatests/google/registry/testing/AbstractEppResourceSubject.java
index f97c238ba..3085b6fc8 100644
--- a/javatests/google/registry/testing/AbstractEppResourceSubject.java
+++ b/javatests/google/registry/testing/AbstractEppResourceSubject.java
@@ -236,6 +236,13 @@ abstract class AbstractEppResourceSubject
return andChainer();
}
+ public And isNotActiveAt(DateTime time) {
+ if (isActive(getSubject(), time)) {
+ fail("is not active at " + time);
+ }
+ return andChainer();
+ }
+
protected void failWithBadResults(String dualVerb, Object expected, Object actual) {
failWithBadResults(dualVerb, expected, dualVerb, actual);
}