From 00ea99960a46e7701e554b6b0ca26f7fcc5966b4 Mon Sep 17 00:00:00 2001 From: mcilwain Date: Wed, 14 Sep 2016 13:54:16 -0700 Subject: [PATCH] Improve efficiency of async contact and host deletion with batching This allows handling of N asynchronous deletion requests simultaneously instead of just 1. An accumulation pull queue is used for deletion requests, and the async deletion [] is now fired off whenever that pull queue isn't empty, and processes many tasks at once. This doesn't particularly take more time, because the bulk of the cost of the async delete operation is simply iterating over all DomainBases (which has to happen regardless of how many contacts and hosts are being deleted). ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=133169336 --- java/google/registry/config/ConfigModule.java | 30 +- .../registry/config/RegistryConfig.java | 2 + .../env/alpha/default/WEB-INF/cron.xml | 11 + .../env/common/backend/WEB-INF/web.xml | 11 + .../env/common/default/WEB-INF/queue.xml | 17 +- .../env/crash/default/WEB-INF/cron.xml | 11 + .../env/production/default/WEB-INF/cron.xml | 11 + .../env/sandbox/default/WEB-INF/cron.xml | 11 + java/google/registry/flows/BUILD | 1 + java/google/registry/flows/FlowComponent.java | 8 +- .../flows/async/AsyncFlowEnqueuer.java | 71 +++ .../registry/flows/async/AsyncFlowUtils.java | 1 + .../flows/async/AsyncFlowsModule.java | 14 +- .../async/DeleteContactResourceAction.java | 1 + .../async/DeleteContactsAndHostsAction.java | 394 ++++++++++++ .../flows/async/DeleteEppResourceAction.java | 1 + .../flows/async/DeleteHostResourceAction.java | 1 + .../flows/contact/ContactDeleteFlow.java | 4 + .../registry/flows/host/HostDeleteFlow.java | 6 +- .../backend/BackendRequestComponent.java | 3 + .../DeleteContactsAndHostsActionTest.java | 586 ++++++++++++++++++ .../testing/AbstractEppResourceSubject.java | 7 + 22 files changed, 1186 insertions(+), 16 deletions(-) create mode 100644 java/google/registry/flows/async/AsyncFlowEnqueuer.java create mode 100644 java/google/registry/flows/async/DeleteContactsAndHostsAction.java create mode 100644 javatests/google/registry/flows/async/DeleteContactsAndHostsActionTest.java 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: + *

+ * + *

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