mirror of
https://github.com/google/nomulus.git
synced 2025-05-13 16:07:15 +02:00
Add a batched [] for DNS refreshing on host renames
This will replace the existing DnsRefreshForHostRenameAction. This is stage one of a three stage migration process. It adds the new queue and [] but doesn't call them yet. Stage two will cut over to using the new functionality, and stage three will remove the old functionality. ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=134793963
This commit is contained in:
parent
e5c0854ae6
commit
e192c11adf
14 changed files with 522 additions and 5 deletions
|
@ -133,6 +133,17 @@
|
|||
<target>backend</target>
|
||||
</cron>
|
||||
|
||||
<cron>
|
||||
<url><![CDATA[/_dr/task/refreshDnsOnHostRename]]></url>
|
||||
<description>
|
||||
This job runs a mapreduce that asynchronously handles DNS refreshes for
|
||||
host renames by mapping over all domains and creating DNS refresh tasks
|
||||
for any domains that reference a renamed host.
|
||||
</description>
|
||||
<schedule>every 5 minutes synchronized</schedule>
|
||||
<target>backend</target>
|
||||
</cron>
|
||||
|
||||
<cron>
|
||||
<url><![CDATA[/_dr/cron/fanout?queue=export-snapshot&endpoint=/_dr/task/exportSnapshot&runInEmpty]]></url>
|
||||
<description>
|
||||
|
|
|
@ -31,6 +31,11 @@
|
|||
<mode>pull</mode>
|
||||
</queue>
|
||||
|
||||
<queue>
|
||||
<name>async-host-rename-pull</name>
|
||||
<mode>pull</mode>
|
||||
</queue>
|
||||
|
||||
<queue>
|
||||
<name>delete-commits</name>
|
||||
<rate>10/s</rate>
|
||||
|
|
|
@ -113,6 +113,17 @@
|
|||
<target>backend</target>
|
||||
</cron>
|
||||
|
||||
<cron>
|
||||
<url><![CDATA[/_dr/task/refreshDnsOnHostRename]]></url>
|
||||
<description>
|
||||
This job runs a mapreduce that asynchronously handles DNS refreshes for
|
||||
host renames by mapping over all domains and creating DNS refresh tasks
|
||||
for any domains that reference a renamed host.
|
||||
</description>
|
||||
<schedule>every 5 minutes synchronized</schedule>
|
||||
<target>backend</target>
|
||||
</cron>
|
||||
|
||||
<cron>
|
||||
<url><![CDATA[/_dr/cron/fanout?queue=export-snapshot&endpoint=/_dr/task/exportSnapshot&runInEmpty]]></url>
|
||||
<description>
|
||||
|
|
|
@ -144,6 +144,17 @@
|
|||
<target>backend</target>
|
||||
</cron>
|
||||
|
||||
<cron>
|
||||
<url><![CDATA[/_dr/task/refreshDnsOnHostRename]]></url>
|
||||
<description>
|
||||
This job runs a mapreduce that asynchronously handles DNS refreshes for
|
||||
host renames by mapping over all domains and creating DNS refresh tasks
|
||||
for any domains that reference a renamed host.
|
||||
</description>
|
||||
<schedule>every 5 minutes synchronized</schedule>
|
||||
<target>backend</target>
|
||||
</cron>
|
||||
|
||||
<cron>
|
||||
<url><![CDATA[/_dr/cron/fanout?queue=export-snapshot&endpoint=/_dr/task/exportSnapshot&runInEmpty]]></url>
|
||||
<description>
|
||||
|
|
|
@ -51,6 +51,17 @@
|
|||
<target>backend</target>
|
||||
</cron>
|
||||
|
||||
<cron>
|
||||
<url><![CDATA[/_dr/task/refreshDnsOnHostRename]]></url>
|
||||
<description>
|
||||
This job runs a mapreduce that asynchronously handles DNS refreshes for
|
||||
host renames by mapping over all domains and creating DNS refresh tasks
|
||||
for any domains that reference a renamed host.
|
||||
</description>
|
||||
<schedule>every 5 minutes synchronized</schedule>
|
||||
<target>backend</target>
|
||||
</cron>
|
||||
|
||||
<cron>
|
||||
<url><![CDATA[/_dr/cron/fanout?queue=export-snapshot&endpoint=/_dr/task/exportSnapshot&runInEmpty]]></url>
|
||||
<description>
|
||||
|
|
|
@ -19,6 +19,7 @@ import static google.registry.flows.async.DeleteContactsAndHostsAction.PARAM_IS_
|
|||
import static google.registry.flows.async.DeleteContactsAndHostsAction.PARAM_REQUESTING_CLIENT_ID;
|
||||
import static google.registry.flows.async.DeleteContactsAndHostsAction.PARAM_RESOURCE_KEY;
|
||||
import static google.registry.flows.async.DnsRefreshForHostRenameAction.PARAM_HOST_KEY;
|
||||
import static google.registry.flows.async.RefreshDnsOnHostRenameAction.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.request.Actions.getPathForAction;
|
||||
|
||||
import com.google.appengine.api.taskqueue.Queue;
|
||||
|
@ -45,6 +46,7 @@ public final class AsyncFlowEnqueuer {
|
|||
|
||||
@Inject @Config("asyncDeleteFlowMapreduceDelay") Duration asyncDeleteDelay;
|
||||
@Inject @Named("async-delete-pull") Queue asyncDeletePullQueue;
|
||||
@Inject @Named(QUEUE_ASYNC_HOST_RENAME) Queue asyncDnsRefreshPullQueue;
|
||||
@Inject Retrier retrier;
|
||||
@Inject AsyncFlowEnqueuer() {}
|
||||
|
||||
|
@ -55,7 +57,7 @@ public final class AsyncFlowEnqueuer {
|
|||
logger.infofmt(
|
||||
"Enqueueing async deletion of %s on behalf of registrar %s.",
|
||||
resourceKey, requestingClientId);
|
||||
final TaskOptions task =
|
||||
TaskOptions task =
|
||||
TaskOptions.Builder
|
||||
.withMethod(Method.PULL)
|
||||
.countdownMillis(asyncDeleteDelay.getMillis())
|
||||
|
@ -65,8 +67,19 @@ public final class AsyncFlowEnqueuer {
|
|||
addTaskToQueueWithRetry(asyncDeletePullQueue, task);
|
||||
}
|
||||
|
||||
/** Enqueues a task to asynchronously refresh DNS for a host. */
|
||||
/** Enqueues a task to asynchronously refresh DNS for a renamed host. */
|
||||
public void enqueueAsyncDnsRefresh(HostResource host) {
|
||||
Key<HostResource> hostKey = Key.create(host);
|
||||
logger.infofmt("Enqueueing async DNS refresh for renamed host %s.", hostKey);
|
||||
addTaskToQueueWithRetry(
|
||||
asyncDnsRefreshPullQueue,
|
||||
TaskOptions.Builder.withMethod(Method.PULL).param(PARAM_HOST_KEY, hostKey.getString()));
|
||||
}
|
||||
|
||||
/** Enqueues a task to asynchronously refresh DNS for a renamed host. */
|
||||
//TODO(b/26140521): Delete this once non-batched DNS host refresh mapreduce is deleted.
|
||||
@Deprecated
|
||||
public void enqueueLegacyAsyncDnsRefresh(HostResource host) {
|
||||
logger.infofmt("Enqueueing async DNS refresh for host %s", Key.create(host));
|
||||
// Aggressively back off if the task fails, to minimize flooding the logs.
|
||||
RetryOptions retryOptions =
|
||||
|
|
|
@ -13,8 +13,10 @@
|
|||
// limitations under the License.
|
||||
|
||||
package google.registry.flows.async;
|
||||
|
||||
import static google.registry.flows.async.DeleteContactsAndHostsAction.QUEUE_ASYNC_DELETE;
|
||||
import static google.registry.flows.async.DnsRefreshForHostRenameAction.PARAM_HOST_KEY;
|
||||
import static google.registry.flows.async.RefreshDnsOnHostRenameAction.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.request.RequestParameters.extractRequiredParameter;
|
||||
|
||||
import com.google.appengine.api.taskqueue.Queue;
|
||||
|
@ -35,6 +37,12 @@ public final class AsyncFlowsModule {
|
|||
return QueueFactory.getQueue(QUEUE_ASYNC_DELETE);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Named(QUEUE_ASYNC_HOST_RENAME)
|
||||
static Queue provideAsyncHostRenamePullQueue() {
|
||||
return QueueFactory.getQueue(QUEUE_ASYNC_HOST_RENAME);
|
||||
}
|
||||
|
||||
//TODO(b/26140521): Delete this method once non-batched DNS host refresh mapreduce is deleted.
|
||||
@Provides
|
||||
@Parameter(PARAM_HOST_KEY)
|
||||
|
|
|
@ -0,0 +1,260 @@
|
|||
// Copyright 2016 The Domain Registry Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package google.registry.flows.async;
|
||||
|
||||
import static com.google.appengine.api.taskqueue.QueueConstants.maxLeaseCount;
|
||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
import static google.registry.mapreduce.inputs.EppResourceInputs.createEntityInput;
|
||||
import static google.registry.model.EppResourceUtils.isActive;
|
||||
import static google.registry.model.EppResourceUtils.isDeleted;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
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 com.google.appengine.api.taskqueue.LeaseOptions;
|
||||
import com.google.appengine.api.taskqueue.Queue;
|
||||
import com.google.appengine.api.taskqueue.TaskHandle;
|
||||
import com.google.appengine.api.taskqueue.TransientFailureException;
|
||||
import com.google.appengine.tools.mapreduce.Mapper;
|
||||
import com.google.appengine.tools.mapreduce.Reducer;
|
||||
import com.google.appengine.tools.mapreduce.ReducerInput;
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.common.base.Optional;
|
||||
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.mapreduce.MapreduceRunner;
|
||||
import google.registry.mapreduce.inputs.NullInput;
|
||||
import google.registry.model.domain.DomainResource;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.request.Action;
|
||||
import google.registry.request.Response;
|
||||
import google.registry.util.Clock;
|
||||
import google.registry.util.FormattingLogger;
|
||||
import google.registry.util.Retrier;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Callable;
|
||||
import javax.annotation.Nullable;
|
||||
import javax.inject.Inject;
|
||||
import javax.inject.Named;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Performs batched DNS refreshes for applicable domains following a host rename. */
|
||||
@Action(path = "/_dr/task/refreshDnsOnHostRename")
|
||||
public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||
|
||||
/** The HTTP parameter name used to specify the websafe key of the host to rename. */
|
||||
public static final String PARAM_HOST_KEY = "hostKey";
|
||||
public static final String QUEUE_ASYNC_HOST_RENAME = "async-host-rename-pull";
|
||||
|
||||
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
||||
private static final long LEASE_MINUTES = 20;
|
||||
|
||||
@Inject Clock clock;
|
||||
@Inject MapreduceRunner mrRunner;
|
||||
@Inject @Named(QUEUE_ASYNC_HOST_RENAME) Queue pullQueue;
|
||||
@Inject Response response;
|
||||
@Inject Retrier retrier;
|
||||
@Inject RefreshDnsOnHostRenameAction() {}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
LeaseOptions options =
|
||||
LeaseOptions.Builder.withCountLimit(maxLeaseCount()).leasePeriod(LEASE_MINUTES, MINUTES);
|
||||
List<TaskHandle> tasks = pullQueue.leaseTasks(options);
|
||||
if (tasks.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
ImmutableList.Builder<DnsRefreshRequest> requestsBuilder = new ImmutableList.Builder<>();
|
||||
ImmutableList.Builder<Key<HostResource>> hostKeys = new ImmutableList.Builder<>();
|
||||
final List<TaskHandle> tasksToDelete = 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());
|
||||
} else {
|
||||
// Skip hosts that are deleted.
|
||||
tasksToDelete.add(task);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.severefmt(
|
||||
e, "Could not parse DNS refresh for host request, delaying task for a day: %s", task);
|
||||
// Grab the lease for a whole day, so it won't continue throwing errors every five minutes.
|
||||
pullQueue.modifyTaskLease(task, 1L, DAYS);
|
||||
}
|
||||
}
|
||||
|
||||
deleteTasksWithRetry(tasksToDelete, pullQueue, retrier);
|
||||
ImmutableList<DnsRefreshRequest> refreshRequests = requestsBuilder.build();
|
||||
if (refreshRequests.isEmpty()) {
|
||||
logger.info(
|
||||
"No asynchronous DNS refreshes to process because all renamed hosts are deleted.");
|
||||
return;
|
||||
}
|
||||
logger.infofmt("Processing asynchronous DNS refresh for renamed hosts: %s", hostKeys.build());
|
||||
runMapreduce(refreshRequests, tasks);
|
||||
}
|
||||
|
||||
private void runMapreduce(
|
||||
ImmutableList<DnsRefreshRequest> refreshRequests, List<TaskHandle> tasks) {
|
||||
try {
|
||||
response.sendJavaScriptRedirect(createJobPath(mrRunner
|
||||
.setJobName("Enqueue DNS refreshes for domains referencing renamed hosts")
|
||||
.setModuleName("backend")
|
||||
.setDefaultReduceShards(1)
|
||||
.runMapreduce(
|
||||
new RefreshDnsOnHostRenameMapper(refreshRequests, retrier),
|
||||
new RefreshDnsOnHostRenameReducer(tasks, retrier),
|
||||
// Add an extra NullInput so that the reducer always fires exactly once.
|
||||
ImmutableList.of(
|
||||
new NullInput<DomainResource>(), createEntityInput(DomainResource.class)))));
|
||||
} catch (Throwable t) {
|
||||
logger.severefmt(t, "Error while kicking off mapreduce to refresh DNS for renamed hosts.");
|
||||
}
|
||||
}
|
||||
|
||||
/** Map over domains and refresh the DNS of those that reference the renamed hosts. */
|
||||
public static class RefreshDnsOnHostRenameMapper
|
||||
extends Mapper<DomainResource, Boolean, Boolean> {
|
||||
|
||||
private static final long serialVersionUID = -5261698524424335531L;
|
||||
private static final DnsQueue dnsQueue = DnsQueue.create();
|
||||
|
||||
private final ImmutableList<DnsRefreshRequest> refreshRequests;
|
||||
private final Retrier retrier;
|
||||
|
||||
RefreshDnsOnHostRenameMapper(
|
||||
ImmutableList<DnsRefreshRequest> refreshRequests, Retrier retrier) {
|
||||
this.refreshRequests = refreshRequests;
|
||||
this.retrier = retrier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void map(@Nullable final DomainResource domain) {
|
||||
if (domain == null) {
|
||||
// Emit a single value so that the reducer always runs. The key and value don't matter.
|
||||
emit(true, true);
|
||||
return;
|
||||
}
|
||||
Key<HostResource> referencingHostKey = null;
|
||||
for (DnsRefreshRequest request : refreshRequests) {
|
||||
if (isActive(domain, request.lastUpdateTime())
|
||||
&& domain.getNameservers().contains(request.hostKey())) {
|
||||
referencingHostKey = request.hostKey();
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (referencingHostKey != null) {
|
||||
retrier.callWithRetry(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
dnsQueue.addDomainRefreshTask(domain.getFullyQualifiedDomainName());
|
||||
return null;
|
||||
}}, TransientFailureException.class);
|
||||
logger.infofmt(
|
||||
"Enqueued DNS refresh for domain %s referenced by host %s.",
|
||||
domain.getFullyQualifiedDomainName(), referencingHostKey);
|
||||
getContext().incrementCounter("domains refreshed");
|
||||
} else {
|
||||
getContext().incrementCounter("domains not refreshed");
|
||||
}
|
||||
|
||||
// Don't catch errors -- we allow the mapreduce to terminate on any errors that can't be
|
||||
// resolved by retrying the transaction. The reducer only fires if the mapper completes
|
||||
// without errors, meaning that it is acceptable to delete all tasks.
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A reducer that always fires exactly once.
|
||||
*
|
||||
* <p>This is really a reducer in name only; what it's really doing is waiting for all of the
|
||||
* mapper tasks to finish, and then delete the pull queue tasks. Note that this only happens if
|
||||
* the mapper completes execution without errors.
|
||||
*/
|
||||
public static class RefreshDnsOnHostRenameReducer extends Reducer<Boolean, Boolean, Void> {
|
||||
|
||||
private static final long serialVersionUID = -2850944843275790412L;
|
||||
|
||||
private final Retrier retrier;
|
||||
private final List<TaskHandle> tasks;
|
||||
|
||||
RefreshDnsOnHostRenameReducer(List<TaskHandle> tasks, Retrier retrier) {
|
||||
this.tasks = tasks;
|
||||
this.retrier = retrier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reduce(Boolean key, ReducerInput<Boolean> values) {
|
||||
deleteTasksWithRetry(tasks, getQueue(QUEUE_ASYNC_HOST_RENAME), retrier);
|
||||
}
|
||||
}
|
||||
|
||||
/** 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()) {
|
||||
return;
|
||||
}
|
||||
retrier.callWithRetry(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
queue.deleteTask(tasks);
|
||||
return null;
|
||||
}}, TransientFailureException.class);
|
||||
}
|
||||
|
||||
/** 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;
|
||||
|
||||
abstract Key<HostResource> hostKey();
|
||||
abstract DateTime lastUpdateTime();
|
||||
|
||||
/**
|
||||
* Returns a packaged-up DnsRefreshRequest parsed from a task queue task, or absent if the host
|
||||
* specified is already deleted.
|
||||
*/
|
||||
static Optional<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()))) {
|
||||
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()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -230,7 +230,7 @@ public final class HostUpdateFlow extends LoggedInFlow implements TransactionalF
|
|||
}
|
||||
// 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(existingResource);
|
||||
asyncFlowEnqueuer.enqueueLegacyAsyncDnsRefresh(existingResource);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -44,6 +44,7 @@ import google.registry.export.sheet.SyncRegistrarsSheetAction;
|
|||
import google.registry.flows.async.AsyncFlowsModule;
|
||||
import google.registry.flows.async.DeleteContactsAndHostsAction;
|
||||
import google.registry.flows.async.DnsRefreshForHostRenameAction;
|
||||
import google.registry.flows.async.RefreshDnsOnHostRenameAction;
|
||||
import google.registry.mapreduce.MapreduceModule;
|
||||
import google.registry.monitoring.whitebox.MetricsExportAction;
|
||||
import google.registry.monitoring.whitebox.VerifyEntityIntegrityAction;
|
||||
|
@ -109,6 +110,7 @@ interface BackendRequestComponent {
|
|||
RdeUploadAction rdeUploadAction();
|
||||
RdeReporter rdeReporter();
|
||||
RefreshDnsAction refreshDnsAction();
|
||||
RefreshDnsOnHostRenameAction refreshDnsOnHostRenameAction();
|
||||
RestoreCommitLogsAction restoreCommitLogsAction();
|
||||
SyncGroupMembersAction syncGroupMembersAction();
|
||||
SyncRegistrarsSheetAction syncRegistrarsSheetAction();
|
||||
|
|
|
@ -0,0 +1,172 @@
|
|||
// Copyright 2016 The Domain Registry Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package google.registry.flows.async;
|
||||
|
||||
import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
|
||||
import static google.registry.flows.async.RefreshDnsOnHostRenameAction.QUEUE_ASYNC_HOST_RENAME;
|
||||
import static google.registry.model.ofy.ObjectifyService.ofy;
|
||||
import static google.registry.testing.DatastoreHelper.createTld;
|
||||
import static google.registry.testing.DatastoreHelper.newDomainApplication;
|
||||
import static google.registry.testing.DatastoreHelper.newDomainResource;
|
||||
import static google.registry.testing.DatastoreHelper.newHostResource;
|
||||
import static google.registry.testing.DatastoreHelper.persistActiveHost;
|
||||
import static google.registry.testing.DatastoreHelper.persistDeletedHost;
|
||||
import static google.registry.testing.DatastoreHelper.persistResource;
|
||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
||||
import static google.registry.testing.TaskQueueHelper.assertNoDnsTasksEnqueued;
|
||||
import static google.registry.testing.TaskQueueHelper.assertNoTasksEnqueued;
|
||||
import static google.registry.testing.TaskQueueHelper.assertTasksEnqueued;
|
||||
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 com.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.googlecode.objectify.Key;
|
||||
import google.registry.mapreduce.MapreduceRunner;
|
||||
import google.registry.model.host.HostResource;
|
||||
import google.registry.testing.ExceptionRule;
|
||||
import google.registry.testing.FakeClock;
|
||||
import google.registry.testing.FakeResponse;
|
||||
import google.registry.testing.FakeSleeper;
|
||||
import google.registry.testing.InjectRule;
|
||||
import google.registry.testing.TaskQueueHelper.TaskMatcher;
|
||||
import google.registry.testing.mapreduce.MapreduceTestCase;
|
||||
import google.registry.util.Retrier;
|
||||
import google.registry.util.Sleeper;
|
||||
import google.registry.util.SystemSleeper;
|
||||
import org.joda.time.DateTime;
|
||||
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 RefreshDnsOnHostRenameAction}. */
|
||||
@RunWith(JUnit4.class)
|
||||
public class RefreshDnsOnHostRenameActionTest
|
||||
extends MapreduceTestCase<RefreshDnsOnHostRenameAction> {
|
||||
|
||||
@Rule
|
||||
public final ExceptionRule thrown = new ExceptionRule();
|
||||
|
||||
@Rule
|
||||
public InjectRule inject = new InjectRule();
|
||||
|
||||
private AsyncFlowEnqueuer enqueuer;
|
||||
private final FakeClock clock = new FakeClock(DateTime.parse("2015-01-15T11:22:33Z"));
|
||||
|
||||
@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);
|
||||
|
||||
action = new RefreshDnsOnHostRenameAction();
|
||||
action.clock = clock;
|
||||
action.mrRunner = new MapreduceRunner(Optional.<Integer>of(5), Optional.<Integer>absent());
|
||||
action.pullQueue = getQueue(QUEUE_ASYNC_HOST_RENAME);
|
||||
action.response = new FakeResponse();
|
||||
action.retrier = new Retrier(new FakeSleeper(clock), 1);
|
||||
}
|
||||
|
||||
private void runMapreduce() throws Exception {
|
||||
clock.advanceOneMilli();
|
||||
// Use hard sleeps to ensure that the tasks are enqueued properly and will be leased.
|
||||
Sleeper sleeper = new SystemSleeper();
|
||||
sleeper.sleep(millis(50));
|
||||
action.run();
|
||||
sleeper.sleep(millis(50));
|
||||
executeTasksUntilEmpty("mapreduce", clock);
|
||||
sleeper.sleep(millis(50));
|
||||
clock.advanceBy(standardSeconds(5));
|
||||
ofy().clearSessionCache();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccess_dnsUpdateEnqueued() throws Exception {
|
||||
HostResource host = persistActiveHost("ns1.example.tld");
|
||||
persistResource(
|
||||
newDomainApplication("notadomain.tld")
|
||||
.asBuilder()
|
||||
.setNameservers(ImmutableSet.of(Key.create(host)))
|
||||
.build());
|
||||
persistResource(newDomainResource("example.tld", host));
|
||||
persistResource(newDomainResource("otherexample.tld", host));
|
||||
persistResource(newDomainResource("untouched.tld", persistActiveHost("ns2.example.tld")));
|
||||
|
||||
enqueuer.enqueueAsyncDnsRefresh(host);
|
||||
runMapreduce();
|
||||
assertDnsTasksEnqueued("example.tld", "otherexample.tld");
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccess_multipleHostsProcessedInBatch() throws Exception {
|
||||
HostResource host1 = persistActiveHost("ns1.example.tld");
|
||||
HostResource host2 = persistActiveHost("ns2.example.tld");
|
||||
HostResource host3 = persistActiveHost("ns3.example.tld");
|
||||
persistResource(newDomainResource("example1.tld", host1));
|
||||
persistResource(newDomainResource("example2.tld", host2));
|
||||
persistResource(newDomainResource("example3.tld", host3));
|
||||
|
||||
enqueuer.enqueueAsyncDnsRefresh(host1);
|
||||
enqueuer.enqueueAsyncDnsRefresh(host2);
|
||||
enqueuer.enqueueAsyncDnsRefresh(host3);
|
||||
runMapreduce();
|
||||
assertDnsTasksEnqueued("example1.tld", "example2.tld", "example3.tld");
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
}
|
||||
|
||||
@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);
|
||||
runMapreduce();
|
||||
assertNoDnsTasksEnqueued();
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccess_noDnsTasksForDeletedDomain() throws Exception {
|
||||
HostResource renamedHost = persistActiveHost("ns1.example.tld");
|
||||
persistResource(
|
||||
newDomainResource("example.tld", renamedHost)
|
||||
.asBuilder()
|
||||
.setDeletionTime(START_OF_TIME)
|
||||
.build());
|
||||
enqueuer.enqueueAsyncDnsRefresh(renamedHost);
|
||||
runMapreduce();
|
||||
assertNoDnsTasksEnqueued();
|
||||
assertNoTasksEnqueued(QUEUE_ASYNC_HOST_RENAME);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRun_hostDoesntExist_delaysTask() throws Exception {
|
||||
HostResource host = newHostResource("ns1.example.tld");
|
||||
enqueuer.enqueueAsyncDnsRefresh(host);
|
||||
runMapreduce();
|
||||
assertNoDnsTasksEnqueued();
|
||||
assertTasksEnqueued(
|
||||
QUEUE_ASYNC_HOST_RENAME,
|
||||
new TaskMatcher()
|
||||
.payload("hostKey=" + Key.create(host).getString())
|
||||
.etaDelta(standardHours(23), standardHours(25)));
|
||||
}
|
||||
}
|
|
@ -126,6 +126,13 @@ public class DatastoreHelper {
|
|||
domainName, generateNewDomainRoid(getTldFromDomainName(domainName)), contact);
|
||||
}
|
||||
|
||||
public static DomainResource newDomainResource(String domainName, HostResource host) {
|
||||
return newDomainResource(domainName)
|
||||
.asBuilder()
|
||||
.setNameservers(ImmutableSet.of(Key.create(host)))
|
||||
.build();
|
||||
}
|
||||
|
||||
public static DomainResource newDomainResource(
|
||||
String domainName, String repoId, ContactResource contact) {
|
||||
Key<ContactResource> contactKey = Key.create(contact);
|
||||
|
|
|
@ -19,6 +19,7 @@ import static org.joda.time.DateTimeZone.UTC;
|
|||
import static org.joda.time.Duration.millis;
|
||||
|
||||
import google.registry.util.Clock;
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import javax.annotation.concurrent.ThreadSafe;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -27,7 +28,9 @@ import org.joda.time.ReadableInstant;
|
|||
|
||||
/** A mock clock for testing purposes that supports telling, setting, and advancing the time. */
|
||||
@ThreadSafe
|
||||
public final class FakeClock implements Clock {
|
||||
public final class FakeClock implements Clock, Serializable {
|
||||
|
||||
private static final long serialVersionUID = 675054721685304599L;
|
||||
|
||||
// Clock isn't a thread synchronization primitive, but tests involving
|
||||
// threads should see a consistent flow.
|
||||
|
|
|
@ -18,12 +18,15 @@ import static com.google.common.base.Preconditions.checkArgument;
|
|||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
|
||||
import google.registry.util.Sleeper;
|
||||
import java.io.Serializable;
|
||||
import javax.annotation.concurrent.ThreadSafe;
|
||||
import org.joda.time.ReadableDuration;
|
||||
|
||||
/** Sleeper implementation for unit tests that advances {@link FakeClock} rather than sleep. */
|
||||
@ThreadSafe
|
||||
public final class FakeSleeper implements Sleeper {
|
||||
public final class FakeSleeper implements Sleeper, Serializable {
|
||||
|
||||
private static final long serialVersionUID = -8975804222581077291L;
|
||||
|
||||
private final FakeClock clock;
|
||||
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue