mirror of
https://github.com/google/nomulus.git
synced 2025-05-15 08:57:12 +02:00
Handle already deleted contacts/hosts in async deletion better
This applies lessons learned from the async batch DNS refresh action, in particular making testing more robust. ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=134833523
This commit is contained in:
parent
3d5ab8d068
commit
67695bfc4b
2 changed files with 132 additions and 60 deletions
|
@ -40,10 +40,12 @@ import static java.util.concurrent.TimeUnit.MINUTES;
|
||||||
import com.google.appengine.api.taskqueue.LeaseOptions;
|
import com.google.appengine.api.taskqueue.LeaseOptions;
|
||||||
import com.google.appengine.api.taskqueue.Queue;
|
import com.google.appengine.api.taskqueue.Queue;
|
||||||
import com.google.appengine.api.taskqueue.TaskHandle;
|
import com.google.appengine.api.taskqueue.TaskHandle;
|
||||||
|
import com.google.appengine.api.taskqueue.TransientFailureException;
|
||||||
import com.google.appengine.tools.mapreduce.Mapper;
|
import com.google.appengine.tools.mapreduce.Mapper;
|
||||||
import com.google.appengine.tools.mapreduce.Reducer;
|
import com.google.appengine.tools.mapreduce.Reducer;
|
||||||
import com.google.appengine.tools.mapreduce.ReducerInput;
|
import com.google.appengine.tools.mapreduce.ReducerInput;
|
||||||
import com.google.auto.value.AutoValue;
|
import com.google.auto.value.AutoValue;
|
||||||
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.collect.HashMultiset;
|
import com.google.common.collect.HashMultiset;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
@ -68,8 +70,11 @@ import google.registry.request.Action;
|
||||||
import google.registry.request.Response;
|
import google.registry.request.Response;
|
||||||
import google.registry.util.Clock;
|
import google.registry.util.Clock;
|
||||||
import google.registry.util.FormattingLogger;
|
import google.registry.util.FormattingLogger;
|
||||||
|
import google.registry.util.Retrier;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
import javax.inject.Inject;
|
import javax.inject.Inject;
|
||||||
import javax.inject.Named;
|
import javax.inject.Named;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -100,6 +105,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
@Inject MapreduceRunner mrRunner;
|
@Inject MapreduceRunner mrRunner;
|
||||||
@Inject @Named(QUEUE_ASYNC_DELETE) Queue queue;
|
@Inject @Named(QUEUE_ASYNC_DELETE) Queue queue;
|
||||||
@Inject Response response;
|
@Inject Response response;
|
||||||
|
@Inject Retrier retrier;
|
||||||
@Inject DeleteContactsAndHostsAction() {}
|
@Inject DeleteContactsAndHostsAction() {}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -112,11 +118,19 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
}
|
}
|
||||||
Multiset<String> kindCounts = HashMultiset.create(2);
|
Multiset<String> kindCounts = HashMultiset.create(2);
|
||||||
ImmutableList.Builder<DeletionRequest> builder = new ImmutableList.Builder<>();
|
ImmutableList.Builder<DeletionRequest> builder = new ImmutableList.Builder<>();
|
||||||
|
ImmutableList.Builder<Key<? extends EppResource>> resourceKeys = new ImmutableList.Builder<>();
|
||||||
|
final List<TaskHandle> tasksToDelete = new ArrayList<>();
|
||||||
for (TaskHandle task : tasks) {
|
for (TaskHandle task : tasks) {
|
||||||
try {
|
try {
|
||||||
DeletionRequest deletionRequest = DeletionRequest.createFromTask(task, clock.nowUtc());
|
Optional<DeletionRequest> deletionRequest =
|
||||||
builder.add(deletionRequest);
|
DeletionRequest.createFromTask(task, clock.nowUtc());
|
||||||
kindCounts.add(deletionRequest.key().getKind());
|
if (deletionRequest.isPresent()) {
|
||||||
|
builder.add(deletionRequest.get());
|
||||||
|
resourceKeys.add(deletionRequest.get().key());
|
||||||
|
kindCounts.add(deletionRequest.get().key().getKind());
|
||||||
|
} else {
|
||||||
|
tasksToDelete.add(task);
|
||||||
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.severefmt(
|
logger.severefmt(
|
||||||
e, "Could not parse async deletion request, delaying task for a day: %s", task);
|
e, "Could not parse async deletion request, delaying task for a day: %s", task);
|
||||||
|
@ -125,12 +139,31 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
queue.modifyTaskLease(task, 1L, DAYS);
|
queue.modifyTaskLease(task, 1L, DAYS);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
deleteTasksWithRetry(tasksToDelete);
|
||||||
ImmutableList<DeletionRequest> deletionRequests = builder.build();
|
ImmutableList<DeletionRequest> deletionRequests = builder.build();
|
||||||
|
if (deletionRequests.isEmpty()) {
|
||||||
|
logger.info("No asynchronous deletions to process because all were already handled.");
|
||||||
|
} else {
|
||||||
logger.infofmt(
|
logger.infofmt(
|
||||||
"Processing asynchronous deletion of %d contacts and %d hosts.",
|
"Processing asynchronous deletion of %d contacts and %d hosts: %s",
|
||||||
kindCounts.count(KIND_CONTACT), kindCounts.count(KIND_HOST));
|
kindCounts.count(KIND_CONTACT), kindCounts.count(KIND_HOST), resourceKeys.build());
|
||||||
runMapreduce(deletionRequests);
|
runMapreduce(deletionRequests);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Deletes a list of tasks from the async delete queue using a retrier. */
|
||||||
|
private void deleteTasksWithRetry(final List<TaskHandle> tasks) {
|
||||||
|
if (tasks.isEmpty()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
retrier.callWithRetry(
|
||||||
|
new Callable<Void>() {
|
||||||
|
@Override
|
||||||
|
public Void call() throws Exception {
|
||||||
|
queue.deleteTask(tasks);
|
||||||
|
return null;
|
||||||
|
}}, TransientFailureException.class);
|
||||||
|
}
|
||||||
|
|
||||||
private void runMapreduce(ImmutableList<DeletionRequest> deletionRequests) {
|
private void runMapreduce(ImmutableList<DeletionRequest> deletionRequests) {
|
||||||
try {
|
try {
|
||||||
|
@ -210,6 +243,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
extends Reducer<DeletionRequest, Boolean, Void> {
|
extends Reducer<DeletionRequest, Boolean, Void> {
|
||||||
|
|
||||||
private static final long serialVersionUID = 6569363449285506326L;
|
private static final long serialVersionUID = 6569363449285506326L;
|
||||||
|
private static final DnsQueue dnsQueue = DnsQueue.create();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void reduce(final DeletionRequest deletionRequest, ReducerInput<Boolean> values) {
|
public void reduce(final DeletionRequest deletionRequest, ReducerInput<Boolean> values) {
|
||||||
|
@ -237,10 +271,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
EppResource resource =
|
EppResource resource =
|
||||||
ofy().load().key(deletionRequest.key()).now().cloneProjectedAtTime(now);
|
ofy().load().key(deletionRequest.key()).now().cloneProjectedAtTime(now);
|
||||||
// Double-check transactionally that the resource is still active and in PENDING_DELETE.
|
// Double-check transactionally that the resource is still active and in PENDING_DELETE.
|
||||||
try {
|
if (!doesResourceStateAllowDeletion(resource, now)) {
|
||||||
checkResourceStateAllowsDeletion(resource, now);
|
|
||||||
} catch (IllegalStateException e) {
|
|
||||||
logger.severefmt(e, "State of %s does not allow async deletion", deletionRequest.key());
|
|
||||||
return DeletionResult.create(Type.ERRORED, "");
|
return DeletionResult.create(Type.ERRORED, "");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -317,7 +348,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
} else if (existingResource instanceof HostResource) {
|
} else if (existingResource instanceof HostResource) {
|
||||||
HostResource host = (HostResource) existingResource;
|
HostResource host = (HostResource) existingResource;
|
||||||
if (host.getSuperordinateDomain() != null) {
|
if (host.getSuperordinateDomain() != null) {
|
||||||
DnsQueue.create().addHostRefreshTask(host.getFullyQualifiedHostName());
|
dnsQueue.addHostRefreshTask(host.getFullyQualifiedHostName());
|
||||||
ofy().save().entity(
|
ofy().save().entity(
|
||||||
ofy().load().key(host.getSuperordinateDomain()).now().asBuilder()
|
ofy().load().key(host.getSuperordinateDomain()).now().asBuilder()
|
||||||
.removeSubordinateHost(host.getFullyQualifiedHostName())
|
.removeSubordinateHost(host.getFullyQualifiedHostName())
|
||||||
|
@ -346,25 +377,30 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
abstract boolean isSuperuser();
|
abstract boolean isSuperuser();
|
||||||
abstract TaskHandle task();
|
abstract TaskHandle task();
|
||||||
|
|
||||||
static DeletionRequest createFromTask(TaskHandle task, DateTime now) throws Exception {
|
static Optional<DeletionRequest> createFromTask(TaskHandle task, DateTime now)
|
||||||
|
throws Exception {
|
||||||
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
||||||
Key<EppResource> resourceKey = Key.create(
|
Key<EppResource> resourceKey =
|
||||||
|
Key.create(
|
||||||
checkNotNull(params.get(PARAM_RESOURCE_KEY), "Resource to delete not specified"));
|
checkNotNull(params.get(PARAM_RESOURCE_KEY), "Resource to delete not specified"));
|
||||||
EppResource resource = checkNotNull(
|
EppResource resource =
|
||||||
ofy().load().key(resourceKey).now(), "Resource to delete doesn't exist");
|
checkNotNull(ofy().load().key(resourceKey).now(), "Resource to delete doesn't exist");
|
||||||
checkState(
|
checkState(
|
||||||
resource instanceof ContactResource || resource instanceof HostResource,
|
resource instanceof ContactResource || resource instanceof HostResource,
|
||||||
"Cannot delete a %s via this action",
|
"Cannot delete a %s via this action",
|
||||||
resource.getClass().getSimpleName());
|
resource.getClass().getSimpleName());
|
||||||
checkResourceStateAllowsDeletion(resource, now);
|
if (!doesResourceStateAllowDeletion(resource, now)) {
|
||||||
return new AutoValue_DeleteContactsAndHostsAction_DeletionRequest(
|
return Optional.absent();
|
||||||
|
}
|
||||||
|
return Optional.<DeletionRequest>of(
|
||||||
|
new AutoValue_DeleteContactsAndHostsAction_DeletionRequest(
|
||||||
resourceKey,
|
resourceKey,
|
||||||
resource.getUpdateAutoTimestamp().getTimestamp(),
|
resource.getUpdateAutoTimestamp().getTimestamp(),
|
||||||
checkNotNull(
|
checkNotNull(
|
||||||
params.get(PARAM_REQUESTING_CLIENT_ID), "Requesting client id not specified"),
|
params.get(PARAM_REQUESTING_CLIENT_ID), "Requesting client id not specified"),
|
||||||
Boolean.valueOf(
|
Boolean.valueOf(
|
||||||
checkNotNull(params.get(PARAM_IS_SUPERUSER), "Is superuser not specified")),
|
checkNotNull(params.get(PARAM_IS_SUPERUSER), "Is superuser not specified")),
|
||||||
task);
|
task));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -396,12 +432,16 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static void checkResourceStateAllowsDeletion(EppResource resource, DateTime now) {
|
static boolean doesResourceStateAllowDeletion(EppResource resource, DateTime now) {
|
||||||
Key<EppResource> key = Key.create(resource);
|
Key<EppResource> key = Key.create(resource);
|
||||||
checkState(!isDeleted(resource, now), "Resource %s is already deleted", key);
|
if (isDeleted(resource, now)) {
|
||||||
checkState(
|
logger.warningfmt("Cannot asynchronously delete %s because it is already deleted", key);
|
||||||
resource.getStatusValues().contains(PENDING_DELETE),
|
return false;
|
||||||
"Resource %s is not set as PENDING_DELETE",
|
}
|
||||||
key);
|
if (!resource.getStatusValues().contains(PENDING_DELETE)) {
|
||||||
|
logger.warningfmt("Cannot asynchronously delete %s because it is not in PENDING_DELETE", key);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -14,7 +14,6 @@
|
||||||
|
|
||||||
package google.registry.flows.async;
|
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.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||||
import static com.google.common.collect.Iterables.getOnlyElement;
|
import static com.google.common.collect.Iterables.getOnlyElement;
|
||||||
import static com.google.common.truth.Truth.assertThat;
|
import static com.google.common.truth.Truth.assertThat;
|
||||||
|
@ -45,13 +44,17 @@ import static google.registry.testing.DatastoreHelper.persistDeletedHost;
|
||||||
import static google.registry.testing.DatastoreHelper.persistResource;
|
import static google.registry.testing.DatastoreHelper.persistResource;
|
||||||
import static google.registry.testing.HostResourceSubject.assertAboutHosts;
|
import static google.registry.testing.HostResourceSubject.assertAboutHosts;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
||||||
|
import static google.registry.testing.TaskQueueHelper.assertNoTasksEnqueued;
|
||||||
|
import static google.registry.testing.TaskQueueHelper.assertTasksEnqueued;
|
||||||
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||||
import static org.joda.time.DateTimeZone.UTC;
|
import static org.joda.time.DateTimeZone.UTC;
|
||||||
import static org.joda.time.Duration.millis;
|
import static org.joda.time.Duration.millis;
|
||||||
|
import static org.joda.time.Duration.standardHours;
|
||||||
import static org.joda.time.Duration.standardSeconds;
|
import static org.joda.time.Duration.standardSeconds;
|
||||||
|
|
||||||
import com.google.appengine.api.taskqueue.LeaseOptions;
|
|
||||||
import com.google.appengine.api.taskqueue.QueueFactory;
|
import com.google.appengine.api.taskqueue.QueueFactory;
|
||||||
|
import com.google.appengine.api.taskqueue.TaskOptions;
|
||||||
|
import com.google.appengine.api.taskqueue.TaskOptions.Method;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.collect.FluentIterable;
|
import com.google.common.collect.FluentIterable;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
@ -80,14 +83,13 @@ import google.registry.testing.FakeClock;
|
||||||
import google.registry.testing.FakeResponse;
|
import google.registry.testing.FakeResponse;
|
||||||
import google.registry.testing.FakeSleeper;
|
import google.registry.testing.FakeSleeper;
|
||||||
import google.registry.testing.InjectRule;
|
import google.registry.testing.InjectRule;
|
||||||
|
import google.registry.testing.TaskQueueHelper.TaskMatcher;
|
||||||
import google.registry.testing.mapreduce.MapreduceTestCase;
|
import google.registry.testing.mapreduce.MapreduceTestCase;
|
||||||
import google.registry.util.Retrier;
|
import google.registry.util.Retrier;
|
||||||
import google.registry.util.Sleeper;
|
import google.registry.util.Sleeper;
|
||||||
import google.registry.util.SystemSleeper;
|
import google.registry.util.SystemSleeper;
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -133,20 +135,13 @@ public class DeleteContactsAndHostsActionTest
|
||||||
action.clock = clock;
|
action.clock = clock;
|
||||||
action.mrRunner = new MapreduceRunner(Optional.<Integer>of(5), Optional.<Integer>of(2));
|
action.mrRunner = new MapreduceRunner(Optional.<Integer>of(5), Optional.<Integer>of(2));
|
||||||
action.response = new FakeResponse();
|
action.response = new FakeResponse();
|
||||||
|
action.retrier = new Retrier(new FakeSleeper(clock), 1);
|
||||||
action.queue = getQueue(QUEUE_ASYNC_DELETE);
|
action.queue = getQueue(QUEUE_ASYNC_DELETE);
|
||||||
inject.setStaticField(Ofy.class, "clock", clock);
|
inject.setStaticField(Ofy.class, "clock", clock);
|
||||||
|
|
||||||
createTld("tld");
|
createTld("tld");
|
||||||
clock.advanceOneMilli();
|
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
|
@Test
|
||||||
public void testSuccess_contact_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
|
public void testSuccess_contact_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
|
||||||
ContactResource contact = persistContactPendingDelete("blah8221");
|
ContactResource contact = persistContactPendingDelete("blah8221");
|
||||||
|
@ -169,6 +164,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
historyEntry,
|
historyEntry,
|
||||||
"TheRegistrar",
|
"TheRegistrar",
|
||||||
"Can't delete contact blah8221 because it is referenced by a domain.");
|
"Can't delete contact blah8221 because it is referenced by a domain.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -199,6 +195,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.hasNullFaxNumber();
|
.hasNullFaxNumber();
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact jim919.");
|
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact jim919.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -240,6 +237,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
assertThat(panData.getTrid())
|
assertThat(panData.getTrid())
|
||||||
.isEqualTo(Trid.create("transferClient-trid", "transferServer-trid"));
|
.isEqualTo(Trid.create("transferClient-trid", "transferServer-trid"));
|
||||||
assertThat(panData.getActionResult()).isFalse();
|
assertThat(panData.getActionResult()).isFalse();
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -267,19 +265,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.hasType(CONTACT_DELETE);
|
.hasType(CONTACT_DELETE);
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactBeforeDeletion, CONTACT_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactBeforeDeletion, CONTACT_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact blah1234.");
|
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact blah1234.");
|
||||||
}
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
|
|
||||||
@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(loadByForeignKey(ContactResource.class, "blah2222", clock.nowUtc()))
|
|
||||||
.isEqualTo(contact);
|
|
||||||
assertThat(loadByForeignKey(HostResource.class, "rustles.your.jimmies", clock.nowUtc()))
|
|
||||||
.isEqualTo(host);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -299,6 +285,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
historyEntry,
|
historyEntry,
|
||||||
"OtherRegistrar",
|
"OtherRegistrar",
|
||||||
"Can't delete contact jane0991 because it was transferred prior to deletion.");
|
"Can't delete contact jane0991 because it was transferred prior to deletion.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -329,26 +316,64 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.hasNullFaxNumber();
|
.hasNullFaxNumber();
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "OtherRegistrar", "Deleted contact nate007.");
|
assertPollMessageFor(historyEntry, "OtherRegistrar", "Deleted contact nate007.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFailure_targetResourcesDontExist() throws Exception {
|
public void testSuccess_targetResourcesDontExist_areDelayedForADay() throws Exception {
|
||||||
ContactResource contactNotSaved = newContactResource("somecontact");
|
ContactResource contactNotSaved = newContactResource("somecontact");
|
||||||
HostResource hostNotSaved = newHostResource("a11.blah.foo");
|
HostResource hostNotSaved = newHostResource("a11.blah.foo");
|
||||||
enqueuer.enqueueAsyncDelete(contactNotSaved, "TheRegistrar", false);
|
enqueuer.enqueueAsyncDelete(contactNotSaved, "TheRegistrar", false);
|
||||||
enqueuer.enqueueAsyncDelete(hostNotSaved, "TheRegistrar", false);
|
enqueuer.enqueueAsyncDelete(hostNotSaved, "TheRegistrar", false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
|
String payloadFormat = "resourceKey=%s&requestingClientId=TheRegistrar&isSuperuser=false";
|
||||||
|
assertTasksEnqueued(
|
||||||
|
QUEUE_ASYNC_DELETE,
|
||||||
|
new TaskMatcher()
|
||||||
|
.payload(String.format(payloadFormat, Key.create(contactNotSaved).getString()))
|
||||||
|
.etaDelta(standardHours(23), standardHours(25)),
|
||||||
|
new TaskMatcher()
|
||||||
|
.payload(String.format(payloadFormat, Key.create(hostNotSaved).getString()))
|
||||||
|
.etaDelta(standardHours(23), standardHours(25)));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFailure_alreadyDeleted() throws Exception {
|
public void testSuccess_unparseableTasks_areDelayedForADay() throws Exception {
|
||||||
ContactResource contactDeleted = persistDeletedContact("blah1236", clock.nowUtc().minusDays(1));
|
TaskOptions task =
|
||||||
|
TaskOptions.Builder.withMethod(Method.PULL).param("gobbledygook", "kljhadfgsd9f7gsdfh");
|
||||||
|
getQueue(QUEUE_ASYNC_DELETE).add(task);
|
||||||
|
runMapreduce();
|
||||||
|
assertTasksEnqueued(
|
||||||
|
QUEUE_ASYNC_DELETE,
|
||||||
|
new TaskMatcher()
|
||||||
|
.payload("gobbledygook=kljhadfgsd9f7gsdfh")
|
||||||
|
.etaDelta(standardHours(23), standardHours(25)));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSuccess_resourcesNotInPendingDelete_areSkipped() throws Exception {
|
||||||
|
ContactResource contact = persistActiveContact("blah2222");
|
||||||
|
HostResource host = persistActiveHost("rustles.your.jimmies");
|
||||||
|
enqueuer.enqueueAsyncDelete(contact, "TheRegistrar", false);
|
||||||
|
enqueuer.enqueueAsyncDelete(host, "TheRegistrar", false);
|
||||||
|
runMapreduce();
|
||||||
|
assertThat(loadByForeignKey(ContactResource.class, "blah2222", clock.nowUtc()))
|
||||||
|
.isEqualTo(contact);
|
||||||
|
assertThat(loadByForeignKey(HostResource.class, "rustles.your.jimmies", clock.nowUtc()))
|
||||||
|
.isEqualTo(host);
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSuccess_alreadyDeletedResources_areSkipped() throws Exception {
|
||||||
|
ContactResource contactDeleted = persistDeletedContact("blah1236", clock.nowUtc().minusDays(2));
|
||||||
HostResource hostDeleted = persistDeletedHost("a.lim.lop", clock.nowUtc().minusDays(3));
|
HostResource hostDeleted = persistDeletedHost("a.lim.lop", clock.nowUtc().minusDays(3));
|
||||||
enqueuer.enqueueAsyncDelete(contactDeleted, "TheRegistrar", false);
|
enqueuer.enqueueAsyncDelete(contactDeleted, "TheRegistrar", false);
|
||||||
enqueuer.enqueueAsyncDelete(hostDeleted, "TheRegistrar", false);
|
enqueuer.enqueueAsyncDelete(hostDeleted, "TheRegistrar", false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertThat(ofy().load().entity(contactDeleted).now()).isEqualTo(contactDeleted);
|
assertThat(ofy().load().entity(contactDeleted).now()).isEqualTo(contactDeleted);
|
||||||
assertThat(ofy().load().entity(hostDeleted).now()).isEqualTo(hostDeleted);
|
assertThat(ofy().load().entity(hostDeleted).now()).isEqualTo(hostDeleted);
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -371,6 +396,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
historyEntry,
|
historyEntry,
|
||||||
"TheRegistrar",
|
"TheRegistrar",
|
||||||
"Can't delete host ns1.example.tld because it is referenced by a domain.");
|
"Can't delete host ns1.example.tld because it is referenced by a domain.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -393,6 +419,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.hasType(HOST_DELETE);
|
.hasType(HOST_DELETE);
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.");
|
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -421,6 +448,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.hasType(HOST_DELETE);
|
.hasType(HOST_DELETE);
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns1.example.tld.");
|
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns1.example.tld.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -459,6 +487,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.hasType(HOST_DELETE);
|
.hasType(HOST_DELETE);
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.");
|
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -478,6 +507,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
historyEntry,
|
historyEntry,
|
||||||
"OtherRegistrar",
|
"OtherRegistrar",
|
||||||
"Can't delete host ns2.example.tld because it was transferred prior to deletion.");
|
"Can't delete host ns2.example.tld because it was transferred prior to deletion.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -500,6 +530,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.hasType(HOST_DELETE);
|
.hasType(HOST_DELETE);
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "OtherRegistrar", "Deleted host ns66.example.tld.");
|
assertPollMessageFor(historyEntry, "OtherRegistrar", "Deleted host ns66.example.tld.");
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -527,6 +558,7 @@ public class DeleteContactsAndHostsActionTest
|
||||||
assertThat(loaded.getDeletionTime()).isEqualTo(END_OF_TIME);
|
assertThat(loaded.getDeletionTime()).isEqualTo(END_OF_TIME);
|
||||||
assertThat(loaded.getStatusValues()).doesNotContain(PENDING_DELETE);
|
assertThat(loaded.getStatusValues()).doesNotContain(PENDING_DELETE);
|
||||||
}
|
}
|
||||||
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static ContactResource persistContactWithPii(String contactId) {
|
private static ContactResource persistContactWithPii(String contactId) {
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue