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.async.AsyncFlowEnqueuer.PARAM_CLIENT_TRANSACTION_ID;
|
||||
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_RESOURCE_KEY;
|
||||
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.util.concurrent.TimeUnit.DAYS;
|
||||
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.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.ReducerInput;
|
||||
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.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
@ -61,6 +64,9 @@ import com.googlecode.objectify.Key;
|
|||
import com.googlecode.objectify.Work;
|
||||
import google.registry.batch.DeleteContactsAndHostsAction.DeletionResult.Type;
|
||||
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.inputs.EppResourceInputs;
|
||||
import google.registry.mapreduce.inputs.NullInput;
|
||||
|
@ -82,7 +88,9 @@ import google.registry.request.Action;
|
|||
import google.registry.request.Response;
|
||||
import google.registry.util.Clock;
|
||||
import google.registry.util.FormattingLogger;
|
||||
import google.registry.util.NonFinalForTesting;
|
||||
import google.registry.util.Retrier;
|
||||
import google.registry.util.SystemClock;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
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 DELETES_PER_SHARD = 5;
|
||||
|
||||
@Inject AsyncFlowMetrics asyncFlowMetrics;
|
||||
@Inject Clock clock;
|
||||
@Inject MapreduceRunner mrRunner;
|
||||
@Inject @Named(QUEUE_ASYNC_DELETE) Queue queue;
|
||||
|
@ -119,6 +128,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
LeaseOptions options =
|
||||
LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(LEASE_MINUTES, MINUTES);
|
||||
List<TaskHandle> tasks = queue.leaseTasks(options);
|
||||
asyncFlowMetrics.recordContactHostDeletionBatchSize(tasks.size());
|
||||
if (tasks.isEmpty()) {
|
||||
response.setPayload("No contact/host deletion tasks in pull queue.");
|
||||
return;
|
||||
|
@ -126,17 +136,16 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
Multiset<String> kindCounts = HashMultiset.create(2);
|
||||
ImmutableList.Builder<DeletionRequest> builder = 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) {
|
||||
try {
|
||||
Optional<DeletionRequest> deletionRequest =
|
||||
DeletionRequest.createFromTask(task, clock.nowUtc());
|
||||
if (deletionRequest.isPresent()) {
|
||||
builder.add(deletionRequest.get());
|
||||
resourceKeys.add(deletionRequest.get().key());
|
||||
kindCounts.add(deletionRequest.get().key().getKind());
|
||||
DeletionRequest deletionRequest = DeletionRequest.createFromTask(task, clock.nowUtc());
|
||||
if (deletionRequest.isDeletionAllowed()) {
|
||||
builder.add(deletionRequest);
|
||||
resourceKeys.add(deletionRequest.key());
|
||||
kindCounts.add(deletionRequest.key().getKind());
|
||||
} else {
|
||||
tasksToDelete.add(task);
|
||||
requestsToDelete.add(deletionRequest);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.severefmt(
|
||||
|
@ -146,7 +155,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
queue.modifyTaskLease(task, 1L, DAYS);
|
||||
}
|
||||
}
|
||||
deleteTasksWithRetry(tasksToDelete);
|
||||
deleteStaleTasksWithRetry(requestsToDelete);
|
||||
ImmutableList<DeletionRequest> deletionRequests = builder.build();
|
||||
if (deletionRequests.isEmpty()) {
|
||||
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) {
|
||||
if (tasks.isEmpty()) {
|
||||
/**
|
||||
* Deletes a list of tasks associated with deletion requests from the async delete queue using a
|
||||
* retrier.
|
||||
*/
|
||||
private void deleteStaleTasksWithRetry(final List<DeletionRequest> deletionRequests) {
|
||||
if (deletionRequests.isEmpty()) {
|
||||
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(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
queue.deleteTask(tasks);
|
||||
return null;
|
||||
}}, TransientFailureException.class);
|
||||
}
|
||||
},
|
||||
TransientFailureException.class);
|
||||
for (DeletionRequest deletionRequest : deletionRequests) {
|
||||
asyncFlowMetrics.recordAsyncFlowResult(
|
||||
deletionRequest.getMetricOperationType(),
|
||||
OperationResult.STALE,
|
||||
deletionRequest.requestedTime());
|
||||
}
|
||||
}
|
||||
|
||||
private void runMapreduce(ImmutableList<DeletionRequest> deletionRequests) {
|
||||
|
@ -253,6 +283,9 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
private static final long serialVersionUID = 6569363449285506326L;
|
||||
private static final DnsQueue dnsQueue = DnsQueue.create();
|
||||
|
||||
@NonFinalForTesting
|
||||
private static AsyncFlowMetrics asyncFlowMetrics = new AsyncFlowMetrics(new SystemClock());
|
||||
|
||||
@Override
|
||||
public void reduce(final DeletionRequest deletionRequest, ReducerInput<Boolean> values) {
|
||||
final boolean hasNoActiveReferences = !Iterators.contains(values, true);
|
||||
|
@ -266,6 +299,10 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
getQueue(QUEUE_ASYNC_DELETE).deleteTask(deletionRequest.task());
|
||||
return deletionResult;
|
||||
}});
|
||||
asyncFlowMetrics.recordAsyncFlowResult(
|
||||
deletionRequest.getMetricOperationType(),
|
||||
result.getMetricOperationResult(),
|
||||
deletionRequest.requestedTime());
|
||||
String resourceNamePlural = deletionRequest.key().getKind() + "s";
|
||||
getContext().incrementCounter(result.type().renderCounterText(resourceNamePlural));
|
||||
logger.infofmt(
|
||||
|
@ -423,7 +460,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
@AutoValue
|
||||
abstract static class DeletionRequest implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 5782119100274089088L;
|
||||
private static final long serialVersionUID = -4612618525760839240L;
|
||||
|
||||
abstract Key<? extends EppResource> key();
|
||||
abstract DateTime lastUpdateTime();
|
||||
|
@ -440,6 +477,8 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
abstract String serverTransactionId();
|
||||
|
||||
abstract boolean isSuperuser();
|
||||
abstract DateTime requestedTime();
|
||||
abstract boolean isDeletionAllowed();
|
||||
abstract TaskHandle task();
|
||||
|
||||
@AutoValue.Builder
|
||||
|
@ -450,11 +489,13 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
abstract Builder setClientTransactionId(String clientTransactionId);
|
||||
abstract Builder setServerTransactionId(String serverTransactionId);
|
||||
abstract Builder setIsSuperuser(boolean isSuperuser);
|
||||
abstract Builder setRequestedTime(DateTime requestedTime);
|
||||
abstract Builder setIsDeletionAllowed(boolean isDeletionAllowed);
|
||||
abstract Builder setTask(TaskHandle task);
|
||||
abstract DeletionRequest build();
|
||||
}
|
||||
|
||||
static Optional<DeletionRequest> createFromTask(TaskHandle task, DateTime now)
|
||||
static DeletionRequest createFromTask(TaskHandle task, DateTime now)
|
||||
throws Exception {
|
||||
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
||||
Key<EppResource> resourceKey =
|
||||
|
@ -466,11 +507,7 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
resource instanceof ContactResource || resource instanceof HostResource,
|
||||
"Cannot delete a %s via this action",
|
||||
resource.getClass().getSimpleName());
|
||||
if (!doesResourceStateAllowDeletion(resource, now)) {
|
||||
return Optional.absent();
|
||||
}
|
||||
return Optional.<DeletionRequest>of(
|
||||
new AutoValue_DeleteContactsAndHostsAction_DeletionRequest.Builder()
|
||||
return new AutoValue_DeleteContactsAndHostsAction_DeletionRequest.Builder()
|
||||
.setKey(resourceKey)
|
||||
.setLastUpdateTime(resource.getUpdateAutoTimestamp().getTimestamp())
|
||||
.setRequestingClientId(
|
||||
|
@ -478,17 +515,35 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
params.get(PARAM_REQUESTING_CLIENT_ID), "Requesting client id not specified"))
|
||||
.setClientTransactionId(
|
||||
checkNotNull(
|
||||
params.get(PARAM_CLIENT_TRANSACTION_ID),
|
||||
"Client transaction id not specified"))
|
||||
params.get(PARAM_CLIENT_TRANSACTION_ID), "Client transaction id not specified"))
|
||||
.setServerTransactionId(
|
||||
checkNotNull(
|
||||
params.get(PARAM_SERVER_TRANSACTION_ID),
|
||||
"Server transaction id not specified"))
|
||||
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());
|
||||
.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()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -497,14 +552,16 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
abstract static class DeletionResult {
|
||||
|
||||
enum Type {
|
||||
DELETED("%s deleted"),
|
||||
NOT_DELETED("%s not deleted"),
|
||||
ERRORED("%s errored out during deletion");
|
||||
DELETED("%s deleted", OperationResult.SUCCESS),
|
||||
NOT_DELETED("%s not deleted", OperationResult.FAILURE),
|
||||
ERRORED("%s errored out during deletion", OperationResult.ERROR);
|
||||
|
||||
private final String counterFormat;
|
||||
private final OperationResult operationResult;
|
||||
|
||||
private Type(String counterFormat) {
|
||||
private Type(String counterFormat, OperationResult operationResult) {
|
||||
this.counterFormat = counterFormat;
|
||||
this.operationResult = operationResult;
|
||||
}
|
||||
|
||||
String renderCounterText(String resourceName) {
|
||||
|
@ -518,6 +575,10 @@ public class DeleteContactsAndHostsAction implements Runnable {
|
|||
static DeletionResult create(Type type, String pollMessageText) {
|
||||
return new AutoValue_DeleteContactsAndHostsAction_DeletionResult(type, pollMessageText);
|
||||
}
|
||||
|
||||
OperationResult getMetricOperationResult() {
|
||||
return type().operationResult;
|
||||
}
|
||||
}
|
||||
|
||||
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.common.base.Preconditions.checkNotNull;
|
||||
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.AsyncFlowMetrics.OperationType.DNS_REFRESH;
|
||||
import static google.registry.mapreduce.inputs.EppResourceInputs.createEntityInput;
|
||||
import static google.registry.model.EppResourceUtils.isActive;
|
||||
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 java.util.concurrent.TimeUnit.DAYS;
|
||||
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.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.ReducerInput;
|
||||
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.ImmutableMap;
|
||||
import com.googlecode.objectify.Key;
|
||||
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.inputs.NullInput;
|
||||
import google.registry.model.domain.DomainResource;
|
||||
|
@ -49,7 +55,9 @@ import google.registry.request.Action;
|
|||
import google.registry.request.Response;
|
||||
import google.registry.util.Clock;
|
||||
import google.registry.util.FormattingLogger;
|
||||
import google.registry.util.NonFinalForTesting;
|
||||
import google.registry.util.Retrier;
|
||||
import google.registry.util.SystemClock;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -66,6 +74,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
||||
private static final long LEASE_MINUTES = 20;
|
||||
|
||||
@Inject AsyncFlowMetrics asyncFlowMetrics;
|
||||
@Inject Clock clock;
|
||||
@Inject MapreduceRunner mrRunner;
|
||||
@Inject @Named(QUEUE_ASYNC_HOST_RENAME) Queue pullQueue;
|
||||
|
@ -78,24 +87,24 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
LeaseOptions options =
|
||||
LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(LEASE_MINUTES, MINUTES);
|
||||
List<TaskHandle> tasks = pullQueue.leaseTasks(options);
|
||||
asyncFlowMetrics.recordDnsRefreshBatchSize(tasks.size());
|
||||
if (tasks.isEmpty()) {
|
||||
response.setPayload("No DNS refresh on host rename tasks to process in pull queue.");
|
||||
return;
|
||||
}
|
||||
ImmutableList.Builder<DnsRefreshRequest> requestsBuilder = 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) {
|
||||
try {
|
||||
Optional<DnsRefreshRequest> request =
|
||||
DnsRefreshRequest.createFromTask(task, clock.nowUtc());
|
||||
if (request.isPresent()) {
|
||||
requestsBuilder.add(request.get());
|
||||
hostKeys.add(request.get().hostKey());
|
||||
DnsRefreshRequest request = DnsRefreshRequest.createFromTask(task, clock.nowUtc());
|
||||
if (request.isRefreshNeeded()) {
|
||||
requestsBuilder.add(request);
|
||||
hostKeys.add(request.hostKey());
|
||||
} else {
|
||||
// Skip hosts that are deleted.
|
||||
tasksToDelete.add(task);
|
||||
requestsToDelete.add(request);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
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();
|
||||
if (refreshRequests.isEmpty()) {
|
||||
logger.info(
|
||||
|
@ -113,12 +123,11 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
response.setPayload("All requested DNS refreshes are on hosts that were since deleted.");
|
||||
} else {
|
||||
logger.infofmt("Processing asynchronous DNS refresh for renamed hosts: %s", hostKeys.build());
|
||||
runMapreduce(refreshRequests, tasks);
|
||||
runMapreduce(refreshRequests);
|
||||
}
|
||||
}
|
||||
|
||||
private void runMapreduce(
|
||||
ImmutableList<DnsRefreshRequest> refreshRequests, List<TaskHandle> tasks) {
|
||||
private void runMapreduce(ImmutableList<DnsRefreshRequest> refreshRequests) {
|
||||
try {
|
||||
response.sendJavaScriptRedirect(createJobPath(mrRunner
|
||||
.setJobName("Enqueue DNS refreshes for domains referencing renamed hosts")
|
||||
|
@ -126,7 +135,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
.setDefaultReduceShards(1)
|
||||
.runMapreduce(
|
||||
new RefreshDnsOnHostRenameMapper(refreshRequests, retrier),
|
||||
new RefreshDnsOnHostRenameReducer(tasks, retrier),
|
||||
new RefreshDnsOnHostRenameReducer(refreshRequests, retrier),
|
||||
// Add an extra NullInput so that the reducer always fires exactly once.
|
||||
ImmutableList.of(
|
||||
new NullInput<DomainResource>(), createEntityInput(DomainResource.class)))));
|
||||
|
@ -199,26 +208,48 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
|
||||
private static final long serialVersionUID = -2850944843275790412L;
|
||||
|
||||
private final Retrier retrier;
|
||||
private final List<TaskHandle> tasks;
|
||||
@NonFinalForTesting
|
||||
private static AsyncFlowMetrics asyncFlowMetrics = new AsyncFlowMetrics(new SystemClock());
|
||||
|
||||
RefreshDnsOnHostRenameReducer(List<TaskHandle> tasks, Retrier retrier) {
|
||||
this.tasks = tasks;
|
||||
private final Retrier retrier;
|
||||
private final List<DnsRefreshRequest> refreshRequests;
|
||||
|
||||
RefreshDnsOnHostRenameReducer(List<DnsRefreshRequest> refreshRequests, Retrier retrier) {
|
||||
this.refreshRequests = refreshRequests;
|
||||
this.retrier = retrier;
|
||||
}
|
||||
|
||||
@Override
|
||||
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. */
|
||||
private static void deleteTasksWithRetry(
|
||||
final List<TaskHandle> tasks, final Queue queue, Retrier retrier) {
|
||||
if (tasks.isEmpty()) {
|
||||
final List<DnsRefreshRequest> refreshRequests,
|
||||
final Queue queue,
|
||||
AsyncFlowMetrics asyncFlowMetrics,
|
||||
Retrier retrier,
|
||||
OperationResult result) {
|
||||
if (refreshRequests.isEmpty()) {
|
||||
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(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
|
@ -226,35 +257,60 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
|||
queue.deleteTask(tasks);
|
||||
return null;
|
||||
}}, 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. */
|
||||
@AutoValue
|
||||
abstract static class DnsRefreshRequest implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 2188894914017230887L;
|
||||
private static final long serialVersionUID = 1772812852271288622L;
|
||||
|
||||
abstract Key<HostResource> hostKey();
|
||||
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
|
||||
* the host specified is already deleted.
|
||||
* Returns a packaged-up {@link DnsRefreshRequest} parsed from a task queue task.
|
||||
*/
|
||||
static Optional<DnsRefreshRequest> createFromTask(TaskHandle task, DateTime now)
|
||||
throws Exception {
|
||||
static DnsRefreshRequest createFromTask(TaskHandle task, DateTime now) throws Exception {
|
||||
ImmutableMap<String, String> params = ImmutableMap.copyOf(task.extractParams());
|
||||
Key<HostResource> hostKey =
|
||||
Key.create(checkNotNull(params.get(PARAM_HOST_KEY), "Host to refresh not specified"));
|
||||
HostResource host =
|
||||
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);
|
||||
return Optional.absent();
|
||||
}
|
||||
return Optional.<DnsRefreshRequest>of(
|
||||
new AutoValue_RefreshDnsOnHostRenameAction_DnsRefreshRequest(
|
||||
hostKey, host.getUpdateAutoTimestamp().getTimestamp()));
|
||||
return new AutoValue_RefreshDnsOnHostRenameAction_DnsRefreshRequest.Builder()
|
||||
.setHostKey(hostKey)
|
||||
.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 javax.inject.Inject;
|
||||
import javax.inject.Named;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/** Helper class to enqueue tasks for handling asynchronous operations in flows. */
|
||||
|
@ -41,6 +42,7 @@ public final class AsyncFlowEnqueuer {
|
|||
public static final String PARAM_SERVER_TRANSACTION_ID = "serverTransactionId";
|
||||
public static final String PARAM_IS_SUPERUSER = "isSuperuser";
|
||||
public static final String PARAM_HOST_KEY = "hostKey";
|
||||
public static final String PARAM_REQUESTED_TIME = "requestedTime";
|
||||
|
||||
/** The task queue names used by async flows. */
|
||||
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();
|
||||
|
||||
@VisibleForTesting
|
||||
@Inject
|
||||
@Config("asyncDeleteFlowMapreduceDelay")
|
||||
public Duration asyncDeleteDelay;
|
||||
private final Duration asyncDeleteDelay;
|
||||
private final Queue asyncDeletePullQueue;
|
||||
private final Queue asyncDnsRefreshPullQueue;
|
||||
private final Retrier retrier;
|
||||
|
||||
@VisibleForTesting
|
||||
@Inject
|
||||
@Named("async-delete-pull")
|
||||
public Queue asyncDeletePullQueue;
|
||||
|
||||
@VisibleForTesting
|
||||
@Inject
|
||||
@Named(QUEUE_ASYNC_HOST_RENAME)
|
||||
public Queue asyncDnsRefreshPullQueue;
|
||||
|
||||
@VisibleForTesting
|
||||
@Inject
|
||||
public Retrier retrier;
|
||||
|
||||
@VisibleForTesting
|
||||
@Inject
|
||||
public AsyncFlowEnqueuer() {}
|
||||
public AsyncFlowEnqueuer(
|
||||
@Named(QUEUE_ASYNC_DELETE) Queue asyncDeletePullQueue,
|
||||
@Named(QUEUE_ASYNC_HOST_RENAME) Queue asyncDnsRefreshPullQueue,
|
||||
@Config("asyncDeleteFlowMapreduceDelay") Duration asyncDeleteDelay,
|
||||
Retrier retrier) {
|
||||
this.asyncDeletePullQueue = asyncDeletePullQueue;
|
||||
this.asyncDnsRefreshPullQueue = asyncDnsRefreshPullQueue;
|
||||
this.asyncDeleteDelay = asyncDeleteDelay;
|
||||
this.retrier = retrier;
|
||||
}
|
||||
|
||||
/** Enqueues a task to asynchronously delete a contact or host, by key. */
|
||||
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);
|
||||
logger.infofmt(
|
||||
"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_CLIENT_TRANSACTION_ID, trid.getClientTransactionId())
|
||||
.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);
|
||||
}
|
||||
|
||||
/** 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);
|
||||
logger.infofmt("Enqueuing async DNS refresh for renamed host %s.", hostKey);
|
||||
addTaskToQueueWithRetry(
|
||||
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) {
|
||||
verifyResourceOwnership(clientId, existingContact);
|
||||
}
|
||||
asyncFlowEnqueuer.enqueueAsyncDelete(existingContact, clientId, trid, isSuperuser);
|
||||
asyncFlowEnqueuer.enqueueAsyncDelete(
|
||||
existingContact, ofy().getTransactionTime(), clientId, trid, isSuperuser);
|
||||
ContactResource newContact =
|
||||
existingContact.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
|
||||
historyBuilder
|
||||
|
|
|
@ -108,7 +108,8 @@ public final class HostDeleteFlow implements TransactionalFlow {
|
|||
: existingHost;
|
||||
verifyResourceOwnership(clientId, owningResource);
|
||||
}
|
||||
asyncFlowEnqueuer.enqueueAsyncDelete(existingHost, clientId, trid, isSuperuser);
|
||||
asyncFlowEnqueuer.enqueueAsyncDelete(
|
||||
existingHost, ofy().getTransactionTime(), clientId, trid, isSuperuser);
|
||||
HostResource newHost =
|
||||
existingHost.asBuilder().addStatusValue(StatusValue.PENDING_DELETE).build();
|
||||
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
|
||||
// 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
|
||||
* EventMetric#reset()}.
|
||||
*/
|
||||
public final class EventMetric extends AbstractMetric<Distribution> {
|
||||
public class EventMetric extends AbstractMetric<Distribution> {
|
||||
|
||||
/**
|
||||
* Default {@link DistributionFitter} suitable for latency measurements.
|
||||
|
@ -86,7 +86,7 @@ public final class EventMetric extends AbstractMetric<Distribution> {
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
final ImmutableList<MetricPoint<Distribution>> getTimestampedValues(Instant endTimestamp) {
|
||||
ImmutableList<MetricPoint<Distribution>> getTimestampedValues(Instant endTimestamp) {
|
||||
ImmutableList.Builder<MetricPoint<Distribution>> timestampedValues =
|
||||
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.
|
||||
*/
|
||||
public final void record(double sample, String... labelValues) {
|
||||
public void record(double sample, String... labelValues) {
|
||||
MetricsUtils.checkLabelValuesLength(this, 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.
|
||||
*/
|
||||
public final void record(double sample, int count, String... labelValues) {
|
||||
public void record(double sample, int count, String... labelValues) {
|
||||
MetricsUtils.checkLabelValuesLength(this, labelValues);
|
||||
|
||||
recordMultiple(sample, count, Instant.now(), ImmutableList.copyOf(labelValues));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
final void recordMultiple(
|
||||
void recordMultiple(
|
||||
double sample, int count, Instant startTimestamp, ImmutableList<String> labelValues) {
|
||||
Lock lock = valueLocks.get(labelValues);
|
||||
lock.lock();
|
||||
|
|
|
@ -13,11 +13,12 @@
|
|||
// limitations under the License.
|
||||
|
||||
package google.registry.batch;
|
||||
|
||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static com.google.common.collect.Iterables.getOnlyElement;
|
||||
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_HOST_RENAME;
|
||||
import static google.registry.flows.async.AsyncFlowMetrics.OperationResult.STALE;
|
||||
import static google.registry.model.EppResourceUtils.loadByForeignKey;
|
||||
import static google.registry.model.eppcommon.StatusValue.PENDING_DELETE;
|
||||
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.standardHours;
|
||||
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.Method;
|
||||
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.Iterables;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.batch.DeleteContactsAndHostsAction.DeleteEppResourceReducer;
|
||||
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.contact.ContactAddress;
|
||||
import google.registry.model.contact.ContactPhoneNumber;
|
||||
|
@ -129,12 +136,16 @@ public class DeleteContactsAndHostsActionTest
|
|||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
enqueuer = new AsyncFlowEnqueuer();
|
||||
enqueuer.asyncDeleteDelay = Duration.ZERO;
|
||||
enqueuer.asyncDeletePullQueue = QueueFactory.getQueue(QUEUE_ASYNC_DELETE);
|
||||
enqueuer.retrier = new Retrier(new FakeSleeper(clock), 1);
|
||||
|
||||
enqueuer =
|
||||
new AsyncFlowEnqueuer(
|
||||
getQueue(QUEUE_ASYNC_DELETE),
|
||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||
Duration.ZERO,
|
||||
new Retrier(new FakeSleeper(clock), 1));
|
||||
AsyncFlowMetrics asyncFlowMetricsMock = mock(AsyncFlowMetrics.class);
|
||||
action = new DeleteContactsAndHostsAction();
|
||||
action.asyncFlowMetrics = asyncFlowMetricsMock;
|
||||
inject.setStaticField(DeleteEppResourceReducer.class, "asyncFlowMetrics", asyncFlowMetricsMock);
|
||||
action.clock = clock;
|
||||
action.mrRunner = makeDefaultRunner();
|
||||
action.response = new FakeResponse();
|
||||
|
@ -150,8 +161,13 @@ public class DeleteContactsAndHostsActionTest
|
|||
public void testSuccess_contact_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
|
||||
ContactResource contact = persistContactPendingDelete("blah8221");
|
||||
persistResource(newDomainResource("example.tld", contact));
|
||||
DateTime timeEnqueued = clock.nowUtc();
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contact,
|
||||
timeEnqueued,
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
ContactResource contactUpdated =
|
||||
loadByForeignKey(ContactResource.class, "blah8221", clock.nowUtc());
|
||||
|
@ -172,13 +188,22 @@ public class DeleteContactsAndHostsActionTest
|
|||
false,
|
||||
contact);
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
.recordAsyncFlowResult(OperationType.CONTACT_DELETE, OperationResult.FAILURE, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccess_contact_notReferenced_getsDeleted_andPiiWipedOut() throws Exception {
|
||||
ContactResource contact = persistContactWithPii("jim919");
|
||||
DateTime timeEnqueued = clock.nowUtc();
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contact,
|
||||
timeEnqueued,
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
assertThat(loadByForeignKey(ContactResource.class, "jim919", clock.nowUtc())).isNull();
|
||||
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
||||
|
@ -204,6 +229,10 @@ public class DeleteContactsAndHostsActionTest
|
|||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(contactAfterDeletion, CONTACT_DELETE);
|
||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted contact jim919.", true, contact);
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
.recordAsyncFlowResult(OperationType.CONTACT_DELETE, OperationResult.SUCCESS, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -211,7 +240,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
throws Exception {
|
||||
ContactResource contact = persistContactPendingDelete("blah8221");
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contact,
|
||||
clock.nowUtc(),
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
||||
assertThat(contactAfterDeletion.getTransferData()).isEqualTo(TransferData.EMPTY);
|
||||
|
@ -227,7 +260,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
transferRequestTime.plus(Registry.DEFAULT_TRANSFER_GRACE_PERIOD),
|
||||
clock.nowUtc());
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contact,
|
||||
clock.nowUtc(),
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
// Check that the contact is deleted as of now.
|
||||
assertThat(loadByForeignKey(ContactResource.class, "sh8013", clock.nowUtc())).isNull();
|
||||
|
@ -273,7 +310,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
.setDeletionTime(clock.nowUtc().minusDays(3))
|
||||
.build());
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contactUsed, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contactUsed,
|
||||
clock.nowUtc(),
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
assertThat(loadByForeignKey(ContactResource.class, "blah1234", clock.nowUtc())).isNull();
|
||||
ContactResource contactBeforeDeletion =
|
||||
|
@ -298,7 +339,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
public void testSuccess_contact_notRequestedByOwner_doesNotGetDeleted() throws Exception {
|
||||
ContactResource contact = persistContactPendingDelete("jane0991");
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contact, "OtherRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contact,
|
||||
clock.nowUtc(),
|
||||
"OtherRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
ContactResource contactAfter =
|
||||
loadByForeignKey(ContactResource.class, "jane0991", clock.nowUtc());
|
||||
|
@ -321,7 +366,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
public void testSuccess_contact_notRequestedByOwner_isSuperuser_getsDeleted() throws Exception {
|
||||
ContactResource contact = persistContactWithPii("nate007");
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contact, "OtherRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), true);
|
||||
contact,
|
||||
clock.nowUtc(),
|
||||
"OtherRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
true);
|
||||
runMapreduce();
|
||||
assertThat(loadByForeignKey(ContactResource.class, "nate007", clock.nowUtc())).isNull();
|
||||
ContactResource contactAfterDeletion = ofy().load().entity(contact).now();
|
||||
|
@ -353,22 +402,38 @@ public class DeleteContactsAndHostsActionTest
|
|||
public void testSuccess_targetResourcesDontExist_areDelayedForADay() throws Exception {
|
||||
ContactResource contactNotSaved = newContactResource("somecontact");
|
||||
HostResource hostNotSaved = newHostResource("a11.blah.foo");
|
||||
DateTime timeBeforeRun = clock.nowUtc();
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contactNotSaved, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contactNotSaved,
|
||||
timeBeforeRun,
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
hostNotSaved, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
hostNotSaved,
|
||||
timeBeforeRun,
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
String payloadFormat =
|
||||
"resourceKey=%s&requestingClientId=TheRegistrar&"
|
||||
+ "clientTransactionId=fakeClientTrid&serverTransactionId=fakeServerTrid&isSuperuser=false";
|
||||
assertTasksEnqueued(
|
||||
QUEUE_ASYNC_DELETE,
|
||||
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()
|
||||
.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
|
||||
|
@ -382,22 +447,39 @@ public class DeleteContactsAndHostsActionTest
|
|||
new TaskMatcher()
|
||||
.payload("gobbledygook=kljhadfgsd9f7gsdfh")
|
||||
.etaDelta(standardHours(23), standardHours(25)));
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccess_resourcesNotInPendingDelete_areSkipped() throws Exception {
|
||||
ContactResource contact = persistActiveContact("blah2222");
|
||||
HostResource host = persistActiveHost("rustles.your.jimmies");
|
||||
DateTime timeEnqueued = clock.nowUtc();
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contact, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contact,
|
||||
timeEnqueued,
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
host,
|
||||
timeEnqueued,
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
assertThat(loadByForeignKey(ContactResource.class, "blah2222", clock.nowUtc()))
|
||||
.isEqualTo(contact);
|
||||
assertThat(loadByForeignKey(HostResource.class, "rustles.your.jimmies", clock.nowUtc()))
|
||||
.isEqualTo(host);
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
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
|
||||
|
@ -405,9 +487,17 @@ public class DeleteContactsAndHostsActionTest
|
|||
ContactResource contactDeleted = persistDeletedContact("blah1236", clock.nowUtc().minusDays(2));
|
||||
HostResource hostDeleted = persistDeletedHost("a.lim.lop", clock.nowUtc().minusDays(3));
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
contactDeleted, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
contactDeleted,
|
||||
clock.nowUtc(),
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
hostDeleted, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
hostDeleted,
|
||||
clock.nowUtc(),
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
assertThat(ofy().load().entity(contactDeleted).now()).isEqualTo(contactDeleted);
|
||||
assertThat(ofy().load().entity(hostDeleted).now()).isEqualTo(hostDeleted);
|
||||
|
@ -418,8 +508,13 @@ public class DeleteContactsAndHostsActionTest
|
|||
public void testSuccess_host_referencedByActiveDomain_doesNotGetDeleted() throws Exception {
|
||||
HostResource host = persistHostPendingDelete("ns1.example.tld");
|
||||
persistUsedDomain("example.tld", persistActiveContact("abc456"), host);
|
||||
DateTime timeEnqueued = clock.nowUtc();
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
host,
|
||||
timeEnqueued,
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
HostResource hostAfter =
|
||||
loadByForeignKey(HostResource.class, "ns1.example.tld", clock.nowUtc());
|
||||
|
@ -438,13 +533,22 @@ public class DeleteContactsAndHostsActionTest
|
|||
false,
|
||||
host);
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
.recordAsyncFlowResult(OperationType.HOST_DELETE, OperationResult.FAILURE, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccess_host_notReferenced_getsDeleted() throws Exception {
|
||||
HostResource host = persistHostPendingDelete("ns2.example.tld");
|
||||
DateTime timeEnqueued = clock.nowUtc();
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
host,
|
||||
timeEnqueued,
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
assertThat(loadByForeignKey(HostResource.class, "ns2.example.tld", clock.nowUtc())).isNull();
|
||||
HostResource hostBeforeDeletion =
|
||||
|
@ -462,6 +566,10 @@ public class DeleteContactsAndHostsActionTest
|
|||
HistoryEntry historyEntry = getOnlyHistoryEntryOfType(hostBeforeDeletion, HOST_DELETE);
|
||||
assertPollMessageFor(historyEntry, "TheRegistrar", "Deleted host ns2.example.tld.", true, host);
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_DELETE);
|
||||
verify(action.asyncFlowMetrics).recordContactHostDeletionBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
.recordAsyncFlowResult(OperationType.HOST_DELETE, OperationResult.SUCCESS, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -474,7 +582,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
.setDeletionTime(clock.nowUtc().minusDays(5))
|
||||
.build());
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
host,
|
||||
clock.nowUtc(),
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
assertThat(loadByForeignKey(HostResource.class, "ns1.example.tld", clock.nowUtc())).isNull();
|
||||
HostResource hostBeforeDeletion =
|
||||
|
@ -509,7 +621,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
.setSuperordinateDomain(Key.create(domain))
|
||||
.build());
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
host, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
host,
|
||||
clock.nowUtc(),
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
// Check that the host is deleted as of now.
|
||||
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 {
|
||||
HostResource host = persistHostPendingDelete("ns2.example.tld");
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
host, "OtherRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
host,
|
||||
clock.nowUtc(),
|
||||
"OtherRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
runMapreduce();
|
||||
HostResource hostAfter =
|
||||
loadByForeignKey(HostResource.class, "ns2.example.tld", clock.nowUtc());
|
||||
|
@ -561,7 +681,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
public void testSuccess_host_notRequestedByOwner_isSuperuser_getsDeleted() throws Exception {
|
||||
HostResource host = persistHostPendingDelete("ns66.example.tld");
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
host, "OtherRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), true);
|
||||
host,
|
||||
clock.nowUtc(),
|
||||
"OtherRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
true);
|
||||
runMapreduce();
|
||||
assertThat(loadByForeignKey(HostResource.class, "ns66.example.tld", clock.nowUtc())).isNull();
|
||||
HostResource hostBeforeDeletion =
|
||||
|
@ -595,7 +719,11 @@ public class DeleteContactsAndHostsActionTest
|
|||
persistUsedDomain("usescontactandhost.tld", c4, h4);
|
||||
for (EppResource resource : ImmutableList.<EppResource>of(c1, c2, c3, c4, h1, h2, h3, h4)) {
|
||||
enqueuer.enqueueAsyncDelete(
|
||||
resource, "TheRegistrar", Trid.create("fakeClientTrid", "fakeServerTrid"), false);
|
||||
resource,
|
||||
clock.nowUtc(),
|
||||
"TheRegistrar",
|
||||
Trid.create("fakeClientTrid", "fakeServerTrid"),
|
||||
false);
|
||||
}
|
||||
runMapreduce();
|
||||
for (EppResource resource : ImmutableList.<EppResource>of(c1, c2, c3, h1, h2, h3)) {
|
||||
|
|
|
@ -15,7 +15,9 @@
|
|||
package google.registry.batch;
|
||||
|
||||
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.AsyncFlowMetrics.OperationType.DNS_REFRESH;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.testing.DatastoreHelper.createTld;
|
||||
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.standardHours;
|
||||
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.googlecode.objectify.Key;
|
||||
import google.registry.batch.RefreshDnsOnHostRenameAction.RefreshDnsOnHostRenameReducer;
|
||||
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.testing.ExceptionRule;
|
||||
import google.registry.testing.FakeClock;
|
||||
|
@ -48,6 +57,7 @@ import google.registry.util.Retrier;
|
|||
import google.registry.util.Sleeper;
|
||||
import google.registry.util.SystemSleeper;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
|
@ -71,12 +81,17 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
@Before
|
||||
public void setup() throws Exception {
|
||||
createTld("tld");
|
||||
|
||||
enqueuer = new AsyncFlowEnqueuer();
|
||||
enqueuer.asyncDnsRefreshPullQueue = getQueue(QUEUE_ASYNC_HOST_RENAME);
|
||||
enqueuer.retrier = new Retrier(new FakeSleeper(clock), 1);
|
||||
|
||||
enqueuer =
|
||||
new AsyncFlowEnqueuer(
|
||||
getQueue(QUEUE_ASYNC_DELETE),
|
||||
getQueue(QUEUE_ASYNC_HOST_RENAME),
|
||||
Duration.ZERO,
|
||||
new Retrier(new FakeSleeper(clock), 1));
|
||||
AsyncFlowMetrics asyncFlowMetricsMock = mock(AsyncFlowMetrics.class);
|
||||
action = new RefreshDnsOnHostRenameAction();
|
||||
action.asyncFlowMetrics = asyncFlowMetricsMock;
|
||||
inject.setStaticField(
|
||||
RefreshDnsOnHostRenameReducer.class, "asyncFlowMetrics", asyncFlowMetricsMock);
|
||||
action.clock = clock;
|
||||
action.mrRunner = makeDefaultRunner();
|
||||
action.pullQueue = getQueue(QUEUE_ASYNC_HOST_RENAME);
|
||||
|
@ -108,11 +123,15 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
persistResource(newDomainResource("example.tld", host));
|
||||
persistResource(newDomainResource("otherexample.tld", host));
|
||||
persistResource(newDomainResource("untouched.tld", persistActiveHost("ns2.example.tld")));
|
||||
|
||||
enqueuer.enqueueAsyncDnsRefresh(host);
|
||||
DateTime timeEnqueued = clock.nowUtc();
|
||||
enqueuer.enqueueAsyncDnsRefresh(host, timeEnqueued);
|
||||
runMapreduce();
|
||||
assertDnsTasksEnqueued("example.tld", "otherexample.tld");
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
verify(action.asyncFlowMetrics).recordDnsRefreshBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.SUCCESS, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -123,23 +142,35 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
persistResource(newDomainResource("example1.tld", host1));
|
||||
persistResource(newDomainResource("example2.tld", host2));
|
||||
persistResource(newDomainResource("example3.tld", host3));
|
||||
|
||||
enqueuer.enqueueAsyncDnsRefresh(host1);
|
||||
enqueuer.enqueueAsyncDnsRefresh(host2);
|
||||
enqueuer.enqueueAsyncDnsRefresh(host3);
|
||||
DateTime timeEnqueued = clock.nowUtc();
|
||||
DateTime laterTimeEnqueued = timeEnqueued.plus(standardSeconds(10));
|
||||
enqueuer.enqueueAsyncDnsRefresh(host1, timeEnqueued);
|
||||
enqueuer.enqueueAsyncDnsRefresh(host2, timeEnqueued);
|
||||
enqueuer.enqueueAsyncDnsRefresh(host3, laterTimeEnqueued);
|
||||
runMapreduce();
|
||||
assertDnsTasksEnqueued("example1.tld", "example2.tld", "example3.tld");
|
||||
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
|
||||
public void testSuccess_deletedHost_doesntTriggerDnsRefresh() throws Exception {
|
||||
HostResource host = persistDeletedHost("ns11.fakesss.tld", clock.nowUtc().minusDays(4));
|
||||
persistResource(newDomainResource("example1.tld", host));
|
||||
enqueuer.enqueueAsyncDnsRefresh(host);
|
||||
DateTime timeEnqueued = clock.nowUtc();
|
||||
enqueuer.enqueueAsyncDnsRefresh(host, timeEnqueued);
|
||||
runMapreduce();
|
||||
assertNoDnsTasksEnqueued();
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
verify(action.asyncFlowMetrics).recordDnsRefreshBatchSize(1L);
|
||||
verify(action.asyncFlowMetrics)
|
||||
.recordAsyncFlowResult(DNS_REFRESH, OperationResult.STALE, timeEnqueued);
|
||||
verifyNoMoreInteractions(action.asyncFlowMetrics);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -150,7 +181,7 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
.asBuilder()
|
||||
.setDeletionTime(START_OF_TIME)
|
||||
.build());
|
||||
enqueuer.enqueueAsyncDnsRefresh(renamedHost);
|
||||
enqueuer.enqueueAsyncDnsRefresh(renamedHost, clock.nowUtc());
|
||||
runMapreduce();
|
||||
assertNoDnsTasksEnqueued();
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
|
@ -159,7 +190,7 @@ public class RefreshDnsOnHostRenameActionTest
|
|||
@Test
|
||||
public void testRun_hostDoesntExist_delaysTask() throws Exception {
|
||||
HostResource host = newHostResource("ns1.example.tld");
|
||||
enqueuer.enqueueAsyncDnsRefresh(host);
|
||||
enqueuer.enqueueAsyncDnsRefresh(host, clock.nowUtc());
|
||||
runMapreduce();
|
||||
assertNoDnsTasksEnqueued();
|
||||
assertTasksEnqueued(
|
||||
|
|
|
@ -28,6 +28,7 @@ java_library(
|
|||
"//java/google/registry/dns",
|
||||
"//java/google/registry/flows",
|
||||
"//java/google/registry/model",
|
||||
"//java/google/registry/monitoring/metrics",
|
||||
"//java/google/registry/monitoring/whitebox",
|
||||
"//java/google/registry/pricing",
|
||||
"//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 */
|
||||
protected static <T extends EppResource> void assertAsyncDeletionTaskEnqueued(
|
||||
protected <T extends EppResource> void assertAsyncDeletionTaskEnqueued(
|
||||
T resource, String requestingClientId, Trid trid, boolean isSuperuser) throws Exception {
|
||||
assertTasksEnqueued(
|
||||
"async-delete-pull",
|
||||
|
@ -166,7 +166,8 @@ public abstract class ResourceFlowTestCase<F extends Flow, R extends EppResource
|
|||
.param("requestingClientId", requestingClientId)
|
||||
.param("clientTransactionId", trid.getClientTransactionId())
|
||||
.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.
|
||||
assertTasksEnqueued(
|
||||
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
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue