mirror of
https://github.com/google/nomulus.git
synced 2025-05-13 07:57:13 +02:00
Add metrics for async batch operation processing
We want to know how long it's actually taking to process asynchronous contact/host deletions and DNS refreshes on host renames. This adds instrumentation. Five metrics are recorded as follows: * An incrementable metric for each async task processed (split out by type of task and result). * Two event metrics for processing time between when a task is enqueued and when it is processed -- tracked separately for contact/host deletion and DNS refresh on host rename. * Two event metrics for batch size every time the two mapreduces are run (this is usually 0). Tracked separately for contact/host deletion and DNS refresh on host rename. ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=157001310
This commit is contained in:
parent
1adeb57fea
commit
bb67841884
14 changed files with 671 additions and 154 deletions
|
@ -25,6 +25,7 @@ import static google.registry.flows.ResourceFlowUtils.handlePendingTransferOnDel
|
||||||
import static google.registry.flows.ResourceFlowUtils.updateForeignKeyIndexDeletionTime;
|
import static google.registry.flows.ResourceFlowUtils.updateForeignKeyIndexDeletionTime;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_CLIENT_TRANSACTION_ID;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_CLIENT_TRANSACTION_ID;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_IS_SUPERUSER;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_IS_SUPERUSER;
|
||||||
|
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTING_CLIENT_ID;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTING_CLIENT_ID;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_RESOURCE_KEY;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_SERVER_TRANSACTION_ID;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_SERVER_TRANSACTION_ID;
|
||||||
|
@ -42,6 +43,7 @@ import static google.registry.util.PipelineUtils.createJobPath;
|
||||||
import static java.math.RoundingMode.CEILING;
|
import static java.math.RoundingMode.CEILING;
|
||||||
import static java.util.concurrent.TimeUnit.DAYS;
|
import static java.util.concurrent.TimeUnit.DAYS;
|
||||||
import static java.util.concurrent.TimeUnit.MINUTES;
|
import static java.util.concurrent.TimeUnit.MINUTES;
|
||||||
|
import static org.joda.time.DateTimeZone.UTC;
|
||||||
|
|
||||||
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;
|
||||||
|
@ -51,7 +53,8 @@ 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.base.Function;
|
||||||
|
import com.google.common.collect.FluentIterable;
|
||||||
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;
|
||||||
|
@ -61,6 +64,9 @@ import com.googlecode.objectify.Key;
|
||||||
import com.googlecode.objectify.Work;
|
import com.googlecode.objectify.Work;
|
||||||
import google.registry.batch.DeleteContactsAndHostsAction.DeletionResult.Type;
|
import google.registry.batch.DeleteContactsAndHostsAction.DeletionResult.Type;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsQueue;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics.OperationResult;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics.OperationType;
|
||||||
import google.registry.mapreduce.MapreduceRunner;
|
import google.registry.mapreduce.MapreduceRunner;
|
||||||
import google.registry.mapreduce.inputs.EppResourceInputs;
|
import google.registry.mapreduce.inputs.EppResourceInputs;
|
||||||
import google.registry.mapreduce.inputs.NullInput;
|
import google.registry.mapreduce.inputs.NullInput;
|
||||||
|
@ -82,7 +88,9 @@ 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.NonFinalForTesting;
|
||||||
import google.registry.util.Retrier;
|
import google.registry.util.Retrier;
|
||||||
|
import google.registry.util.SystemClock;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -107,6 +115,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
private static final int MAX_REDUCE_SHARDS = 50;
|
private static final int MAX_REDUCE_SHARDS = 50;
|
||||||
private static final int DELETES_PER_SHARD = 5;
|
private static final int DELETES_PER_SHARD = 5;
|
||||||
|
|
||||||
|
@Inject AsyncFlowMetrics asyncFlowMetrics;
|
||||||
@Inject Clock clock;
|
@Inject Clock clock;
|
||||||
@Inject MapreduceRunner mrRunner;
|
@Inject MapreduceRunner mrRunner;
|
||||||
@Inject @Named(QUEUE_ASYNC_DELETE) Queue queue;
|
@Inject @Named(QUEUE_ASYNC_DELETE) Queue queue;
|
||||||
|
@ -119,6 +128,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
LeaseOptions options =
|
LeaseOptions options =
|
||||||
LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(LEASE_MINUTES, MINUTES);
|
LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(LEASE_MINUTES, MINUTES);
|
||||||
List<TaskHandle> tasks = queue.leaseTasks(options);
|
List<TaskHandle> tasks = queue.leaseTasks(options);
|
||||||
|
asyncFlowMetrics.recordContactHostDeletionBatchSize(tasks.size());
|
||||||
if (tasks.isEmpty()) {
|
if (tasks.isEmpty()) {
|
||||||
response.setPayload("No contact/host deletion tasks in pull queue.");
|
response.setPayload("No contact/host deletion tasks in pull queue.");
|
||||||
return;
|
return;
|
||||||
|
@ -126,17 +136,16 @@ 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<>();
|
ImmutableList.Builder<Key<? extends EppResource>> resourceKeys = new ImmutableList.Builder<>();
|
||||||
final List<TaskHandle> tasksToDelete = new ArrayList<>();
|
final List<DeletionRequest> requestsToDelete = new ArrayList<>();
|
||||||
for (TaskHandle task : tasks) {
|
for (TaskHandle task : tasks) {
|
||||||
try {
|
try {
|
||||||
Optional<DeletionRequest> deletionRequest =
|
DeletionRequest deletionRequest = DeletionRequest.createFromTask(task, clock.nowUtc());
|
||||||
DeletionRequest.createFromTask(task, clock.nowUtc());
|
if (deletionRequest.isDeletionAllowed()) {
|
||||||
if (deletionRequest.isPresent()) {
|
builder.add(deletionRequest);
|
||||||
builder.add(deletionRequest.get());
|
resourceKeys.add(deletionRequest.key());
|
||||||
resourceKeys.add(deletionRequest.get().key());
|
kindCounts.add(deletionRequest.key().getKind());
|
||||||
kindCounts.add(deletionRequest.get().key().getKind());
|
|
||||||
} else {
|
} else {
|
||||||
tasksToDelete.add(task);
|
requestsToDelete.add(deletionRequest);
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.severefmt(
|
logger.severefmt(
|
||||||
|
@ -146,7 +155,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
queue.modifyTaskLease(task, 1L, DAYS);
|
queue.modifyTaskLease(task, 1L, DAYS);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
deleteTasksWithRetry(tasksToDelete);
|
deleteStaleTasksWithRetry(requestsToDelete);
|
||||||
ImmutableList<DeletionRequest> deletionRequests = builder.build();
|
ImmutableList<DeletionRequest> deletionRequests = builder.build();
|
||||||
if (deletionRequests.isEmpty()) {
|
if (deletionRequests.isEmpty()) {
|
||||||
logger.info("No asynchronous deletions to process because all were already handled.");
|
logger.info("No asynchronous deletions to process because all were already handled.");
|
||||||
|
@ -159,18 +168,39 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Deletes a list of tasks from the async delete queue using a retrier. */
|
/**
|
||||||
private void deleteTasksWithRetry(final List<TaskHandle> tasks) {
|
* Deletes a list of tasks associated with deletion requests from the async delete queue using a
|
||||||
if (tasks.isEmpty()) {
|
* retrier.
|
||||||
|
*/
|
||||||
|
private void deleteStaleTasksWithRetry(final List<DeletionRequest> deletionRequests) {
|
||||||
|
if (deletionRequests.isEmpty()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
final List<TaskHandle> tasks =
|
||||||
|
FluentIterable.from(deletionRequests)
|
||||||
|
.transform(
|
||||||
|
new Function<DeletionRequest, TaskHandle>() {
|
||||||
|
@Override
|
||||||
|
public TaskHandle apply(DeletionRequest deletionRequest) {
|
||||||
|
return deletionRequest.task();
|
||||||
|
}
|
||||||
|
})
|
||||||
|
.toList();
|
||||||
retrier.callWithRetry(
|
retrier.callWithRetry(
|
||||||
new Callable<Void>() {
|
new Callable<Void>() {
|
||||||
@Override
|
@Override
|
||||||
public Void call() throws Exception {
|
public Void call() throws Exception {
|
||||||
queue.deleteTask(tasks);
|
queue.deleteTask(tasks);
|
||||||
return null;
|
return null;
|
||||||
}}, TransientFailureException.class);
|
}
|
||||||
|
},
|
||||||
|
TransientFailureException.class);
|
||||||
|
for (DeletionRequest deletionRequest : deletionRequests) {
|
||||||
|
asyncFlowMetrics.recordAsyncFlowResult(
|
||||||
|
deletionRequest.getMetricOperationType(),
|
||||||
|
OperationResult.STALE,
|
||||||
|
deletionRequest.requestedTime());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void runMapreduce(ImmutableList<DeletionRequest> deletionRequests) {
|
private void runMapreduce(ImmutableList<DeletionRequest> deletionRequests) {
|
||||||
|
@ -253,6 +283,9 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
private static final long serialVersionUID = 6569363449285506326L;
|
private static final long serialVersionUID = 6569363449285506326L;
|
||||||
private static final DnsQueue dnsQueue = DnsQueue.create();
|
private static final DnsQueue dnsQueue = DnsQueue.create();
|
||||||
|
|
||||||
|
@NonFinalForTesting
|
||||||
|
private static AsyncFlowMetrics asyncFlowMetrics = new AsyncFlowMetrics(new SystemClock());
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void reduce(final DeletionRequest deletionRequest, ReducerInput<Boolean> values) {
|
public void reduce(final DeletionRequest deletionRequest, ReducerInput<Boolean> values) {
|
||||||
final boolean hasNoActiveReferences = !Iterators.contains(values, true);
|
final boolean hasNoActiveReferences = !Iterators.contains(values, true);
|
||||||
|
@ -266,6 +299,10 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
getQueue(QUEUE_ASYNC_DELETE).deleteTask(deletionRequest.task());
|
getQueue(QUEUE_ASYNC_DELETE).deleteTask(deletionRequest.task());
|
||||||
return deletionResult;
|
return deletionResult;
|
||||||
}});
|
}});
|
||||||
|
asyncFlowMetrics.recordAsyncFlowResult(
|
||||||
|
deletionRequest.getMetricOperationType(),
|
||||||
|
result.getMetricOperationResult(),
|
||||||
|
deletionRequest.requestedTime());
|
||||||
String resourceNamePlural = deletionRequest.key().getKind() + "s";
|
String resourceNamePlural = deletionRequest.key().getKind() + "s";
|
||||||
getContext().incrementCounter(result.type().renderCounterText(resourceNamePlural));
|
getContext().incrementCounter(result.type().renderCounterText(resourceNamePlural));
|
||||||
logger.infofmt(
|
logger.infofmt(
|
||||||
|
@ -423,7 +460,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
@AutoValue
|
@AutoValue
|
||||||
abstract static class DeletionRequest implements Serializable {
|
abstract static class DeletionRequest implements Serializable {
|
||||||
|
|
||||||
private static final long serialVersionUID = 5782119100274089088L;
|
private static final long serialVersionUID = -4612618525760839240L;
|
||||||
|
|
||||||
abstract Key<? extends EppResource> key();
|
abstract Key<? extends EppResource> key();
|
||||||
abstract DateTime lastUpdateTime();
|
abstract DateTime lastUpdateTime();
|
||||||
|
@ -440,6 +477,8 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
abstract String serverTransactionId();
|
abstract String serverTransactionId();
|
||||||
|
|
||||||
abstract boolean isSuperuser();
|
abstract boolean isSuperuser();
|
||||||
|
abstract DateTime requestedTime();
|
||||||
|
abstract boolean isDeletionAllowed();
|
||||||
abstract TaskHandle task();
|
abstract TaskHandle task();
|
||||||
|
|
||||||
@AutoValue.Builder
|
@AutoValue.Builder
|
||||||
|
@ -450,11 +489,13 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
abstract Builder setClientTransactionId(String clientTransactionId);
|
abstract Builder setClientTransactionId(String clientTransactionId);
|
||||||
abstract Builder setServerTransactionId(String serverTransactionId);
|
abstract Builder setServerTransactionId(String serverTransactionId);
|
||||||
abstract Builder setIsSuperuser(boolean isSuperuser);
|
abstract Builder setIsSuperuser(boolean isSuperuser);
|
||||||
|
abstract Builder setRequestedTime(DateTime requestedTime);
|
||||||
|
abstract Builder setIsDeletionAllowed(boolean isDeletionAllowed);
|
||||||
abstract Builder setTask(TaskHandle task);
|
abstract Builder setTask(TaskHandle task);
|
||||||
abstract DeletionRequest build();
|
abstract DeletionRequest build();
|
||||||
}
|
}
|
||||||
|
|
||||||
static Optional<DeletionRequest> createFromTask(TaskHandle task, DateTime now)
|
static DeletionRequest createFromTask(TaskHandle task, DateTime now)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
||||||
Key<EppResource> resourceKey =
|
Key<EppResource> resourceKey =
|
||||||
|
@ -466,29 +507,43 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
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());
|
||||||
if (!doesResourceStateAllowDeletion(resource, now)) {
|
return new AutoValue_DeleteContactsAndHostsAction_DeletionRequest.Builder()
|
||||||
return Optional.absent();
|
.setKey(resourceKey)
|
||||||
|
.setLastUpdateTime(resource.getUpdateAutoTimestamp().getTimestamp())
|
||||||
|
.setRequestingClientId(
|
||||||
|
checkNotNull(
|
||||||
|
params.get(PARAM_REQUESTING_CLIENT_ID), "Requesting client id not specified"))
|
||||||
|
.setClientTransactionId(
|
||||||
|
checkNotNull(
|
||||||
|
params.get(PARAM_CLIENT_TRANSACTION_ID), "Client transaction id not specified"))
|
||||||
|
.setServerTransactionId(
|
||||||
|
checkNotNull(
|
||||||
|
params.get(PARAM_SERVER_TRANSACTION_ID), "Server transaction id not specified"))
|
||||||
|
.setIsSuperuser(
|
||||||
|
Boolean.valueOf(
|
||||||
|
checkNotNull(params.get(PARAM_IS_SUPERUSER), "Is superuser not specified")))
|
||||||
|
// TODO(b/38386090): After push is completed and all old tasks are processed, change to:
|
||||||
|
// .setRequestedTime(DateTime.parse(
|
||||||
|
// checkNotNull(params.get(PARAM_REQUESTED_TIME), "Requested time not specified")))
|
||||||
|
.setRequestedTime(
|
||||||
|
(params.containsKey(PARAM_REQUESTED_TIME))
|
||||||
|
? DateTime.parse(params.get(PARAM_REQUESTED_TIME))
|
||||||
|
: DateTime.now(UTC))
|
||||||
|
.setIsDeletionAllowed(doesResourceStateAllowDeletion(resource, now))
|
||||||
|
.setTask(task)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
OperationType getMetricOperationType() {
|
||||||
|
if (key().getKind().equals(KIND_CONTACT)) {
|
||||||
|
return OperationType.CONTACT_DELETE;
|
||||||
|
} else if (key().getKind().equals(KIND_HOST)) {
|
||||||
|
return OperationType.HOST_DELETE;
|
||||||
|
} else {
|
||||||
|
throw new IllegalStateException(
|
||||||
|
String.format(
|
||||||
|
"Cannot determine async operation type for metric for resource %s", key()));
|
||||||
}
|
}
|
||||||
return Optional.<DeletionRequest>of(
|
|
||||||
new AutoValue_DeleteContactsAndHostsAction_DeletionRequest.Builder()
|
|
||||||
.setKey(resourceKey)
|
|
||||||
.setLastUpdateTime(resource.getUpdateAutoTimestamp().getTimestamp())
|
|
||||||
.setRequestingClientId(
|
|
||||||
checkNotNull(
|
|
||||||
params.get(PARAM_REQUESTING_CLIENT_ID), "Requesting client id not specified"))
|
|
||||||
.setClientTransactionId(
|
|
||||||
checkNotNull(
|
|
||||||
params.get(PARAM_CLIENT_TRANSACTION_ID),
|
|
||||||
"Client transaction id not specified"))
|
|
||||||
.setServerTransactionId(
|
|
||||||
checkNotNull(
|
|
||||||
params.get(PARAM_SERVER_TRANSACTION_ID),
|
|
||||||
"Server transaction id not specified"))
|
|
||||||
.setIsSuperuser(
|
|
||||||
Boolean.valueOf(
|
|
||||||
checkNotNull(params.get(PARAM_IS_SUPERUSER), "Is superuser not specified")))
|
|
||||||
.setTask(task)
|
|
||||||
.build());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -497,14 +552,16 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
abstract static class DeletionResult {
|
abstract static class DeletionResult {
|
||||||
|
|
||||||
enum Type {
|
enum Type {
|
||||||
DELETED("%s deleted"),
|
DELETED("%s deleted", OperationResult.SUCCESS),
|
||||||
NOT_DELETED("%s not deleted"),
|
NOT_DELETED("%s not deleted", OperationResult.FAILURE),
|
||||||
ERRORED("%s errored out during deletion");
|
ERRORED("%s errored out during deletion", OperationResult.ERROR);
|
||||||
|
|
||||||
private final String counterFormat;
|
private final String counterFormat;
|
||||||
|
private final OperationResult operationResult;
|
||||||
|
|
||||||
private Type(String counterFormat) {
|
private Type(String counterFormat, OperationResult operationResult) {
|
||||||
this.counterFormat = counterFormat;
|
this.counterFormat = counterFormat;
|
||||||
|
this.operationResult = operationResult;
|
||||||
}
|
}
|
||||||
|
|
||||||
String renderCounterText(String resourceName) {
|
String renderCounterText(String resourceName) {
|
||||||
|
@ -518,6 +575,10 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
||||||
static DeletionResult create(Type type, String pollMessageText) {
|
static DeletionResult create(Type type, String pollMessageText) {
|
||||||
return new AutoValue_DeleteContactsAndHostsAction_DeletionResult(type, pollMessageText);
|
return new AutoValue_DeleteContactsAndHostsAction_DeletionResult(type, pollMessageText);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
OperationResult getMetricOperationResult() {
|
||||||
|
return type().operationResult;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static boolean doesResourceStateAllowDeletion(EppResource resource, DateTime now) {
|
static boolean doesResourceStateAllowDeletion(EppResource resource, DateTime now) {
|
||||||
|
|
|
@ -18,7 +18,9 @@ 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.base.Preconditions.checkNotNull;
|
import static com.google.common.base.Preconditions.checkNotNull;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_HOST_KEY;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_HOST_KEY;
|
||||||
|
import static google.registry.flows.async.AsyncFlowEnqueuer.PARAM_REQUESTED_TIME;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||||
|
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.DNS_REFRESH;
|
||||||
import static google.registry.mapreduce.inputs.EppResourceInputs.createEntityInput;
|
import static google.registry.mapreduce.inputs.EppResourceInputs.createEntityInput;
|
||||||
import static google.registry.model.EppResourceUtils.isActive;
|
import static google.registry.model.EppResourceUtils.isActive;
|
||||||
import static google.registry.model.EppResourceUtils.isDeleted;
|
import static google.registry.model.EppResourceUtils.isDeleted;
|
||||||
|
@ -27,6 +29,7 @@ import static google.registry.util.DateTimeUtils.latestOf;
|
||||||
import static google.registry.util.PipelineUtils.createJobPath;
|
import static google.registry.util.PipelineUtils.createJobPath;
|
||||||
import static java.util.concurrent.TimeUnit.DAYS;
|
import static java.util.concurrent.TimeUnit.DAYS;
|
||||||
import static java.util.concurrent.TimeUnit.MINUTES;
|
import static java.util.concurrent.TimeUnit.MINUTES;
|
||||||
|
import static org.joda.time.DateTimeZone.UTC;
|
||||||
|
|
||||||
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;
|
||||||
|
@ -36,11 +39,14 @@ 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.base.Function;
|
||||||
|
import com.google.common.collect.FluentIterable;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.googlecode.objectify.Key;
|
import com.googlecode.objectify.Key;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsQueue;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics.OperationResult;
|
||||||
import google.registry.mapreduce.MapreduceRunner;
|
import google.registry.mapreduce.MapreduceRunner;
|
||||||
import google.registry.mapreduce.inputs.NullInput;
|
import google.registry.mapreduce.inputs.NullInput;
|
||||||
import google.registry.model.domain.DomainResource;
|
import google.registry.model.domain.DomainResource;
|
||||||
|
@ -49,7 +55,9 @@ 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.NonFinalForTesting;
|
||||||
import google.registry.util.Retrier;
|
import google.registry.util.Retrier;
|
||||||
|
import google.registry.util.SystemClock;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -66,6 +74,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
||||||
private static final long LEASE_MINUTES = 20;
|
private static final long LEASE_MINUTES = 20;
|
||||||
|
|
||||||
|
@Inject AsyncFlowMetrics asyncFlowMetrics;
|
||||||
@Inject Clock clock;
|
@Inject Clock clock;
|
||||||
@Inject MapreduceRunner mrRunner;
|
@Inject MapreduceRunner mrRunner;
|
||||||
@Inject @Named(QUEUE_ASYNC_HOST_RENAME) Queue pullQueue;
|
@Inject @Named(QUEUE_ASYNC_HOST_RENAME) Queue pullQueue;
|
||||||
|
@ -78,24 +87,24 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
LeaseOptions options =
|
LeaseOptions options =
|
||||||
LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(LEASE_MINUTES, MINUTES);
|
LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(LEASE_MINUTES, MINUTES);
|
||||||
List<TaskHandle> tasks = pullQueue.leaseTasks(options);
|
List<TaskHandle> tasks = pullQueue.leaseTasks(options);
|
||||||
|
asyncFlowMetrics.recordDnsRefreshBatchSize(tasks.size());
|
||||||
if (tasks.isEmpty()) {
|
if (tasks.isEmpty()) {
|
||||||
response.setPayload("No DNS refresh on host rename tasks to process in pull queue.");
|
response.setPayload("No DNS refresh on host rename tasks to process in pull queue.");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
ImmutableList.Builder<DnsRefreshRequest> requestsBuilder = new ImmutableList.Builder<>();
|
ImmutableList.Builder<DnsRefreshRequest> requestsBuilder = new ImmutableList.Builder<>();
|
||||||
ImmutableList.Builder<Key<HostResource>> hostKeys = new ImmutableList.Builder<>();
|
ImmutableList.Builder<Key<HostResource>> hostKeys = new ImmutableList.Builder<>();
|
||||||
final List<TaskHandle> tasksToDelete = new ArrayList<>();
|
final List<DnsRefreshRequest> requestsToDelete = new ArrayList<>();
|
||||||
|
|
||||||
for (TaskHandle task : tasks) {
|
for (TaskHandle task : tasks) {
|
||||||
try {
|
try {
|
||||||
Optional<DnsRefreshRequest> request =
|
DnsRefreshRequest request = DnsRefreshRequest.createFromTask(task, clock.nowUtc());
|
||||||
DnsRefreshRequest.createFromTask(task, clock.nowUtc());
|
if (request.isRefreshNeeded()) {
|
||||||
if (request.isPresent()) {
|
requestsBuilder.add(request);
|
||||||
requestsBuilder.add(request.get());
|
hostKeys.add(request.hostKey());
|
||||||
hostKeys.add(request.get().hostKey());
|
|
||||||
} else {
|
} else {
|
||||||
// Skip hosts that are deleted.
|
// Skip hosts that are deleted.
|
||||||
tasksToDelete.add(task);
|
requestsToDelete.add(request);
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.severefmt(
|
logger.severefmt(
|
||||||
|
@ -105,7 +114,8 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
deleteTasksWithRetry(tasksToDelete, pullQueue, retrier);
|
deleteTasksWithRetry(
|
||||||
|
requestsToDelete, pullQueue, asyncFlowMetrics, retrier, OperationResult.STALE);
|
||||||
ImmutableList<DnsRefreshRequest> refreshRequests = requestsBuilder.build();
|
ImmutableList<DnsRefreshRequest> refreshRequests = requestsBuilder.build();
|
||||||
if (refreshRequests.isEmpty()) {
|
if (refreshRequests.isEmpty()) {
|
||||||
logger.info(
|
logger.info(
|
||||||
|
@ -113,12 +123,11 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
response.setPayload("All requested DNS refreshes are on hosts that were since deleted.");
|
response.setPayload("All requested DNS refreshes are on hosts that were since deleted.");
|
||||||
} else {
|
} else {
|
||||||
logger.infofmt("Processing asynchronous DNS refresh for renamed hosts: %s", hostKeys.build());
|
logger.infofmt("Processing asynchronous DNS refresh for renamed hosts: %s", hostKeys.build());
|
||||||
runMapreduce(refreshRequests, tasks);
|
runMapreduce(refreshRequests);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void runMapreduce(
|
private void runMapreduce(ImmutableList<DnsRefreshRequest> refreshRequests) {
|
||||||
ImmutableList<DnsRefreshRequest> refreshRequests, List<TaskHandle> tasks) {
|
|
||||||
try {
|
try {
|
||||||
response.sendJavaScriptRedirect(createJobPath(mrRunner
|
response.sendJavaScriptRedirect(createJobPath(mrRunner
|
||||||
.setJobName("Enqueue DNS refreshes for domains referencing renamed hosts")
|
.setJobName("Enqueue DNS refreshes for domains referencing renamed hosts")
|
||||||
|
@ -126,7 +135,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
.setDefaultReduceShards(1)
|
.setDefaultReduceShards(1)
|
||||||
.runMapreduce(
|
.runMapreduce(
|
||||||
new RefreshDnsOnHostRenameMapper(refreshRequests, retrier),
|
new RefreshDnsOnHostRenameMapper(refreshRequests, retrier),
|
||||||
new RefreshDnsOnHostRenameReducer(tasks, retrier),
|
new RefreshDnsOnHostRenameReducer(refreshRequests, retrier),
|
||||||
// Add an extra NullInput so that the reducer always fires exactly once.
|
// Add an extra NullInput so that the reducer always fires exactly once.
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new NullInput<DomainResource>(), createEntityInput(DomainResource.class)))));
|
new NullInput<DomainResource>(), createEntityInput(DomainResource.class)))));
|
||||||
|
@ -199,26 +208,48 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
|
|
||||||
private static final long serialVersionUID = -2850944843275790412L;
|
private static final long serialVersionUID = -2850944843275790412L;
|
||||||
|
|
||||||
private final Retrier retrier;
|
@NonFinalForTesting
|
||||||
private final List<TaskHandle> tasks;
|
private static AsyncFlowMetrics asyncFlowMetrics = new AsyncFlowMetrics(new SystemClock());
|
||||||
|
|
||||||
RefreshDnsOnHostRenameReducer(List<TaskHandle> tasks, Retrier retrier) {
|
private final Retrier retrier;
|
||||||
this.tasks = tasks;
|
private final List<DnsRefreshRequest> refreshRequests;
|
||||||
|
|
||||||
|
RefreshDnsOnHostRenameReducer(List<DnsRefreshRequest> refreshRequests, Retrier retrier) {
|
||||||
|
this.refreshRequests = refreshRequests;
|
||||||
this.retrier = retrier;
|
this.retrier = retrier;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void reduce(Boolean key, ReducerInput<Boolean> values) {
|
public void reduce(Boolean key, ReducerInput<Boolean> values) {
|
||||||
deleteTasksWithRetry(tasks, getQueue(QUEUE_ASYNC_HOST_RENAME), retrier);
|
deleteTasksWithRetry(
|
||||||
|
refreshRequests,
|
||||||
|
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||||
|
asyncFlowMetrics,
|
||||||
|
retrier,
|
||||||
|
OperationResult.SUCCESS);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Deletes a list of tasks from the given queue using a retrier. */
|
/** Deletes a list of tasks from the given queue using a retrier. */
|
||||||
private static void deleteTasksWithRetry(
|
private static void deleteTasksWithRetry(
|
||||||
final List<TaskHandle> tasks, final Queue queue, Retrier retrier) {
|
final List<DnsRefreshRequest> refreshRequests,
|
||||||
if (tasks.isEmpty()) {
|
final Queue queue,
|
||||||
|
AsyncFlowMetrics asyncFlowMetrics,
|
||||||
|
Retrier retrier,
|
||||||
|
OperationResult result) {
|
||||||
|
if (refreshRequests.isEmpty()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
final List<TaskHandle> tasks =
|
||||||
|
FluentIterable.from(refreshRequests)
|
||||||
|
.transform(
|
||||||
|
new Function<DnsRefreshRequest, TaskHandle>() {
|
||||||
|
@Override
|
||||||
|
public TaskHandle apply(DnsRefreshRequest refreshRequest) {
|
||||||
|
return refreshRequest.task();
|
||||||
|
}
|
||||||
|
})
|
||||||
|
.toList();
|
||||||
retrier.callWithRetry(
|
retrier.callWithRetry(
|
||||||
new Callable<Void>() {
|
new Callable<Void>() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -226,35 +257,60 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
queue.deleteTask(tasks);
|
queue.deleteTask(tasks);
|
||||||
return null;
|
return null;
|
||||||
}}, TransientFailureException.class);
|
}}, TransientFailureException.class);
|
||||||
|
for (DnsRefreshRequest refreshRequest : refreshRequests) {
|
||||||
|
asyncFlowMetrics.recordAsyncFlowResult(DNS_REFRESH, result, refreshRequest.requestedTime());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** A class that encapsulates the values of a request to refresh DNS for a renamed host. */
|
/** A class that encapsulates the values of a request to refresh DNS for a renamed host. */
|
||||||
@AutoValue
|
@AutoValue
|
||||||
abstract static class DnsRefreshRequest implements Serializable {
|
abstract static class DnsRefreshRequest implements Serializable {
|
||||||
|
|
||||||
private static final long serialVersionUID = 2188894914017230887L;
|
private static final long serialVersionUID = 1772812852271288622L;
|
||||||
|
|
||||||
abstract Key<HostResource> hostKey();
|
abstract Key<HostResource> hostKey();
|
||||||
abstract DateTime lastUpdateTime();
|
abstract DateTime lastUpdateTime();
|
||||||
|
abstract DateTime requestedTime();
|
||||||
|
abstract boolean isRefreshNeeded();
|
||||||
|
abstract TaskHandle task();
|
||||||
|
|
||||||
|
@AutoValue.Builder
|
||||||
|
abstract static class Builder {
|
||||||
|
abstract Builder setHostKey(Key<HostResource> hostKey);
|
||||||
|
abstract Builder setLastUpdateTime(DateTime lastUpdateTime);
|
||||||
|
abstract Builder setRequestedTime(DateTime requestedTime);
|
||||||
|
abstract Builder setIsRefreshNeeded(boolean isRefreshNeeded);
|
||||||
|
abstract Builder setTask(TaskHandle task);
|
||||||
|
abstract DnsRefreshRequest build();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a packaged-up {@link DnsRefreshRequest} parsed from a task queue task, or absent if
|
* Returns a packaged-up {@link DnsRefreshRequest} parsed from a task queue task.
|
||||||
* the host specified is already deleted.
|
|
||||||
*/
|
*/
|
||||||
static Optional<DnsRefreshRequest> createFromTask(TaskHandle task, DateTime now)
|
static DnsRefreshRequest createFromTask(TaskHandle task, DateTime now) throws Exception {
|
||||||
throws Exception {
|
|
||||||
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
||||||
Key<HostResource> hostKey =
|
Key<HostResource> hostKey =
|
||||||
Key.create(checkNotNull(params.get(PARAM_HOST_KEY), "Host to refresh not specified"));
|
Key.create(checkNotNull(params.get(PARAM_HOST_KEY), "Host to refresh not specified"));
|
||||||
HostResource host =
|
HostResource host =
|
||||||
checkNotNull(ofy().load().key(hostKey).now(), "Host to refresh doesn't exist");
|
checkNotNull(ofy().load().key(hostKey).now(), "Host to refresh doesn't exist");
|
||||||
if (isDeleted(host, latestOf(now, host.getUpdateAutoTimestamp().getTimestamp()))) {
|
boolean isHostDeleted =
|
||||||
|
isDeleted(host, latestOf(now, host.getUpdateAutoTimestamp().getTimestamp()));
|
||||||
|
if (isHostDeleted) {
|
||||||
logger.infofmt("Host %s is already deleted, not refreshing DNS.", hostKey);
|
logger.infofmt("Host %s is already deleted, not refreshing DNS.", hostKey);
|
||||||
return Optional.absent();
|
|
||||||
}
|
}
|
||||||
return Optional.<DnsRefreshRequest>of(
|
return new AutoValue_RefreshDnsOnHostRenameAction_DnsRefreshRequest.Builder()
|
||||||
new AutoValue_RefreshDnsOnHostRenameAction_DnsRefreshRequest(
|
.setHostKey(hostKey)
|
||||||
hostKey, host.getUpdateAutoTimestamp().getTimestamp()));
|
.setLastUpdateTime(host.getUpdateAutoTimestamp().getTimestamp())
|
||||||
|
// TODO(b/38386090): After push is completed and all old tasks are processed, change to:
|
||||||
|
// .setRequestedTime(DateTime.parse(
|
||||||
|
// checkNotNull(params.get(PARAM_REQUESTED_TIME), "Requested time not specified")))
|
||||||
|
.setRequestedTime(
|
||||||
|
(params.containsKey(PARAM_REQUESTED_TIME))
|
||||||
|
? DateTime.parse(params.get(PARAM_REQUESTED_TIME))
|
||||||
|
: DateTime.now(UTC))
|
||||||
|
.setIsRefreshNeeded(!isHostDeleted)
|
||||||
|
.setTask(task)
|
||||||
|
.build();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import google.registry.util.Retrier;
|
||||||
import java.util.concurrent.Callable;
|
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.Duration;
|
import org.joda.time.Duration;
|
||||||
|
|
||||||
/** Helper class to enqueue tasks for handling asynchronous operations in flows. */
|
/** Helper class to enqueue tasks for handling asynchronous operations in flows. */
|
||||||
|
@ -41,6 +42,7 @@ public final class AsyncFlowEnqueuer {
|
||||||
public static final String PARAM_SERVER_TRANSACTION_ID = "serverTransactionId";
|
public static final String PARAM_SERVER_TRANSACTION_ID = "serverTransactionId";
|
||||||
public static final String PARAM_IS_SUPERUSER = "isSuperuser";
|
public static final String PARAM_IS_SUPERUSER = "isSuperuser";
|
||||||
public static final String PARAM_HOST_KEY = "hostKey";
|
public static final String PARAM_HOST_KEY = "hostKey";
|
||||||
|
public static final String PARAM_REQUESTED_TIME = "requestedTime";
|
||||||
|
|
||||||
/** The task queue names used by async flows. */
|
/** The task queue names used by async flows. */
|
||||||
public static final String QUEUE_ASYNC_DELETE = "async-delete-pull";
|
public static final String QUEUE_ASYNC_DELETE = "async-delete-pull";
|
||||||
|
@ -48,32 +50,31 @@ public final class AsyncFlowEnqueuer {
|
||||||
|
|
||||||
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
||||||
|
|
||||||
@VisibleForTesting
|
private final Duration asyncDeleteDelay;
|
||||||
@Inject
|
private final Queue asyncDeletePullQueue;
|
||||||
@Config("asyncDeleteFlowMapreduceDelay")
|
private final Queue asyncDnsRefreshPullQueue;
|
||||||
public Duration asyncDeleteDelay;
|
private final Retrier retrier;
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
@Inject
|
@Inject
|
||||||
@Named("async-delete-pull")
|
public AsyncFlowEnqueuer(
|
||||||
public Queue asyncDeletePullQueue;
|
@Named(QUEUE_ASYNC_DELETE) Queue asyncDeletePullQueue,
|
||||||
|
@Named(QUEUE_ASYNC_HOST_RENAME) Queue asyncDnsRefreshPullQueue,
|
||||||
@VisibleForTesting
|
@Config("asyncDeleteFlowMapreduceDelay") Duration asyncDeleteDelay,
|
||||||
@Inject
|
Retrier retrier) {
|
||||||
@Named(QUEUE_ASYNC_HOST_RENAME)
|
this.asyncDeletePullQueue = asyncDeletePullQueue;
|
||||||
public Queue asyncDnsRefreshPullQueue;
|
this.asyncDnsRefreshPullQueue = asyncDnsRefreshPullQueue;
|
||||||
|
this.asyncDeleteDelay = asyncDeleteDelay;
|
||||||
@VisibleForTesting
|
this.retrier = retrier;
|
||||||
@Inject
|
}
|
||||||
public Retrier retrier;
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
@Inject
|
|
||||||
public AsyncFlowEnqueuer() {}
|
|
||||||
|
|
||||||
/** Enqueues a task to asynchronously delete a contact or host, by key. */
|
/** Enqueues a task to asynchronously delete a contact or host, by key. */
|
||||||
public void enqueueAsyncDelete(
|
public void enqueueAsyncDelete(
|
||||||
EppResource resourceToDelete, String requestingClientId, Trid trid, boolean isSuperuser) {
|
EppResource resourceToDelete,
|
||||||
|
DateTime now,
|
||||||
|
String requestingClientId,
|
||||||
|
Trid trid,
|
||||||
|
boolean isSuperuser) {
|
||||||
Key<EppResource> resourceKey = Key.create(resourceToDelete);
|
Key<EppResource> resourceKey = Key.create(resourceToDelete);
|
||||||
logger.infofmt(
|
logger.infofmt(
|
||||||
"Enqueuing async deletion of %s on behalf of registrar %s.",
|
"Enqueuing async deletion of %s on behalf of registrar %s.",
|
||||||
|
@ -85,17 +86,20 @@ public final class AsyncFlowEnqueuer {
|
||||||
.param(PARAM_REQUESTING_CLIENT_ID, requestingClientId)
|
.param(PARAM_REQUESTING_CLIENT_ID, requestingClientId)
|
||||||
.param(PARAM_CLIENT_TRANSACTION_ID, trid.getClientTransactionId())
|
.param(PARAM_CLIENT_TRANSACTION_ID, trid.getClientTransactionId())
|
||||||
.param(PARAM_SERVER_TRANSACTION_ID, trid.getServerTransactionId())
|
.param(PARAM_SERVER_TRANSACTION_ID, trid.getServerTransactionId())
|
||||||
.param(PARAM_IS_SUPERUSER, Boolean.toString(isSuperuser));
|
.param(PARAM_IS_SUPERUSER, Boolean.toString(isSuperuser))
|
||||||
|
.param(PARAM_REQUESTED_TIME, now.toString());
|
||||||
addTaskToQueueWithRetry(asyncDeletePullQueue, task);
|
addTaskToQueueWithRetry(asyncDeletePullQueue, task);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Enqueues a task to asynchronously refresh DNS for a renamed host. */
|
/** Enqueues a task to asynchronously refresh DNS for a renamed host. */
|
||||||
public void enqueueAsyncDnsRefresh(HostResource host) {
|
public void enqueueAsyncDnsRefresh(HostResource host, DateTime now) {
|
||||||
Key<HostResource> hostKey = Key.create(host);
|
Key<HostResource> hostKey = Key.create(host);
|
||||||
logger.infofmt("Enqueuing async DNS refresh for renamed host %s.", hostKey);
|
logger.infofmt("Enqueuing async DNS refresh for renamed host %s.", hostKey);
|
||||||
addTaskToQueueWithRetry(
|
addTaskToQueueWithRetry(
|
||||||
asyncDnsRefreshPullQueue,
|
asyncDnsRefreshPullQueue,
|
||||||
TaskOptions.Builder.withMethod(Method.PULL).param(PARAM_HOST_KEY, hostKey.getString()));
|
TaskOptions.Builder.withMethod(Method.PULL)
|
||||||
|
.param(PARAM_HOST_KEY, hostKey.getString())
|
||||||
|
.param(PARAM_REQUESTED_TIME, now.toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
161
java/google/registry/flows/async/AsyncFlowMetrics.java
Normal file
161
java/google/registry/flows/async/AsyncFlowMetrics.java
Normal file
|
@ -0,0 +1,161 @@
|
||||||
|
// Copyright 2017 The Nomulus Authors. All Rights Reserved.
|
||||||
|
//
|
||||||
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
// you may not use this file except in compliance with the License.
|
||||||
|
// You may obtain a copy of the License at
|
||||||
|
//
|
||||||
|
// http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
//
|
||||||
|
// Unless required by applicable law or agreed to in writing, software
|
||||||
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package google.registry.flows.async;
|
||||||
|
|
||||||
|
import static com.google.appengine.api.taskqueue.QueueConstants.maxLeaseCount;
|
||||||
|
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.CONTACT_AND_HOST_DELETE;
|
||||||
|
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.DNS_REFRESH;
|
||||||
|
import static google.registry.monitoring.metrics.EventMetric.DEFAULT_FITTER;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
import google.registry.monitoring.metrics.DistributionFitter;
|
||||||
|
import google.registry.monitoring.metrics.EventMetric;
|
||||||
|
import google.registry.monitoring.metrics.FibonacciFitter;
|
||||||
|
import google.registry.monitoring.metrics.IncrementableMetric;
|
||||||
|
import google.registry.monitoring.metrics.LabelDescriptor;
|
||||||
|
import google.registry.monitoring.metrics.MetricRegistryImpl;
|
||||||
|
import google.registry.util.Clock;
|
||||||
|
import google.registry.util.NonFinalForTesting;
|
||||||
|
import javax.inject.Inject;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Duration;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Instrumentation for async flows (contact/host deletion and DNS refreshes).
|
||||||
|
*
|
||||||
|
* @see AsyncFlowEnqueuer
|
||||||
|
*/
|
||||||
|
public class AsyncFlowMetrics {
|
||||||
|
|
||||||
|
private final Clock clock;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public AsyncFlowMetrics(Clock clock) {
|
||||||
|
this.clock = clock;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A Fibonacci fitter used for bucketing the batch count.
|
||||||
|
*
|
||||||
|
* <p>We use a Fibonacci filter because it provides better resolution at the low end than an
|
||||||
|
* exponential fitter, which is important because most batch sizes are likely to be very low,
|
||||||
|
* despite going up to 1,000 on the high end. Also, the precision is better, as batch size is
|
||||||
|
* inherently an integer, whereas an exponential fitter with an exponent base less than 2 would
|
||||||
|
* have unintuitive boundaries.
|
||||||
|
*/
|
||||||
|
private static final DistributionFitter FITTER_BATCH_SIZE =
|
||||||
|
FibonacciFitter.create(maxLeaseCount());
|
||||||
|
|
||||||
|
private static final ImmutableSet<LabelDescriptor> LABEL_DESCRIPTORS =
|
||||||
|
ImmutableSet.of(
|
||||||
|
LabelDescriptor.create("operation_type", "The type of async flow operation."),
|
||||||
|
LabelDescriptor.create("result", "The result of the async flow operation."));
|
||||||
|
|
||||||
|
@NonFinalForTesting
|
||||||
|
@VisibleForTesting
|
||||||
|
static IncrementableMetric asyncFlowOperationCounts =
|
||||||
|
MetricRegistryImpl.getDefault()
|
||||||
|
.newIncrementableMetric(
|
||||||
|
"/async_flows/operations",
|
||||||
|
"Count of Async Flow Operations",
|
||||||
|
"count",
|
||||||
|
LABEL_DESCRIPTORS);
|
||||||
|
|
||||||
|
@NonFinalForTesting
|
||||||
|
@VisibleForTesting
|
||||||
|
static EventMetric asyncFlowOperationProcessingTime =
|
||||||
|
MetricRegistryImpl.getDefault()
|
||||||
|
.newEventMetric(
|
||||||
|
"/async_flows/processing_time",
|
||||||
|
"Async Flow Processing Time",
|
||||||
|
"milliseconds",
|
||||||
|
LABEL_DESCRIPTORS,
|
||||||
|
DEFAULT_FITTER);
|
||||||
|
|
||||||
|
@NonFinalForTesting
|
||||||
|
@VisibleForTesting
|
||||||
|
static EventMetric asyncFlowBatchSize =
|
||||||
|
MetricRegistryImpl.getDefault()
|
||||||
|
.newEventMetric(
|
||||||
|
"/async_flows/batch_size",
|
||||||
|
"Async Operation Batch Size",
|
||||||
|
"batch size",
|
||||||
|
ImmutableSet.of(
|
||||||
|
LabelDescriptor.create("operation_type", "The type of async flow operation.")),
|
||||||
|
FITTER_BATCH_SIZE);
|
||||||
|
|
||||||
|
/** The type of asynchronous operation. */
|
||||||
|
public enum OperationType {
|
||||||
|
CONTACT_DELETE("contactDelete"),
|
||||||
|
HOST_DELETE("hostDelete"),
|
||||||
|
CONTACT_AND_HOST_DELETE("contactAndHostDelete"),
|
||||||
|
DNS_REFRESH("dnsRefresh");
|
||||||
|
|
||||||
|
private final String metricLabelValue;
|
||||||
|
|
||||||
|
private OperationType(String metricLabelValue) {
|
||||||
|
this.metricLabelValue = metricLabelValue;
|
||||||
|
}
|
||||||
|
|
||||||
|
String getMetricLabelValue() {
|
||||||
|
return metricLabelValue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** The result of an asynchronous operation. */
|
||||||
|
public enum OperationResult {
|
||||||
|
/** The operation processed correctly and the result was success. */
|
||||||
|
SUCCESS("success"),
|
||||||
|
|
||||||
|
/** The operation processed correctly and the result was failure. */
|
||||||
|
FAILURE("failure"),
|
||||||
|
|
||||||
|
/** The operation did not process correctly due to some unexpected error. */
|
||||||
|
ERROR("error"),
|
||||||
|
|
||||||
|
/** The operation was skipped because the request is now stale. */
|
||||||
|
STALE("stale");
|
||||||
|
|
||||||
|
private final String metricLabelValue;
|
||||||
|
|
||||||
|
private OperationResult(String metricLabelValue) {
|
||||||
|
this.metricLabelValue = metricLabelValue;
|
||||||
|
}
|
||||||
|
|
||||||
|
String getMetricLabelValue() {
|
||||||
|
return metricLabelValue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void recordAsyncFlowResult(
|
||||||
|
OperationType operationType, OperationResult operationResult, DateTime whenEnqueued) {
|
||||||
|
asyncFlowOperationCounts.increment(
|
||||||
|
operationType.getMetricLabelValue(), operationResult.getMetricLabelValue());
|
||||||
|
asyncFlowOperationProcessingTime.record(
|
||||||
|
new Duration(whenEnqueued, clock.nowUtc()).getMillis(),
|
||||||
|
operationType.getMetricLabelValue(),
|
||||||
|
operationResult.getMetricLabelValue());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void recordContactHostDeletionBatchSize(long batchSize) {
|
||||||
|
asyncFlowBatchSize.record(
|
||||||
|
Double.valueOf(batchSize), CONTACT_AND_HOST_DELETE.getMetricLabelValue());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void recordDnsRefreshBatchSize(long batchSize) {
|
||||||
|
asyncFlowBatchSize.record(Double.valueOf(batchSize), DNS_REFRESH.getMetricLabelValue());
|
||||||
|
}
|
||||||
|
}
|
|
@ -100,7 +100,8 @@ public final class ContactDeleteFlow implements TransactionalFlow {
|
||||||
if (!isSuperuser) {
|
if (!isSuperuser) {
|
||||||
verifyResourceOwnership(clientId, existingContact);
|
verifyResourceOwnership(clientId, existingContact);
|
||||||
}
|
}
|
||||||
asyncFlowEnqueuer.enqueueAsyncDelete(existingContact, clientId, trid, isSuperuser);
|
asyncFlowEnqueuer.enqueueAsyncDelete(
|
||||||
|
existingContact, ofy().getTransactionTime(), clientId, trid, isSuperuser);
|
||||||
ContactResource newContact =
|
ContactResource newContact =
|
||||||
existingContact.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
|
existingContact.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
|
||||||
historyBuilder
|
historyBuilder
|
||||||
|
|
|
@ -108,7 +108,8 @@ public final class HostDeleteFlow implements TransactionalFlow {
|
||||||
: existingHost;
|
: existingHost;
|
||||||
verifyResourceOwnership(clientId, owningResource);
|
verifyResourceOwnership(clientId, owningResource);
|
||||||
}
|
}
|
||||||
asyncFlowEnqueuer.enqueueAsyncDelete(existingHost, clientId, trid, isSuperuser);
|
asyncFlowEnqueuer.enqueueAsyncDelete(
|
||||||
|
existingHost, ofy().getTransactionTime(), clientId, trid, isSuperuser);
|
||||||
HostResource newHost =
|
HostResource newHost =
|
||||||
existingHost.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
|
existingHost.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
|
||||||
historyBuilder
|
historyBuilder
|
||||||
|
|
|
@ -272,7 +272,7 @@ public final class HostUpdateFlow implements TransactionalFlow {
|
||||||
}
|
}
|
||||||
// We must also enqueue updates for all domains that use this host as their nameserver so
|
// We must also enqueue updates for all domains that use this host as their nameserver so
|
||||||
// that their NS records can be updated to point at the new name.
|
// that their NS records can be updated to point at the new name.
|
||||||
asyncFlowEnqueuer.enqueueAsyncDnsRefresh(existingHost);
|
asyncFlowEnqueuer.enqueueAsyncDnsRefresh(existingHost, ofy().getTransactionTime());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -39,7 +39,7 @@ import org.joda.time.Instant;
|
||||||
* <p>The {@link MutableDistribution} values tracked by this metric can be reset with {@link
|
* <p>The {@link MutableDistribution} values tracked by this metric can be reset with {@link
|
||||||
* EventMetric#reset()}.
|
* EventMetric#reset()}.
|
||||||
*/
|
*/
|
||||||
public final class EventMetric extends AbstractMetric<Distribution> {
|
public class EventMetric extends AbstractMetric<Distribution> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Default {@link DistributionFitter} suitable for latency measurements.
|
* Default {@link DistributionFitter} suitable for latency measurements.
|
||||||
|
@ -86,7 +86,7 @@ public final class EventMetric extends AbstractMetric<Distribution> {
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
final ImmutableList<MetricPoint<Distribution>> getTimestampedValues(Instant endTimestamp) {
|
ImmutableList<MetricPoint<Distribution>> getTimestampedValues(Instant endTimestamp) {
|
||||||
ImmutableList.Builder<MetricPoint<Distribution>> timestampedValues =
|
ImmutableList.Builder<MetricPoint<Distribution>> timestampedValues =
|
||||||
new ImmutableList.Builder<>();
|
new ImmutableList.Builder<>();
|
||||||
|
|
||||||
|
@ -125,7 +125,7 @@ public final class EventMetric extends AbstractMetric<Distribution> {
|
||||||
*
|
*
|
||||||
* <p>The count of {@code labelValues} must be equal to the underlying metric's count of labels.
|
* <p>The count of {@code labelValues} must be equal to the underlying metric's count of labels.
|
||||||
*/
|
*/
|
||||||
public final void record(double sample, String... labelValues) {
|
public void record(double sample, String... labelValues) {
|
||||||
MetricsUtils.checkLabelValuesLength(this, labelValues);
|
MetricsUtils.checkLabelValuesLength(this, labelValues);
|
||||||
|
|
||||||
recordMultiple(sample, 1, Instant.now(), ImmutableList.copyOf(labelValues));
|
recordMultiple(sample, 1, Instant.now(), ImmutableList.copyOf(labelValues));
|
||||||
|
@ -140,14 +140,14 @@ public final class EventMetric extends AbstractMetric<Distribution> {
|
||||||
*
|
*
|
||||||
* <p>The count of {@code labelValues} must be equal to the underlying metric's count of labels.
|
* <p>The count of {@code labelValues} must be equal to the underlying metric's count of labels.
|
||||||
*/
|
*/
|
||||||
public final void record(double sample, int count, String... labelValues) {
|
public void record(double sample, int count, String... labelValues) {
|
||||||
MetricsUtils.checkLabelValuesLength(this, labelValues);
|
MetricsUtils.checkLabelValuesLength(this, labelValues);
|
||||||
|
|
||||||
recordMultiple(sample, count, Instant.now(), ImmutableList.copyOf(labelValues));
|
recordMultiple(sample, count, Instant.now(), ImmutableList.copyOf(labelValues));
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
final void recordMultiple(
|
void recordMultiple(
|
||||||
double sample, int count, Instant startTimestamp, ImmutableList<String> labelValues) {
|
double sample, int count, Instant startTimestamp, ImmutableList<String> labelValues) {
|
||||||
Lock lock = valueLocks.get(labelValues);
|
Lock lock = valueLocks.get(labelValues);
|
||||||
lock.lock();
|
lock.lock();
|
||||||
|
|
|
@ -13,11 +13,12 @@
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
package google.registry.batch;
|
package google.registry.batch;
|
||||||
|
|
||||||
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;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||||
|
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||||
|
import static google.registry.flows.async.AsyncFlowMetrics.OperationResult.STALE;
|
||||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||||
import static google.registry.model.eppcommon.StatusValue.PENDING_DELETE;
|
import static google.registry.model.eppcommon.StatusValue.PENDING_DELETE;
|
||||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||||
|
@ -51,8 +52,10 @@ 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.standardHours;
|
||||||
import static org.joda.time.Duration.standardSeconds;
|
import static org.joda.time.Duration.standardSeconds;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||||
|
|
||||||
import com.google.appengine.api.taskqueue.QueueFactory;
|
|
||||||
import com.google.appengine.api.taskqueue.TaskOptions;
|
import com.google.appengine.api.taskqueue.TaskOptions;
|
||||||
import com.google.appengine.api.taskqueue.TaskOptions.Method;
|
import com.google.appengine.api.taskqueue.TaskOptions.Method;
|
||||||
import com.google.common.collect.FluentIterable;
|
import com.google.common.collect.FluentIterable;
|
||||||
|
@ -60,7 +63,11 @@ import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.googlecode.objectify.Key;
|
import com.googlecode.objectify.Key;
|
||||||
|
import google.registry.batch.DeleteContactsAndHostsAction.DeleteEppResourceReducer;
|
||||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics.OperationResult;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics.OperationType;
|
||||||
import google.registry.model.EppResource;
|
import google.registry.model.EppResource;
|
||||||
import google.registry.model.contact.ContactAddress;
|
import google.registry.model.contact.ContactAddress;
|
||||||
import google.registry.model.contact.ContactPhoneNumber;
|
import google.registry.model.contact.ContactPhoneNumber;
|
||||||
|
@ -129,12 +136,16 @@ public class DeleteContactsAndHostsActionTest
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
enqueuer = new AsyncFlowEnqueuer();
|
enqueuer =
|
||||||
enqueuer.asyncDeleteDelay = Duration.ZERO;
|
new AsyncFlowEnqueuer(
|
||||||
enqueuer.asyncDeletePullQueue = QueueFactory.getQueue(QUEUE_ASYNC_DELETE);
|
getQueue(QUEUE_ASYNC_DELETE),
|
||||||
enqueuer.retrier = new Retrier(new FakeSleeper(clock), 1);
|
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||||
|
Duration.ZERO,
|
||||||
|
new Retrier(new FakeSleeper(clock), 1));
|
||||||
|
AsyncFlowMetrics asyncFlowMetricsMock = mock(AsyncFlowMetrics.class);
|
||||||
action = new DeleteContactsAndHostsAction();
|
action = new DeleteContactsAndHostsAction();
|
||||||
|
action.asyncFlowMetrics = asyncFlowMetricsMock;
|
||||||
|
inject.setStaticField(DeleteEppResourceReducer.class, "asyncFlowMetrics", asyncFlowMetricsMock);
|
||||||
action.clock = clock;
|
action.clock = clock;
|
||||||
action.mrRunner = makeDefaultRunner();
|
action.mrRunner = makeDefaultRunner();
|
||||||
action.response = new FakeResponse();
|
action.response = new FakeResponse();
|
||||||
|
@ -150,8 +161,13 @@ public class DeleteContactsAndHostsActionTest
|
||||||
public void testSuccess_contact_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
|
public void testSuccess_contact_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
|
||||||
ContactResource contact = persistContactPendingDelete("blah8221");
|
ContactResource contact = persistContactPendingDelete("blah8221");
|
||||||
persistResource(newDomainResource("example.tld", contact));
|
persistResource(newDomainResource("example.tld", contact));
|
||||||
|
DateTime timeEnqueued = clock.nowUtc();
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contact,
|
||||||
|
timeEnqueued,
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
ContactResource contactUpdated =
|
ContactResource contactUpdated =
|
||||||
loadByForeignKey(ContactResource.class, "blah8221", clock.nowUtc());
|
loadByForeignKey(ContactResource.class, "blah8221", clock.nowUtc());
|
||||||
|
@ -172,13 +188,22 @@ public class DeleteContactsAndHostsActionTest
|
||||||
false,
|
false,
|
||||||
contact);
|
contact);
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
|
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(OperationType.CONTACT_DELETE, OperationResult.FAILURE, timeEnqueued);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSuccess_contact_notReferenced_getsDeleted_andPiiWipedOut() throws Exception {
|
public void testSuccess_contact_notReferenced_getsDeleted_andPiiWipedOut() throws Exception {
|
||||||
ContactResource contact = persistContactWithPii("jim919");
|
ContactResource contact = persistContactWithPii("jim919");
|
||||||
|
DateTime timeEnqueued = clock.nowUtc();
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contact,
|
||||||
|
timeEnqueued,
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertThat(loadByForeignKey(ContactResource.class, "jim919", clock.nowUtc())).isNull();
|
assertThat(loadByForeignKey(ContactResource.class, "jim919", clock.nowUtc())).isNull();
|
||||||
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
||||||
|
@ -204,6 +229,10 @@ public class DeleteContactsAndHostsActionTest
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact jim919.", true, contact);
|
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact jim919.", true, contact);
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
|
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(OperationType.CONTACT_DELETE, OperationResult.SUCCESS, timeEnqueued);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -211,7 +240,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
throws Exception {
|
throws Exception {
|
||||||
ContactResource contact = persistContactPendingDelete("blah8221");
|
ContactResource contact = persistContactPendingDelete("blah8221");
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contact,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
||||||
assertThat(contactAfterDeletion.getTransferData()).isEqualTo(TransferData.EMPTY);
|
assertThat(contactAfterDeletion.getTransferData()).isEqualTo(TransferData.EMPTY);
|
||||||
|
@ -227,7 +260,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
transferRequestTime.plus(Registry.DEFAULT_TRANSFER_GRACE_PERIOD),
|
transferRequestTime.plus(Registry.DEFAULT_TRANSFER_GRACE_PERIOD),
|
||||||
clock.nowUtc());
|
clock.nowUtc());
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contact,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
// Check that the contact is deleted as of now.
|
// Check that the contact is deleted as of now.
|
||||||
assertThat(loadByForeignKey(ContactResource.class, "sh8013", clock.nowUtc())).isNull();
|
assertThat(loadByForeignKey(ContactResource.class, "sh8013", clock.nowUtc())).isNull();
|
||||||
|
@ -273,7 +310,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.setDeletionTime(clock.nowUtc().minusDays(3))
|
.setDeletionTime(clock.nowUtc().minusDays(3))
|
||||||
.build());
|
.build());
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contactUsed, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contactUsed,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertThat(loadByForeignKey(ContactResource.class, "blah1234", clock.nowUtc())).isNull();
|
assertThat(loadByForeignKey(ContactResource.class, "blah1234", clock.nowUtc())).isNull();
|
||||||
ContactResource contactBeforeDeletion =
|
ContactResource contactBeforeDeletion =
|
||||||
|
@ -298,7 +339,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
public void testSuccess_contact_notRequestedByOwner_doesNotGetDeleted() throws Exception {
|
public void testSuccess_contact_notRequestedByOwner_doesNotGetDeleted() throws Exception {
|
||||||
ContactResource contact = persistContactPendingDelete("jane0991");
|
ContactResource contact = persistContactPendingDelete("jane0991");
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contact, "OtherRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contact,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"OtherRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
ContactResource contactAfter =
|
ContactResource contactAfter =
|
||||||
loadByForeignKey(ContactResource.class, "jane0991", clock.nowUtc());
|
loadByForeignKey(ContactResource.class, "jane0991", clock.nowUtc());
|
||||||
|
@ -321,7 +366,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
public void testSuccess_contact_notRequestedByOwner_isSuperuser_getsDeleted() throws Exception {
|
public void testSuccess_contact_notRequestedByOwner_isSuperuser_getsDeleted() throws Exception {
|
||||||
ContactResource contact = persistContactWithPii("nate007");
|
ContactResource contact = persistContactWithPii("nate007");
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contact, "OtherRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), true);
|
contact,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"OtherRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
true);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertThat(loadByForeignKey(ContactResource.class, "nate007", clock.nowUtc())).isNull();
|
assertThat(loadByForeignKey(ContactResource.class, "nate007", clock.nowUtc())).isNull();
|
||||||
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
||||||
|
@ -353,22 +402,38 @@ public class DeleteContactsAndHostsActionTest
|
||||||
public void testSuccess_targetResourcesDontExist_areDelayedForADay() 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");
|
||||||
|
DateTime timeBeforeRun = clock.nowUtc();
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contactNotSaved, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contactNotSaved,
|
||||||
|
timeBeforeRun,
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
hostNotSaved, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
hostNotSaved,
|
||||||
|
timeBeforeRun,
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
String payloadFormat =
|
|
||||||
"resourceKey=%s&requestingClientId=TheRegistrar&"
|
|
||||||
+ "clientTransactionId=fakeClientTrid&serverTransactionId=fakeServerTrid&isSuperuser=false";
|
|
||||||
assertTasksEnqueued(
|
assertTasksEnqueued(
|
||||||
QUEUE_ASYNC_DELETE,
|
QUEUE_ASYNC_DELETE,
|
||||||
new TaskMatcher()
|
new TaskMatcher()
|
||||||
.payload(String.format(payloadFormat, Key.create(contactNotSaved).getString()))
|
.etaDelta(standardHours(23), standardHours(25))
|
||||||
.etaDelta(standardHours(23), standardHours(25)),
|
.param("resourceKey", Key.create(contactNotSaved).getString())
|
||||||
|
.param("requestingClientId", "TheRegistrar")
|
||||||
|
.param("clientTransactionId", "fakeClientTrid")
|
||||||
|
.param("serverTransactionId", "fakeServerTrid")
|
||||||
|
.param("isSuperuser", "false")
|
||||||
|
.param("requestedTime", timeBeforeRun.toString()),
|
||||||
new TaskMatcher()
|
new TaskMatcher()
|
||||||
.payload(String.format(payloadFormat, Key.create(hostNotSaved).getString()))
|
.etaDelta(standardHours(23), standardHours(25))
|
||||||
.etaDelta(standardHours(23), standardHours(25)));
|
.param("resourceKey", Key.create(hostNotSaved).getString())
|
||||||
|
.param("requestingClientId", "TheRegistrar")
|
||||||
|
.param("clientTransactionId", "fakeClientTrid")
|
||||||
|
.param("serverTransactionId", "fakeServerTrid")
|
||||||
|
.param("isSuperuser", "false")
|
||||||
|
.param("requestedTime", timeBeforeRun.toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -382,22 +447,39 @@ public class DeleteContactsAndHostsActionTest
|
||||||
new TaskMatcher()
|
new TaskMatcher()
|
||||||
.payload("gobbledygook=kljhadfgsd9f7gsdfh")
|
.payload("gobbledygook=kljhadfgsd9f7gsdfh")
|
||||||
.etaDelta(standardHours(23), standardHours(25)));
|
.etaDelta(standardHours(23), standardHours(25)));
|
||||||
|
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSuccess_resourcesNotInPendingDelete_areSkipped() throws Exception {
|
public void testSuccess_resourcesNotInPendingDelete_areSkipped() throws Exception {
|
||||||
ContactResource contact = persistActiveContact("blah2222");
|
ContactResource contact = persistActiveContact("blah2222");
|
||||||
HostResource host = persistActiveHost("rustles.your.jimmies");
|
HostResource host = persistActiveHost("rustles.your.jimmies");
|
||||||
|
DateTime timeEnqueued = clock.nowUtc();
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contact,
|
||||||
|
timeEnqueued,
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
host,
|
||||||
|
timeEnqueued,
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertThat(loadByForeignKey(ContactResource.class, "blah2222", clock.nowUtc()))
|
assertThat(loadByForeignKey(ContactResource.class, "blah2222", clock.nowUtc()))
|
||||||
.isEqualTo(contact);
|
.isEqualTo(contact);
|
||||||
assertThat(loadByForeignKey(HostResource.class, "rustles.your.jimmies", clock.nowUtc()))
|
assertThat(loadByForeignKey(HostResource.class, "rustles.your.jimmies", clock.nowUtc()))
|
||||||
.isEqualTo(host);
|
.isEqualTo(host);
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
|
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(2L);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(OperationType.CONTACT_DELETE, STALE, timeEnqueued);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(OperationType.HOST_DELETE, STALE, timeEnqueued);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -405,9 +487,17 @@ public class DeleteContactsAndHostsActionTest
|
||||||
ContactResource contactDeleted = persistDeletedContact("blah1236", clock.nowUtc().minusDays(2));
|
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(
|
enqueuer.enqueueAsyncDelete(
|
||||||
contactDeleted, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
contactDeleted,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
hostDeleted, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
hostDeleted,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
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);
|
||||||
|
@ -418,8 +508,13 @@ public class DeleteContactsAndHostsActionTest
|
||||||
public void testSuccess_host_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
|
public void testSuccess_host_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
|
||||||
HostResource host = persistHostPendingDelete("ns1.example.tld");
|
HostResource host = persistHostPendingDelete("ns1.example.tld");
|
||||||
persistUsedDomain("example.tld", persistActiveContact("abc456"), host);
|
persistUsedDomain("example.tld", persistActiveContact("abc456"), host);
|
||||||
|
DateTime timeEnqueued = clock.nowUtc();
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
host,
|
||||||
|
timeEnqueued,
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
HostResource hostAfter =
|
HostResource hostAfter =
|
||||||
loadByForeignKey(HostResource.class, "ns1.example.tld", clock.nowUtc());
|
loadByForeignKey(HostResource.class, "ns1.example.tld", clock.nowUtc());
|
||||||
|
@ -438,13 +533,22 @@ public class DeleteContactsAndHostsActionTest
|
||||||
false,
|
false,
|
||||||
host);
|
host);
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
|
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(OperationType.HOST_DELETE, OperationResult.FAILURE, timeEnqueued);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSuccess_host_notReferenced_getsDeleted() throws Exception {
|
public void testSuccess_host_notReferenced_getsDeleted() throws Exception {
|
||||||
HostResource host = persistHostPendingDelete("ns2.example.tld");
|
HostResource host = persistHostPendingDelete("ns2.example.tld");
|
||||||
|
DateTime timeEnqueued = clock.nowUtc();
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
host,
|
||||||
|
timeEnqueued,
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertThat(loadByForeignKey(HostResource.class, "ns2.example.tld", clock.nowUtc())).isNull();
|
assertThat(loadByForeignKey(HostResource.class, "ns2.example.tld", clock.nowUtc())).isNull();
|
||||||
HostResource hostBeforeDeletion =
|
HostResource hostBeforeDeletion =
|
||||||
|
@ -462,6 +566,10 @@ public class DeleteContactsAndHostsActionTest
|
||||||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
||||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.", true, host);
|
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.", true, host);
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||||
|
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(OperationType.HOST_DELETE, OperationResult.SUCCESS, timeEnqueued);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -474,7 +582,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.setDeletionTime(clock.nowUtc().minusDays(5))
|
.setDeletionTime(clock.nowUtc().minusDays(5))
|
||||||
.build());
|
.build());
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
host,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertThat(loadByForeignKey(HostResource.class, "ns1.example.tld", clock.nowUtc())).isNull();
|
assertThat(loadByForeignKey(HostResource.class, "ns1.example.tld", clock.nowUtc())).isNull();
|
||||||
HostResource hostBeforeDeletion =
|
HostResource hostBeforeDeletion =
|
||||||
|
@ -509,7 +621,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
.setSuperordinateDomain(Key.create(domain))
|
.setSuperordinateDomain(Key.create(domain))
|
||||||
.build());
|
.build());
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
host,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
// Check that the host is deleted as of now.
|
// Check that the host is deleted as of now.
|
||||||
assertThat(loadByForeignKey(HostResource.class, "ns2.example.tld", clock.nowUtc())).isNull();
|
assertThat(loadByForeignKey(HostResource.class, "ns2.example.tld", clock.nowUtc())).isNull();
|
||||||
|
@ -538,7 +654,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
public void testSuccess_host_notRequestedByOwner_doesNotGetDeleted() throws Exception {
|
public void testSuccess_host_notRequestedByOwner_doesNotGetDeleted() throws Exception {
|
||||||
HostResource host = persistHostPendingDelete("ns2.example.tld");
|
HostResource host = persistHostPendingDelete("ns2.example.tld");
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
host, "OtherRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
host,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"OtherRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
HostResource hostAfter =
|
HostResource hostAfter =
|
||||||
loadByForeignKey(HostResource.class, "ns2.example.tld", clock.nowUtc());
|
loadByForeignKey(HostResource.class, "ns2.example.tld", clock.nowUtc());
|
||||||
|
@ -561,7 +681,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
public void testSuccess_host_notRequestedByOwner_isSuperuser_getsDeleted() throws Exception {
|
public void testSuccess_host_notRequestedByOwner_isSuperuser_getsDeleted() throws Exception {
|
||||||
HostResource host = persistHostPendingDelete("ns66.example.tld");
|
HostResource host = persistHostPendingDelete("ns66.example.tld");
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
host, "OtherRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), true);
|
host,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"OtherRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
true);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertThat(loadByForeignKey(HostResource.class, "ns66.example.tld", clock.nowUtc())).isNull();
|
assertThat(loadByForeignKey(HostResource.class, "ns66.example.tld", clock.nowUtc())).isNull();
|
||||||
HostResource hostBeforeDeletion =
|
HostResource hostBeforeDeletion =
|
||||||
|
@ -595,7 +719,11 @@ public class DeleteContactsAndHostsActionTest
|
||||||
persistUsedDomain("usescontactandhost.tld", c4, h4);
|
persistUsedDomain("usescontactandhost.tld", c4, h4);
|
||||||
for (EppResource resource : ImmutableList.<EppResource>of(c1, c2, c3, c4, h1, h2, h3, h4)) {
|
for (EppResource resource : ImmutableList.<EppResource>of(c1, c2, c3, c4, h1, h2, h3, h4)) {
|
||||||
enqueuer.enqueueAsyncDelete(
|
enqueuer.enqueueAsyncDelete(
|
||||||
resource, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
resource,
|
||||||
|
clock.nowUtc(),
|
||||||
|
"TheRegistrar",
|
||||||
|
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||||
|
false);
|
||||||
}
|
}
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
for (EppResource resource : ImmutableList.<EppResource>of(c1, c2, c3, h1, h2, h3)) {
|
for (EppResource resource : ImmutableList.<EppResource>of(c1, c2, c3, h1, h2, h3)) {
|
||||||
|
|
|
@ -15,7 +15,9 @@
|
||||||
package google.registry.batch;
|
package google.registry.batch;
|
||||||
|
|
||||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||||
|
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_DELETE;
|
||||||
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
import static google.registry.flows.async.AsyncFlowEnqueuer.QUEUE_ASYNC_HOST_RENAME;
|
||||||
|
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.DNS_REFRESH;
|
||||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||||
import static google.registry.testing.DatastoreHelper.createTld;
|
import static google.registry.testing.DatastoreHelper.createTld;
|
||||||
import static google.registry.testing.DatastoreHelper.newDomainApplication;
|
import static google.registry.testing.DatastoreHelper.newDomainApplication;
|
||||||
|
@ -32,10 +34,17 @@ import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||||
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.standardHours;
|
||||||
import static org.joda.time.Duration.standardSeconds;
|
import static org.joda.time.Duration.standardSeconds;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.times;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.googlecode.objectify.Key;
|
import com.googlecode.objectify.Key;
|
||||||
|
import google.registry.batch.RefreshDnsOnHostRenameAction.RefreshDnsOnHostRenameReducer;
|
||||||
import google.registry.flows.async.AsyncFlowEnqueuer;
|
import google.registry.flows.async.AsyncFlowEnqueuer;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics;
|
||||||
|
import google.registry.flows.async.AsyncFlowMetrics.OperationResult;
|
||||||
import google.registry.model.host.HostResource;
|
import google.registry.model.host.HostResource;
|
||||||
import google.registry.testing.ExceptionRule;
|
import google.registry.testing.ExceptionRule;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
|
@ -48,6 +57,7 @@ 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 org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Duration;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -71,12 +81,17 @@ public class RefreshDnsOnHostRenameActionTest
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
createTld("tld");
|
createTld("tld");
|
||||||
|
enqueuer =
|
||||||
enqueuer = new AsyncFlowEnqueuer();
|
new AsyncFlowEnqueuer(
|
||||||
enqueuer.asyncDnsRefreshPullQueue = getQueue(QUEUE_ASYNC_HOST_RENAME);
|
getQueue(QUEUE_ASYNC_DELETE),
|
||||||
enqueuer.retrier = new Retrier(new FakeSleeper(clock), 1);
|
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||||
|
Duration.ZERO,
|
||||||
|
new Retrier(new FakeSleeper(clock), 1));
|
||||||
|
AsyncFlowMetrics asyncFlowMetricsMock = mock(AsyncFlowMetrics.class);
|
||||||
action = new RefreshDnsOnHostRenameAction();
|
action = new RefreshDnsOnHostRenameAction();
|
||||||
|
action.asyncFlowMetrics = asyncFlowMetricsMock;
|
||||||
|
inject.setStaticField(
|
||||||
|
RefreshDnsOnHostRenameReducer.class, "asyncFlowMetrics", asyncFlowMetricsMock);
|
||||||
action.clock = clock;
|
action.clock = clock;
|
||||||
action.mrRunner = makeDefaultRunner();
|
action.mrRunner = makeDefaultRunner();
|
||||||
action.pullQueue = getQueue(QUEUE_ASYNC_HOST_RENAME);
|
action.pullQueue = getQueue(QUEUE_ASYNC_HOST_RENAME);
|
||||||
|
@ -108,11 +123,15 @@ public class RefreshDnsOnHostRenameActionTest
|
||||||
persistResource(newDomainResource("example.tld", host));
|
persistResource(newDomainResource("example.tld", host));
|
||||||
persistResource(newDomainResource("otherexample.tld", host));
|
persistResource(newDomainResource("otherexample.tld", host));
|
||||||
persistResource(newDomainResource("untouched.tld", persistActiveHost("ns2.example.tld")));
|
persistResource(newDomainResource("untouched.tld", persistActiveHost("ns2.example.tld")));
|
||||||
|
DateTime timeEnqueued = clock.nowUtc();
|
||||||
enqueuer.enqueueAsyncDnsRefresh(host);
|
enqueuer.enqueueAsyncDnsRefresh(host, timeEnqueued);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertDnsTasksEnqueued("example.tld", "otherexample.tld");
|
assertDnsTasksEnqueued("example.tld", "otherexample.tld");
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||||
|
verify(action.asyncFlowMetrics).recordDnsRefreshBatchSize(1L);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.SUCCESS, timeEnqueued);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -123,23 +142,35 @@ public class RefreshDnsOnHostRenameActionTest
|
||||||
persistResource(newDomainResource("example1.tld", host1));
|
persistResource(newDomainResource("example1.tld", host1));
|
||||||
persistResource(newDomainResource("example2.tld", host2));
|
persistResource(newDomainResource("example2.tld", host2));
|
||||||
persistResource(newDomainResource("example3.tld", host3));
|
persistResource(newDomainResource("example3.tld", host3));
|
||||||
|
DateTime timeEnqueued = clock.nowUtc();
|
||||||
enqueuer.enqueueAsyncDnsRefresh(host1);
|
DateTime laterTimeEnqueued = timeEnqueued.plus(standardSeconds(10));
|
||||||
enqueuer.enqueueAsyncDnsRefresh(host2);
|
enqueuer.enqueueAsyncDnsRefresh(host1, timeEnqueued);
|
||||||
enqueuer.enqueueAsyncDnsRefresh(host3);
|
enqueuer.enqueueAsyncDnsRefresh(host2, timeEnqueued);
|
||||||
|
enqueuer.enqueueAsyncDnsRefresh(host3, laterTimeEnqueued);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertDnsTasksEnqueued("example1.tld", "example2.tld", "example3.tld");
|
assertDnsTasksEnqueued("example1.tld", "example2.tld", "example3.tld");
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||||
|
verify(action.asyncFlowMetrics).recordDnsRefreshBatchSize(3L);
|
||||||
|
verify(action.asyncFlowMetrics, times(2))
|
||||||
|
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.SUCCESS, timeEnqueued);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.SUCCESS, laterTimeEnqueued);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSuccess_deletedHost_doesntTriggerDnsRefresh() throws Exception {
|
public void testSuccess_deletedHost_doesntTriggerDnsRefresh() throws Exception {
|
||||||
HostResource host = persistDeletedHost("ns11.fakesss.tld", clock.nowUtc().minusDays(4));
|
HostResource host = persistDeletedHost("ns11.fakesss.tld", clock.nowUtc().minusDays(4));
|
||||||
persistResource(newDomainResource("example1.tld", host));
|
persistResource(newDomainResource("example1.tld", host));
|
||||||
enqueuer.enqueueAsyncDnsRefresh(host);
|
DateTime timeEnqueued = clock.nowUtc();
|
||||||
|
enqueuer.enqueueAsyncDnsRefresh(host, timeEnqueued);
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertNoDnsTasksEnqueued();
|
assertNoDnsTasksEnqueued();
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||||
|
verify(action.asyncFlowMetrics).recordDnsRefreshBatchSize(1L);
|
||||||
|
verify(action.asyncFlowMetrics)
|
||||||
|
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.STALE, timeEnqueued);
|
||||||
|
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -150,7 +181,7 @@ public class RefreshDnsOnHostRenameActionTest
|
||||||
.asBuilder()
|
.asBuilder()
|
||||||
.setDeletionTime(START_OF_TIME)
|
.setDeletionTime(START_OF_TIME)
|
||||||
.build());
|
.build());
|
||||||
enqueuer.enqueueAsyncDnsRefresh(renamedHost);
|
enqueuer.enqueueAsyncDnsRefresh(renamedHost, clock.nowUtc());
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertNoDnsTasksEnqueued();
|
assertNoDnsTasksEnqueued();
|
||||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||||
|
@ -159,7 +190,7 @@ public class RefreshDnsOnHostRenameActionTest
|
||||||
@Test
|
@Test
|
||||||
public void testRun_hostDoesntExist_delaysTask() throws Exception {
|
public void testRun_hostDoesntExist_delaysTask() throws Exception {
|
||||||
HostResource host = newHostResource("ns1.example.tld");
|
HostResource host = newHostResource("ns1.example.tld");
|
||||||
enqueuer.enqueueAsyncDnsRefresh(host);
|
enqueuer.enqueueAsyncDnsRefresh(host, clock.nowUtc());
|
||||||
runMapreduce();
|
runMapreduce();
|
||||||
assertNoDnsTasksEnqueued();
|
assertNoDnsTasksEnqueued();
|
||||||
assertTasksEnqueued(
|
assertTasksEnqueued(
|
||||||
|
|
|
@ -28,6 +28,7 @@ java_library(
|
||||||
"//java/google/registry/dns",
|
"//java/google/registry/dns",
|
||||||
"//java/google/registry/flows",
|
"//java/google/registry/flows",
|
||||||
"//java/google/registry/model",
|
"//java/google/registry/model",
|
||||||
|
"//java/google/registry/monitoring/metrics",
|
||||||
"//java/google/registry/monitoring/whitebox",
|
"//java/google/registry/monitoring/whitebox",
|
||||||
"//java/google/registry/pricing",
|
"//java/google/registry/pricing",
|
||||||
"//java/google/registry/request",
|
"//java/google/registry/request",
|
||||||
|
|
|
@ -156,7 +156,7 @@ public abstract class ResourceFlowTestCase<F extends Flow, R extends EppResource
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Asserts the presence of a single enqueued async contact or host deletion */
|
/** Asserts the presence of a single enqueued async contact or host deletion */
|
||||||
protected static <T extends EppResource> void assertAsyncDeletionTaskEnqueued(
|
protected <T extends EppResource> void assertAsyncDeletionTaskEnqueued(
|
||||||
T resource, String requestingClientId, Trid trid, boolean isSuperuser) throws Exception {
|
T resource, String requestingClientId, Trid trid, boolean isSuperuser) throws Exception {
|
||||||
assertTasksEnqueued(
|
assertTasksEnqueued(
|
||||||
"async-delete-pull",
|
"async-delete-pull",
|
||||||
|
@ -166,7 +166,8 @@ public abstract class ResourceFlowTestCase<F extends Flow, R extends EppResource
|
||||||
.param("requestingClientId", requestingClientId)
|
.param("requestingClientId", requestingClientId)
|
||||||
.param("clientTransactionId", trid.getClientTransactionId())
|
.param("clientTransactionId", trid.getClientTransactionId())
|
||||||
.param("serverTransactionId", trid.getServerTransactionId())
|
.param("serverTransactionId", trid.getServerTransactionId())
|
||||||
.param("isSuperuser", Boolean.toString(isSuperuser)));
|
.param("isSuperuser", Boolean.toString(isSuperuser))
|
||||||
|
.param("requestedTime", clock.nowUtc().toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,70 @@
|
||||||
|
// Copyright 2017 The Nomulus Authors. All Rights Reserved.
|
||||||
|
//
|
||||||
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
// you may not use this file except in compliance with the License.
|
||||||
|
// You may obtain a copy of the License at
|
||||||
|
//
|
||||||
|
// http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
//
|
||||||
|
// Unless required by applicable law or agreed to in writing, software
|
||||||
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package google.registry.flows.async;
|
||||||
|
|
||||||
|
import static google.registry.flows.async.AsyncFlowMetrics.OperationResult.SUCCESS;
|
||||||
|
import static google.registry.flows.async.AsyncFlowMetrics.OperationType.CONTACT_AND_HOST_DELETE;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||||
|
|
||||||
|
import google.registry.monitoring.metrics.EventMetric;
|
||||||
|
import google.registry.monitoring.metrics.IncrementableMetric;
|
||||||
|
import google.registry.testing.FakeClock;
|
||||||
|
import google.registry.testing.InjectRule;
|
||||||
|
import google.registry.testing.ShardableTestCase;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.JUnit4;
|
||||||
|
|
||||||
|
/** Unit tests for {@link AsyncFlowMetrics}. */
|
||||||
|
@RunWith(JUnit4.class)
|
||||||
|
public class AsyncFlowMetricsTest extends ShardableTestCase {
|
||||||
|
|
||||||
|
@Rule public final InjectRule inject = new InjectRule();
|
||||||
|
|
||||||
|
private final IncrementableMetric asyncFlowOperationCounts = mock(IncrementableMetric.class);
|
||||||
|
private final EventMetric asyncFlowOperationProcessingTime = mock(EventMetric.class);
|
||||||
|
private final EventMetric asyncFlowBatchSize = mock(EventMetric.class);
|
||||||
|
private AsyncFlowMetrics asyncFlowMetrics;
|
||||||
|
private FakeClock clock;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
clock = new FakeClock();
|
||||||
|
asyncFlowMetrics = new AsyncFlowMetrics(clock);
|
||||||
|
inject.setStaticField(
|
||||||
|
AsyncFlowMetrics.class, "asyncFlowOperationCounts", asyncFlowOperationCounts);
|
||||||
|
inject.setStaticField(
|
||||||
|
AsyncFlowMetrics.class,
|
||||||
|
"asyncFlowOperationProcessingTime",
|
||||||
|
asyncFlowOperationProcessingTime);
|
||||||
|
inject.setStaticField(AsyncFlowMetrics.class, "asyncFlowBatchSize", asyncFlowBatchSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRecordAsyncFlowResult_calculatesDurationMillisCorrectly() {
|
||||||
|
asyncFlowMetrics.recordAsyncFlowResult(
|
||||||
|
CONTACT_AND_HOST_DELETE,
|
||||||
|
SUCCESS,
|
||||||
|
clock.nowUtc().minusMinutes(10).minusSeconds(5).minusMillis(566));
|
||||||
|
verify(asyncFlowOperationCounts).increment("contactAndHostDelete", "success");
|
||||||
|
verify(asyncFlowOperationProcessingTime).record(605566.0, "contactAndHostDelete", "success");
|
||||||
|
verifyNoMoreInteractions(asyncFlowOperationCounts);
|
||||||
|
verifyNoMoreInteractions(asyncFlowOperationProcessingTime);
|
||||||
|
}
|
||||||
|
}
|
|
@ -192,7 +192,9 @@ public class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Hos
|
||||||
// Task enqueued to change the NS record of the referencing domain via mapreduce.
|
// Task enqueued to change the NS record of the referencing domain via mapreduce.
|
||||||
assertTasksEnqueued(
|
assertTasksEnqueued(
|
||||||
QUEUE_ASYNC_HOST_RENAME,
|
QUEUE_ASYNC_HOST_RENAME,
|
||||||
new TaskMatcher().param("hostKey", Key.create(renamedHost).getString()));
|
new TaskMatcher()
|
||||||
|
.param("hostKey", Key.create(renamedHost).getString())
|
||||||
|
.param("requestedTime", clock.nowUtc().toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue