mirror of
https://github.com/google/nomulus.git
synced 2025-04-29 19:47:51 +02:00
Create a DnsRefreshRequest entity backed by the corresponding table (#1941)
Also adds a DnsUtils class to deal with adding, polling, and removing DNS refresh requests (only adding is implemented for now). The class also takes care of choosing which mechanism to use (pull queue vs. SQL) based on the current time and the database migration schedule map.
This commit is contained in:
parent
8ce47b9e0e
commit
3d809e762b
38 changed files with 760 additions and 231 deletions
|
@ -32,7 +32,7 @@ import com.google.common.collect.Sets;
|
||||||
import com.google.common.flogger.FluentLogger;
|
import com.google.common.flogger.FluentLogger;
|
||||||
import google.registry.config.RegistryConfig.Config;
|
import google.registry.config.RegistryConfig.Config;
|
||||||
import google.registry.config.RegistryEnvironment;
|
import google.registry.config.RegistryEnvironment;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.model.CreateAutoTimestamp;
|
import google.registry.model.CreateAutoTimestamp;
|
||||||
import google.registry.model.EppResourceUtils;
|
import google.registry.model.EppResourceUtils;
|
||||||
import google.registry.model.domain.Domain;
|
import google.registry.model.domain.Domain;
|
||||||
|
@ -98,7 +98,7 @@ public class DeleteProberDataAction implements Runnable {
|
||||||
/** Number of domains to retrieve and delete per SQL transaction. */
|
/** Number of domains to retrieve and delete per SQL transaction. */
|
||||||
private static final int BATCH_SIZE = 1000;
|
private static final int BATCH_SIZE = 1000;
|
||||||
|
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
@Parameter(PARAM_DRY_RUN)
|
@Parameter(PARAM_DRY_RUN)
|
||||||
|
@ -264,6 +264,6 @@ public class DeleteProberDataAction implements Runnable {
|
||||||
// messages, or auto-renews because those will all be hard-deleted the next time the job runs
|
// messages, or auto-renews because those will all be hard-deleted the next time the job runs
|
||||||
// anyway.
|
// anyway.
|
||||||
tm().putAll(ImmutableList.of(deletedDomain, historyEntry));
|
tm().putAll(ImmutableList.of(deletedDomain, historyEntry));
|
||||||
dnsQueue.addDomainRefreshTask(deletedDomain.getDomainName());
|
dnsUtils.requestDomainDnsRefresh(deletedDomain.getDomainName());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,5 +34,8 @@ public class DnsConstants {
|
||||||
public static final String DNS_TARGET_CREATE_TIME_PARAM = "Create-Time";
|
public static final String DNS_TARGET_CREATE_TIME_PARAM = "Create-Time";
|
||||||
|
|
||||||
/** The possible values of the {@code DNS_TARGET_TYPE_PARAM} parameter. */
|
/** The possible values of the {@code DNS_TARGET_TYPE_PARAM} parameter. */
|
||||||
public enum TargetType { DOMAIN, HOST, ZONE }
|
public enum TargetType {
|
||||||
|
DOMAIN,
|
||||||
|
HOST
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -77,11 +77,15 @@ public class DnsQueue {
|
||||||
private static final RateLimiter rateLimiter = RateLimiter.create(9);
|
private static final RateLimiter rateLimiter = RateLimiter.create(9);
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public DnsQueue(@Named(DNS_PULL_QUEUE_NAME) Queue queue, Clock clock) {
|
DnsQueue(@Named(DNS_PULL_QUEUE_NAME) Queue queue, Clock clock) {
|
||||||
this.queue = queue;
|
this.queue = queue;
|
||||||
this.clock = clock;
|
this.clock = clock;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Clock getClock() {
|
||||||
|
return clock;
|
||||||
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public static DnsQueue createForTesting(Clock clock) {
|
public static DnsQueue createForTesting(Clock clock) {
|
||||||
return new DnsQueue(getQueue(DNS_PULL_QUEUE_NAME), clock);
|
return new DnsQueue(getQueue(DNS_PULL_QUEUE_NAME), clock);
|
||||||
|
@ -108,7 +112,7 @@ public class DnsQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Adds a task to the queue to refresh the DNS information for the specified subordinate host. */
|
/** Adds a task to the queue to refresh the DNS information for the specified subordinate host. */
|
||||||
public TaskHandle addHostRefreshTask(String hostName) {
|
TaskHandle addHostRefreshTask(String hostName) {
|
||||||
Optional<InternetDomainName> tld = Registries.findTldForName(InternetDomainName.from(hostName));
|
Optional<InternetDomainName> tld = Registries.findTldForName(InternetDomainName.from(hostName));
|
||||||
checkArgument(
|
checkArgument(
|
||||||
tld.isPresent(), String.format("%s is not a subordinate host to a known tld", hostName));
|
tld.isPresent(), String.format("%s is not a subordinate host to a known tld", hostName));
|
||||||
|
@ -116,12 +120,12 @@ public class DnsQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Enqueues a task to refresh DNS for the specified domain now. */
|
/** Enqueues a task to refresh DNS for the specified domain now. */
|
||||||
public TaskHandle addDomainRefreshTask(String domainName) {
|
TaskHandle addDomainRefreshTask(String domainName) {
|
||||||
return addDomainRefreshTask(domainName, Duration.ZERO);
|
return addDomainRefreshTask(domainName, Duration.ZERO);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Enqueues a task to refresh DNS for the specified domain at some point in the future. */
|
/** Enqueues a task to refresh DNS for the specified domain at some point in the future. */
|
||||||
public TaskHandle addDomainRefreshTask(String domainName, Duration countdown) {
|
TaskHandle addDomainRefreshTask(String domainName, Duration countdown) {
|
||||||
return addToQueue(
|
return addToQueue(
|
||||||
TargetType.DOMAIN,
|
TargetType.DOMAIN,
|
||||||
domainName,
|
domainName,
|
||||||
|
@ -129,11 +133,6 @@ public class DnsQueue {
|
||||||
countdown);
|
countdown);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Adds a task to the queue to refresh the DNS information for the specified zone. */
|
|
||||||
public TaskHandle addZoneRefreshTask(String zoneName) {
|
|
||||||
return addToQueue(TargetType.ZONE, zoneName, zoneName, Duration.ZERO);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the maximum number of tasks that can be leased with {@link #leaseTasks}.
|
* Returns the maximum number of tasks that can be leased with {@link #leaseTasks}.
|
||||||
*
|
*
|
||||||
|
|
74
core/src/main/java/google/registry/dns/DnsUtils.java
Normal file
74
core/src/main/java/google/registry/dns/DnsUtils.java
Normal file
|
@ -0,0 +1,74 @@
|
||||||
|
// Copyright 2023 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.dns;
|
||||||
|
|
||||||
|
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||||
|
|
||||||
|
import com.google.common.net.InternetDomainName;
|
||||||
|
import google.registry.dns.DnsConstants.TargetType;
|
||||||
|
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
||||||
|
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
|
||||||
|
import google.registry.model.common.DnsRefreshRequest;
|
||||||
|
import google.registry.model.tld.Registries;
|
||||||
|
import javax.inject.Inject;
|
||||||
|
import org.joda.time.Duration;
|
||||||
|
|
||||||
|
/** Utility class to handle DNS refresh requests. */
|
||||||
|
// TODO: Make this a static util function once we are done with the DNS pull queue migration.
|
||||||
|
public class DnsUtils {
|
||||||
|
|
||||||
|
private final DnsQueue dnsQueue;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
DnsUtils(DnsQueue dnsQueue) {
|
||||||
|
this.dnsQueue = dnsQueue;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void requestDnsRefresh(String name, TargetType type, Duration delay) {
|
||||||
|
// Throws an IllegalArgumentException if the name is not under a managed TLD -- we only update
|
||||||
|
// DNS for names that are under our management.
|
||||||
|
String tld = Registries.findTldForNameOrThrow(InternetDomainName.from(name)).toString();
|
||||||
|
if (usePullQueue()) {
|
||||||
|
if (TargetType.HOST.equals(type)) {
|
||||||
|
dnsQueue.addHostRefreshTask(name);
|
||||||
|
} else {
|
||||||
|
dnsQueue.addDomainRefreshTask(name, delay);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
tm().transact(
|
||||||
|
() ->
|
||||||
|
tm().insert(
|
||||||
|
new DnsRefreshRequest(
|
||||||
|
type, name, tld, tm().getTransactionTime().plus(delay))));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void requestDomainDnsRefresh(String domainName, Duration delay) {
|
||||||
|
requestDnsRefresh(domainName, TargetType.DOMAIN, delay);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void requestDomainDnsRefresh(String domainName) {
|
||||||
|
requestDomainDnsRefresh(domainName, Duration.ZERO);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void requestHostDnsRefresh(String hostName) {
|
||||||
|
requestDnsRefresh(hostName, TargetType.HOST, Duration.ZERO);
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean usePullQueue() {
|
||||||
|
return !DatabaseMigrationStateSchedule.getValueAtTime(dnsQueue.getClock().nowUtc())
|
||||||
|
.equals(MigrationState.DNS_SQL);
|
||||||
|
}
|
||||||
|
}
|
|
@ -85,7 +85,7 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable<Void> {
|
||||||
|
|
||||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||||
|
|
||||||
private final DnsQueue dnsQueue;
|
private final DnsUtils dnsUtils;
|
||||||
private final DnsWriterProxy dnsWriterProxy;
|
private final DnsWriterProxy dnsWriterProxy;
|
||||||
private final DnsMetrics dnsMetrics;
|
private final DnsMetrics dnsMetrics;
|
||||||
private final Duration timeout;
|
private final Duration timeout;
|
||||||
|
@ -139,7 +139,7 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable<Void> {
|
||||||
@Config("gSuiteOutgoingEmailAddress") InternetAddress gSuiteOutgoingEmailAddress,
|
@Config("gSuiteOutgoingEmailAddress") InternetAddress gSuiteOutgoingEmailAddress,
|
||||||
@Header(APP_ENGINE_RETRY_HEADER) Optional<Integer> appEngineRetryCount,
|
@Header(APP_ENGINE_RETRY_HEADER) Optional<Integer> appEngineRetryCount,
|
||||||
@Header(CLOUD_TASKS_RETRY_HEADER) Optional<Integer> cloudTasksRetryCount,
|
@Header(CLOUD_TASKS_RETRY_HEADER) Optional<Integer> cloudTasksRetryCount,
|
||||||
DnsQueue dnsQueue,
|
DnsUtils dnsUtils,
|
||||||
DnsWriterProxy dnsWriterProxy,
|
DnsWriterProxy dnsWriterProxy,
|
||||||
DnsMetrics dnsMetrics,
|
DnsMetrics dnsMetrics,
|
||||||
LockHandler lockHandler,
|
LockHandler lockHandler,
|
||||||
|
@ -147,7 +147,7 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable<Void> {
|
||||||
CloudTasksUtils cloudTasksUtils,
|
CloudTasksUtils cloudTasksUtils,
|
||||||
SendEmailService sendEmailService,
|
SendEmailService sendEmailService,
|
||||||
Response response) {
|
Response response) {
|
||||||
this.dnsQueue = dnsQueue;
|
this.dnsUtils = dnsUtils;
|
||||||
this.dnsWriterProxy = dnsWriterProxy;
|
this.dnsWriterProxy = dnsWriterProxy;
|
||||||
this.dnsMetrics = dnsMetrics;
|
this.dnsMetrics = dnsMetrics;
|
||||||
this.timeout = timeout;
|
this.timeout = timeout;
|
||||||
|
@ -356,10 +356,10 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable<Void> {
|
||||||
private void requeueBatch() {
|
private void requeueBatch() {
|
||||||
logger.atInfo().log("Requeueing batch for retry.");
|
logger.atInfo().log("Requeueing batch for retry.");
|
||||||
for (String domain : nullToEmpty(domains)) {
|
for (String domain : nullToEmpty(domains)) {
|
||||||
dnsQueue.addDomainRefreshTask(domain);
|
dnsUtils.requestDomainDnsRefresh(domain);
|
||||||
}
|
}
|
||||||
for (String host : nullToEmpty(hosts)) {
|
for (String host : nullToEmpty(hosts)) {
|
||||||
dnsQueue.addHostRefreshTask(host);
|
dnsUtils.requestHostDnsRefresh(host);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -39,7 +39,7 @@ import javax.inject.Inject;
|
||||||
public final class RefreshDnsAction implements Runnable {
|
public final class RefreshDnsAction implements Runnable {
|
||||||
|
|
||||||
private final Clock clock;
|
private final Clock clock;
|
||||||
private final DnsQueue dnsQueue;
|
private final DnsUtils dnsUtils;
|
||||||
private final String domainOrHostName;
|
private final String domainOrHostName;
|
||||||
private final TargetType type;
|
private final TargetType type;
|
||||||
|
|
||||||
|
@ -48,11 +48,11 @@ public final class RefreshDnsAction implements Runnable {
|
||||||
@Parameter("domainOrHostName") String domainOrHostName,
|
@Parameter("domainOrHostName") String domainOrHostName,
|
||||||
@Parameter("type") TargetType type,
|
@Parameter("type") TargetType type,
|
||||||
Clock clock,
|
Clock clock,
|
||||||
DnsQueue dnsQueue) {
|
DnsUtils dnsUtils) {
|
||||||
this.domainOrHostName = domainOrHostName;
|
this.domainOrHostName = domainOrHostName;
|
||||||
this.type = type;
|
this.type = type;
|
||||||
this.clock = clock;
|
this.clock = clock;
|
||||||
this.dnsQueue = dnsQueue;
|
this.dnsUtils = dnsUtils;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -63,11 +63,11 @@ public final class RefreshDnsAction implements Runnable {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case DOMAIN:
|
case DOMAIN:
|
||||||
loadAndVerifyExistence(Domain.class, domainOrHostName);
|
loadAndVerifyExistence(Domain.class, domainOrHostName);
|
||||||
dnsQueue.addDomainRefreshTask(domainOrHostName);
|
dnsUtils.requestDomainDnsRefresh(domainOrHostName);
|
||||||
break;
|
break;
|
||||||
case HOST:
|
case HOST:
|
||||||
verifyHostIsSubordinate(loadAndVerifyExistence(Host.class, domainOrHostName));
|
verifyHostIsSubordinate(loadAndVerifyExistence(Host.class, domainOrHostName));
|
||||||
dnsQueue.addHostRefreshTask(domainOrHostName);
|
dnsUtils.requestHostDnsRefresh(domainOrHostName);
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new BadRequestException("Unsupported type: " + type);
|
throw new BadRequestException("Unsupported type: " + type);
|
||||||
|
|
|
@ -45,14 +45,14 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
|
|
||||||
private final VKey<Host> hostKey;
|
private final VKey<Host> hostKey;
|
||||||
private final Response response;
|
private final Response response;
|
||||||
private final DnsQueue dnsQueue;
|
private final DnsUtils dnsUtils;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
RefreshDnsOnHostRenameAction(
|
RefreshDnsOnHostRenameAction(
|
||||||
@Parameter(PARAM_HOST_KEY) String hostKey, Response response, DnsQueue dnsQueue) {
|
@Parameter(PARAM_HOST_KEY) String hostKey, Response response, DnsUtils dnsUtils) {
|
||||||
this.hostKey = VKey.createEppVKeyFromString(hostKey);
|
this.hostKey = VKey.createEppVKeyFromString(hostKey);
|
||||||
this.response = response;
|
this.response = response;
|
||||||
this.dnsQueue = dnsQueue;
|
this.dnsUtils = dnsUtils;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -76,7 +76,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable {
|
||||||
.stream()
|
.stream()
|
||||||
.map(domainKey -> tm().loadByKey(domainKey))
|
.map(domainKey -> tm().loadByKey(domainKey))
|
||||||
.filter(Domain::shouldPublishToDns)
|
.filter(Domain::shouldPublishToDns)
|
||||||
.forEach(domain -> dnsQueue.addDomainRefreshTask(domain.getDomainName()));
|
.forEach(domain -> dnsUtils.requestDomainDnsRefresh(domain.getDomainName()));
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!hostValid) {
|
if (!hostValid) {
|
||||||
|
|
|
@ -62,7 +62,7 @@ import com.google.auto.value.AutoValue;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.net.InternetDomainName;
|
import com.google.common.net.InternetDomainName;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.flows.EppException;
|
import google.registry.flows.EppException;
|
||||||
import google.registry.flows.EppException.AssociationProhibitsOperationException;
|
import google.registry.flows.EppException.AssociationProhibitsOperationException;
|
||||||
import google.registry.flows.EppException.CommandUseErrorException;
|
import google.registry.flows.EppException.CommandUseErrorException;
|
||||||
|
@ -235,7 +235,8 @@ public final class DomainCreateFlow implements TransactionalFlow {
|
||||||
@Inject DomainCreateFlowCustomLogic flowCustomLogic;
|
@Inject DomainCreateFlowCustomLogic flowCustomLogic;
|
||||||
@Inject DomainFlowTmchUtils tmchUtils;
|
@Inject DomainFlowTmchUtils tmchUtils;
|
||||||
@Inject DomainPricingLogic pricingLogic;
|
@Inject DomainPricingLogic pricingLogic;
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
|
|
||||||
@Inject DomainCreateFlow() {}
|
@Inject DomainCreateFlow() {}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -405,8 +406,10 @@ public final class DomainCreateFlow implements TransactionalFlow {
|
||||||
.addGracePeriod(
|
.addGracePeriod(
|
||||||
GracePeriod.forBillingEvent(GracePeriodStatus.ADD, repoId, createBillingEvent))
|
GracePeriod.forBillingEvent(GracePeriodStatus.ADD, repoId, createBillingEvent))
|
||||||
.setLordnPhase(
|
.setLordnPhase(
|
||||||
!DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime())
|
!(DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime())
|
||||||
.equals(MigrationState.NORDN_SQL)
|
.equals(MigrationState.NORDN_SQL)
|
||||||
|
|| DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime())
|
||||||
|
.equals(MigrationState.DNS_SQL))
|
||||||
? LordnPhase.NONE
|
? LordnPhase.NONE
|
||||||
: hasSignedMarks
|
: hasSignedMarks
|
||||||
? LordnPhase.SUNRISE
|
? LordnPhase.SUNRISE
|
||||||
|
@ -709,10 +712,12 @@ public final class DomainCreateFlow implements TransactionalFlow {
|
||||||
|
|
||||||
private void enqueueTasks(Domain newDomain, boolean hasSignedMarks, boolean hasClaimsNotice) {
|
private void enqueueTasks(Domain newDomain, boolean hasSignedMarks, boolean hasClaimsNotice) {
|
||||||
if (newDomain.shouldPublishToDns()) {
|
if (newDomain.shouldPublishToDns()) {
|
||||||
dnsQueue.addDomainRefreshTask(newDomain.getDomainName());
|
dnsUtils.requestDomainDnsRefresh(newDomain.getDomainName());
|
||||||
}
|
}
|
||||||
if (!DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime())
|
if (!(DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime())
|
||||||
.equals(MigrationState.NORDN_SQL)
|
.equals(MigrationState.NORDN_SQL)
|
||||||
|
|| DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime())
|
||||||
|
.equals(MigrationState.DNS_SQL))
|
||||||
&& (hasClaimsNotice || hasSignedMarks)) {
|
&& (hasClaimsNotice || hasSignedMarks)) {
|
||||||
LordnTaskUtils.enqueueDomainTask(newDomain);
|
LordnTaskUtils.enqueueDomainTask(newDomain);
|
||||||
}
|
}
|
||||||
|
|
|
@ -44,7 +44,7 @@ import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.ImmutableSortedSet;
|
import com.google.common.collect.ImmutableSortedSet;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import google.registry.batch.AsyncTaskEnqueuer;
|
import google.registry.batch.AsyncTaskEnqueuer;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.flows.EppException;
|
import google.registry.flows.EppException;
|
||||||
import google.registry.flows.EppException.AssociationProhibitsOperationException;
|
import google.registry.flows.EppException.AssociationProhibitsOperationException;
|
||||||
import google.registry.flows.ExtensionManager;
|
import google.registry.flows.ExtensionManager;
|
||||||
|
@ -129,7 +129,7 @@ public final class DomainDeleteFlow implements TransactionalFlow {
|
||||||
@Inject @TargetId String targetId;
|
@Inject @TargetId String targetId;
|
||||||
@Inject @Superuser boolean isSuperuser;
|
@Inject @Superuser boolean isSuperuser;
|
||||||
@Inject DomainHistory.Builder historyBuilder;
|
@Inject DomainHistory.Builder historyBuilder;
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
@Inject Trid trid;
|
@Inject Trid trid;
|
||||||
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||||
@Inject EppResponse.Builder responseBuilder;
|
@Inject EppResponse.Builder responseBuilder;
|
||||||
|
@ -260,7 +260,7 @@ public final class DomainDeleteFlow implements TransactionalFlow {
|
||||||
// If there's a pending transfer, the gaining client's autorenew billing
|
// If there's a pending transfer, the gaining client's autorenew billing
|
||||||
// event and poll message will already have been deleted in
|
// event and poll message will already have been deleted in
|
||||||
// ResourceDeleteFlow since it's listed in serverApproveEntities.
|
// ResourceDeleteFlow since it's listed in serverApproveEntities.
|
||||||
dnsQueue.addDomainRefreshTask(existingDomain.getDomainName());
|
dnsUtils.requestDomainDnsRefresh(existingDomain.getDomainName());
|
||||||
|
|
||||||
entitiesToSave.add(newDomain, domainHistory);
|
entitiesToSave.add(newDomain, domainHistory);
|
||||||
EntityChanges entityChanges =
|
EntityChanges entityChanges =
|
||||||
|
|
|
@ -34,7 +34,7 @@ import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.net.InternetDomainName;
|
import com.google.common.net.InternetDomainName;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.flows.EppException;
|
import google.registry.flows.EppException;
|
||||||
import google.registry.flows.EppException.CommandUseErrorException;
|
import google.registry.flows.EppException.CommandUseErrorException;
|
||||||
import google.registry.flows.EppException.StatusProhibitsOperationException;
|
import google.registry.flows.EppException.StatusProhibitsOperationException;
|
||||||
|
@ -122,7 +122,7 @@ public final class DomainRestoreRequestFlow implements TransactionalFlow {
|
||||||
@Inject @TargetId String targetId;
|
@Inject @TargetId String targetId;
|
||||||
@Inject @Superuser boolean isSuperuser;
|
@Inject @Superuser boolean isSuperuser;
|
||||||
@Inject DomainHistory.Builder historyBuilder;
|
@Inject DomainHistory.Builder historyBuilder;
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
@Inject EppResponse.Builder responseBuilder;
|
@Inject EppResponse.Builder responseBuilder;
|
||||||
@Inject DomainPricingLogic pricingLogic;
|
@Inject DomainPricingLogic pricingLogic;
|
||||||
@Inject DomainRestoreRequestFlow() {}
|
@Inject DomainRestoreRequestFlow() {}
|
||||||
|
@ -186,7 +186,7 @@ public final class DomainRestoreRequestFlow implements TransactionalFlow {
|
||||||
entitiesToSave.add(newDomain, domainHistory, autorenewEvent, autorenewPollMessage);
|
entitiesToSave.add(newDomain, domainHistory, autorenewEvent, autorenewPollMessage);
|
||||||
tm().putAll(entitiesToSave.build());
|
tm().putAll(entitiesToSave.build());
|
||||||
tm().delete(existingDomain.getDeletePollMessage());
|
tm().delete(existingDomain.getDeletePollMessage());
|
||||||
dnsQueue.addDomainRefreshTask(existingDomain.getDomainName());
|
dnsUtils.requestDomainDnsRefresh(existingDomain.getDomainName());
|
||||||
return responseBuilder
|
return responseBuilder
|
||||||
.setExtensions(createResponseExtensions(feesAndCredits, feeUpdate, isExpired))
|
.setExtensions(createResponseExtensions(feesAndCredits, feeUpdate, isExpired))
|
||||||
.build();
|
.build();
|
||||||
|
|
|
@ -49,7 +49,7 @@ import com.google.common.collect.ImmutableSortedSet;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.google.common.net.InternetDomainName;
|
import com.google.common.net.InternetDomainName;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.flows.EppException;
|
import google.registry.flows.EppException;
|
||||||
import google.registry.flows.ExtensionManager;
|
import google.registry.flows.ExtensionManager;
|
||||||
import google.registry.flows.FlowModule.RegistrarId;
|
import google.registry.flows.FlowModule.RegistrarId;
|
||||||
|
@ -156,7 +156,7 @@ public final class DomainUpdateFlow implements TransactionalFlow {
|
||||||
@Inject @Superuser boolean isSuperuser;
|
@Inject @Superuser boolean isSuperuser;
|
||||||
@Inject Trid trid;
|
@Inject Trid trid;
|
||||||
@Inject DomainHistory.Builder historyBuilder;
|
@Inject DomainHistory.Builder historyBuilder;
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
@Inject EppResponse.Builder responseBuilder;
|
@Inject EppResponse.Builder responseBuilder;
|
||||||
@Inject DomainUpdateFlowCustomLogic flowCustomLogic;
|
@Inject DomainUpdateFlowCustomLogic flowCustomLogic;
|
||||||
@Inject DomainPricingLogic pricingLogic;
|
@Inject DomainPricingLogic pricingLogic;
|
||||||
|
@ -183,7 +183,7 @@ public final class DomainUpdateFlow implements TransactionalFlow {
|
||||||
historyBuilder.setType(DOMAIN_UPDATE).setDomain(newDomain).build();
|
historyBuilder.setType(DOMAIN_UPDATE).setDomain(newDomain).build();
|
||||||
validateNewState(newDomain);
|
validateNewState(newDomain);
|
||||||
if (requiresDnsUpdate(existingDomain, newDomain)) {
|
if (requiresDnsUpdate(existingDomain, newDomain)) {
|
||||||
dnsQueue.addDomainRefreshTask(targetId);
|
dnsUtils.requestDomainDnsRefresh(targetId);
|
||||||
}
|
}
|
||||||
ImmutableSet.Builder<ImmutableObject> entitiesToSave = new ImmutableSet.Builder<>();
|
ImmutableSet.Builder<ImmutableObject> entitiesToSave = new ImmutableSet.Builder<>();
|
||||||
entitiesToSave.add(newDomain, domainHistory);
|
entitiesToSave.add(newDomain, domainHistory);
|
||||||
|
|
|
@ -28,7 +28,7 @@ import static google.registry.util.CollectionUtils.isNullOrEmpty;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import google.registry.config.RegistryConfig.Config;
|
import google.registry.config.RegistryConfig.Config;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.flows.EppException;
|
import google.registry.flows.EppException;
|
||||||
import google.registry.flows.EppException.ParameterValueRangeErrorException;
|
import google.registry.flows.EppException.ParameterValueRangeErrorException;
|
||||||
import google.registry.flows.EppException.RequiredParameterMissingException;
|
import google.registry.flows.EppException.RequiredParameterMissingException;
|
||||||
|
@ -85,7 +85,7 @@ public final class HostCreateFlow implements TransactionalFlow {
|
||||||
@Inject @RegistrarId String registrarId;
|
@Inject @RegistrarId String registrarId;
|
||||||
@Inject @TargetId String targetId;
|
@Inject @TargetId String targetId;
|
||||||
@Inject HostHistory.Builder historyBuilder;
|
@Inject HostHistory.Builder historyBuilder;
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
@Inject EppResponse.Builder responseBuilder;
|
@Inject EppResponse.Builder responseBuilder;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
|
@ -138,7 +138,7 @@ public final class HostCreateFlow implements TransactionalFlow {
|
||||||
.build());
|
.build());
|
||||||
// Only update DNS if this is a subordinate host. External hosts have no glue to write, so
|
// Only update DNS if this is a subordinate host. External hosts have no glue to write, so
|
||||||
// they are only written as NS records from the referencing domain.
|
// they are only written as NS records from the referencing domain.
|
||||||
dnsQueue.addHostRefreshTask(targetId);
|
dnsUtils.requestHostDnsRefresh(targetId);
|
||||||
}
|
}
|
||||||
tm().insertAll(entitiesToSave);
|
tm().insertAll(entitiesToSave);
|
||||||
return responseBuilder.setResData(HostCreateData.create(targetId, now)).build();
|
return responseBuilder.setResData(HostCreateData.create(targetId, now)).build();
|
||||||
|
|
|
@ -24,7 +24,7 @@ import static google.registry.model.eppoutput.Result.Code.SUCCESS;
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.flows.EppException;
|
import google.registry.flows.EppException;
|
||||||
import google.registry.flows.ExtensionManager;
|
import google.registry.flows.ExtensionManager;
|
||||||
import google.registry.flows.FlowModule.RegistrarId;
|
import google.registry.flows.FlowModule.RegistrarId;
|
||||||
|
@ -71,7 +71,7 @@ public final class HostDeleteFlow implements TransactionalFlow {
|
||||||
StatusValue.PENDING_DELETE,
|
StatusValue.PENDING_DELETE,
|
||||||
StatusValue.SERVER_DELETE_PROHIBITED);
|
StatusValue.SERVER_DELETE_PROHIBITED);
|
||||||
|
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
@Inject ExtensionManager extensionManager;
|
@Inject ExtensionManager extensionManager;
|
||||||
@Inject @RegistrarId String registrarId;
|
@Inject @RegistrarId String registrarId;
|
||||||
@Inject @TargetId String targetId;
|
@Inject @TargetId String targetId;
|
||||||
|
@ -104,7 +104,7 @@ public final class HostDeleteFlow implements TransactionalFlow {
|
||||||
}
|
}
|
||||||
Host newHost = existingHost.asBuilder().setStatusValues(null).setDeletionTime(now).build();
|
Host newHost = existingHost.asBuilder().setStatusValues(null).setDeletionTime(now).build();
|
||||||
if (existingHost.isSubordinate()) {
|
if (existingHost.isSubordinate()) {
|
||||||
dnsQueue.addHostRefreshTask(existingHost.getHostName());
|
dnsUtils.requestHostDnsRefresh(existingHost.getHostName());
|
||||||
tm().update(
|
tm().update(
|
||||||
tm().loadByKey(existingHost.getSuperordinateDomain())
|
tm().loadByKey(existingHost.getSuperordinateDomain())
|
||||||
.asBuilder()
|
.asBuilder()
|
||||||
|
|
|
@ -37,7 +37,7 @@ import com.google.common.collect.ImmutableMultimap;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import google.registry.batch.AsyncTaskEnqueuer;
|
import google.registry.batch.AsyncTaskEnqueuer;
|
||||||
import google.registry.batch.CloudTasksUtils;
|
import google.registry.batch.CloudTasksUtils;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.dns.RefreshDnsOnHostRenameAction;
|
import google.registry.dns.RefreshDnsOnHostRenameAction;
|
||||||
import google.registry.flows.EppException;
|
import google.registry.flows.EppException;
|
||||||
import google.registry.flows.EppException.ObjectAlreadyExistsException;
|
import google.registry.flows.EppException.ObjectAlreadyExistsException;
|
||||||
|
@ -124,7 +124,7 @@ public final class HostUpdateFlow implements TransactionalFlow {
|
||||||
@Inject @Superuser boolean isSuperuser;
|
@Inject @Superuser boolean isSuperuser;
|
||||||
@Inject HostHistory.Builder historyBuilder;
|
@Inject HostHistory.Builder historyBuilder;
|
||||||
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
@Inject AsyncTaskEnqueuer asyncTaskEnqueuer;
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
@Inject EppResponse.Builder responseBuilder;
|
@Inject EppResponse.Builder responseBuilder;
|
||||||
@Inject CloudTasksUtils cloudTasksUtils;
|
@Inject CloudTasksUtils cloudTasksUtils;
|
||||||
|
|
||||||
|
@ -266,14 +266,14 @@ public final class HostUpdateFlow implements TransactionalFlow {
|
||||||
// Only update DNS for subordinate hosts. External hosts have no glue to write, so they
|
// Only update DNS for subordinate hosts. External hosts have no glue to write, so they
|
||||||
// are only written as NS records from the referencing domain.
|
// are only written as NS records from the referencing domain.
|
||||||
if (existingHost.isSubordinate()) {
|
if (existingHost.isSubordinate()) {
|
||||||
dnsQueue.addHostRefreshTask(existingHost.getHostName());
|
dnsUtils.requestHostDnsRefresh(existingHost.getHostName());
|
||||||
}
|
}
|
||||||
// In case of a rename, there are many updates we need to queue up.
|
// In case of a rename, there are many updates we need to queue up.
|
||||||
if (((Update) resourceCommand).getInnerChange().getHostName() != null) {
|
if (((Update) resourceCommand).getInnerChange().getHostName() != null) {
|
||||||
// If the renamed host is also subordinate, then we must enqueue an update to write the new
|
// If the renamed host is also subordinate, then we must enqueue an update to write the new
|
||||||
// glue.
|
// glue.
|
||||||
if (newHost.isSubordinate()) {
|
if (newHost.isSubordinate()) {
|
||||||
dnsQueue.addHostRefreshTask(newHost.getHostName());
|
dnsUtils.requestHostDnsRefresh(newHost.getHostName());
|
||||||
}
|
}
|
||||||
// We must also enqueue updates for all domains that use this host as their nameserver so
|
// We must also enqueue updates for all domains that use this host as their nameserver so
|
||||||
// that their NS records can be updated to point at the new name.
|
// that their NS records can be updated to point at the new name.
|
||||||
|
|
|
@ -90,7 +90,10 @@ public class DatabaseMigrationStateSchedule extends CrossTldSingleton {
|
||||||
SEQUENCE_BASED_ALLOCATE_ID(PrimaryDatabase.CLOUD_SQL, false, ReplayDirection.NO_REPLAY),
|
SEQUENCE_BASED_ALLOCATE_ID(PrimaryDatabase.CLOUD_SQL, false, ReplayDirection.NO_REPLAY),
|
||||||
|
|
||||||
/** Use SQL-based Nordn upload flow instead of the pull queue-based one. */
|
/** Use SQL-based Nordn upload flow instead of the pull queue-based one. */
|
||||||
NORDN_SQL(PrimaryDatabase.CLOUD_SQL, false, ReplayDirection.NO_REPLAY);
|
NORDN_SQL(PrimaryDatabase.CLOUD_SQL, false, ReplayDirection.NO_REPLAY),
|
||||||
|
|
||||||
|
/** Use SQL-based DNS update flow instead of the pull queue-based one. */
|
||||||
|
DNS_SQL(PrimaryDatabase.CLOUD_SQL, false, ReplayDirection.NO_REPLAY);
|
||||||
|
|
||||||
private final PrimaryDatabase primaryDatabase;
|
private final PrimaryDatabase primaryDatabase;
|
||||||
private final boolean isReadOnly;
|
private final boolean isReadOnly;
|
||||||
|
@ -171,7 +174,11 @@ public class DatabaseMigrationStateSchedule extends CrossTldSingleton {
|
||||||
MigrationState.SQL_PRIMARY)
|
MigrationState.SQL_PRIMARY)
|
||||||
.putAll(MigrationState.SQL_ONLY, MigrationState.SEQUENCE_BASED_ALLOCATE_ID)
|
.putAll(MigrationState.SQL_ONLY, MigrationState.SEQUENCE_BASED_ALLOCATE_ID)
|
||||||
.putAll(MigrationState.SEQUENCE_BASED_ALLOCATE_ID, MigrationState.NORDN_SQL)
|
.putAll(MigrationState.SEQUENCE_BASED_ALLOCATE_ID, MigrationState.NORDN_SQL)
|
||||||
.putAll(MigrationState.NORDN_SQL, MigrationState.SEQUENCE_BASED_ALLOCATE_ID);
|
.putAll(
|
||||||
|
MigrationState.NORDN_SQL,
|
||||||
|
MigrationState.SEQUENCE_BASED_ALLOCATE_ID,
|
||||||
|
MigrationState.DNS_SQL)
|
||||||
|
.putAll(MigrationState.DNS_SQL, MigrationState.NORDN_SQL);
|
||||||
|
|
||||||
// In addition, we can always transition from a state to itself (useful when updating the map).
|
// In addition, we can always transition from a state to itself (useful when updating the map).
|
||||||
Arrays.stream(MigrationState.values()).forEach(state -> builder.put(state, state));
|
Arrays.stream(MigrationState.values()).forEach(state -> builder.put(state, state));
|
||||||
|
|
|
@ -0,0 +1,133 @@
|
||||||
|
// Copyright 2023 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.model.common;
|
||||||
|
|
||||||
|
import static com.google.common.base.Preconditions.checkArgument;
|
||||||
|
import static com.google.common.base.Preconditions.checkNotNull;
|
||||||
|
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||||
|
|
||||||
|
import google.registry.dns.DnsConstants.TargetType;
|
||||||
|
import google.registry.dns.PublishDnsUpdatesAction;
|
||||||
|
import google.registry.model.ImmutableObject;
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import javax.persistence.Column;
|
||||||
|
import javax.persistence.Entity;
|
||||||
|
import javax.persistence.EnumType;
|
||||||
|
import javax.persistence.Enumerated;
|
||||||
|
import javax.persistence.GeneratedValue;
|
||||||
|
import javax.persistence.GenerationType;
|
||||||
|
import javax.persistence.Id;
|
||||||
|
import javax.persistence.Index;
|
||||||
|
import javax.persistence.Table;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
|
@Entity
|
||||||
|
@Table(indexes = {@Index(columnList = "requestTime"), @Index(columnList = "lastProcessTime")})
|
||||||
|
public class DnsRefreshRequest extends ImmutableObject {
|
||||||
|
|
||||||
|
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
||||||
|
@Id
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
protected long id;
|
||||||
|
|
||||||
|
@Column(nullable = false)
|
||||||
|
@Enumerated(EnumType.STRING)
|
||||||
|
private TargetType type;
|
||||||
|
|
||||||
|
@Column(nullable = false)
|
||||||
|
private String name;
|
||||||
|
|
||||||
|
@Column(nullable = false)
|
||||||
|
private String tld;
|
||||||
|
|
||||||
|
@Column(nullable = false)
|
||||||
|
private DateTime requestTime;
|
||||||
|
|
||||||
|
@Column(nullable = false)
|
||||||
|
private DateTime lastProcessTime;
|
||||||
|
|
||||||
|
public TargetType getType() {
|
||||||
|
return type;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getName() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getTld() {
|
||||||
|
return tld;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateTime getRequestTime() {
|
||||||
|
return requestTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The time at which the entity was last processed.
|
||||||
|
*
|
||||||
|
* <p>Note that "processed" means that it was read, not necessarily that the DNS request was
|
||||||
|
* processed successfully. The subsequent steps to bundle requests together and enqueue them in a
|
||||||
|
* Cloud Tasks queue for {@link PublishDnsUpdatesAction} to process can still fail.
|
||||||
|
*
|
||||||
|
* <p>This value allows us to control if a row is just recently read and should be skipped, should
|
||||||
|
* there are concurrent reads that all attempt to read the rows with oldest {@link #requestTime},
|
||||||
|
* or another read that comes too early after the previous read.
|
||||||
|
*/
|
||||||
|
public DateTime getLastProcessTime() {
|
||||||
|
return lastProcessTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected DnsRefreshRequest() {}
|
||||||
|
|
||||||
|
private DnsRefreshRequest(
|
||||||
|
@Nullable Long id,
|
||||||
|
TargetType type,
|
||||||
|
String name,
|
||||||
|
String tld,
|
||||||
|
DateTime requestTime,
|
||||||
|
DateTime lastProcessTime) {
|
||||||
|
checkNotNull(type, "Target type cannot be null");
|
||||||
|
checkNotNull(name, "Domain/host name cannot be null");
|
||||||
|
checkNotNull(tld, "TLD cannot be null");
|
||||||
|
checkNotNull(requestTime, "Request time cannot be null");
|
||||||
|
checkNotNull(lastProcessTime, "Last process time cannot be null");
|
||||||
|
if (id != null) {
|
||||||
|
this.id = id;
|
||||||
|
}
|
||||||
|
this.type = type;
|
||||||
|
this.name = name;
|
||||||
|
this.tld = tld;
|
||||||
|
this.requestTime = requestTime;
|
||||||
|
this.lastProcessTime = lastProcessTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DnsRefreshRequest(TargetType type, String name, String tld, DateTime requestTime) {
|
||||||
|
this(null, type, name, tld, requestTime, START_OF_TIME);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DnsRefreshRequest updateProcessTime(DateTime processTime) {
|
||||||
|
checkArgument(
|
||||||
|
processTime.isAfter(getRequestTime()),
|
||||||
|
"Process time %s must be later than request time %s",
|
||||||
|
processTime,
|
||||||
|
getRequestTime());
|
||||||
|
checkArgument(
|
||||||
|
processTime.isAfter(getLastProcessTime()),
|
||||||
|
"New process time %s must be later than the old one %s",
|
||||||
|
processTime,
|
||||||
|
getLastProcessTime());
|
||||||
|
return new DnsRefreshRequest(id, getType(), getName(), getTld(), getRequestTime(), processTime);
|
||||||
|
}
|
||||||
|
}
|
|
@ -21,7 +21,7 @@ import static google.registry.request.RequestParameters.PARAM_TLDS;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.flogger.FluentLogger;
|
import com.google.common.flogger.FluentLogger;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.request.Action;
|
import google.registry.request.Action;
|
||||||
import google.registry.request.Parameter;
|
import google.registry.request.Parameter;
|
||||||
import google.registry.request.Response;
|
import google.registry.request.Response;
|
||||||
|
@ -66,7 +66,7 @@ public class RefreshDnsForAllDomainsAction implements Runnable {
|
||||||
@Parameter("smearMinutes")
|
@Parameter("smearMinutes")
|
||||||
int smearMinutes;
|
int smearMinutes;
|
||||||
|
|
||||||
@Inject DnsQueue dnsQueue;
|
@Inject DnsUtils dnsUtils;
|
||||||
@Inject Clock clock;
|
@Inject Clock clock;
|
||||||
@Inject Random random;
|
@Inject Random random;
|
||||||
|
|
||||||
|
@ -91,7 +91,7 @@ public class RefreshDnsForAllDomainsAction implements Runnable {
|
||||||
domainName -> {
|
domainName -> {
|
||||||
try {
|
try {
|
||||||
// Smear the task execution time over the next N minutes.
|
// Smear the task execution time over the next N minutes.
|
||||||
dnsQueue.addDomainRefreshTask(
|
dnsUtils.requestDomainDnsRefresh(
|
||||||
domainName, Duration.standardMinutes(random.nextInt(smearMinutes)));
|
domainName, Duration.standardMinutes(random.nextInt(smearMinutes)));
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
logger.atSevere().withCause(t).log(
|
logger.atSevere().withCause(t).log(
|
||||||
|
|
|
@ -43,6 +43,7 @@
|
||||||
<class>google.registry.model.billing.BillingEvent$Recurring</class>
|
<class>google.registry.model.billing.BillingEvent$Recurring</class>
|
||||||
<class>google.registry.model.common.Cursor</class>
|
<class>google.registry.model.common.Cursor</class>
|
||||||
<class>google.registry.model.common.DatabaseMigrationStateSchedule</class>
|
<class>google.registry.model.common.DatabaseMigrationStateSchedule</class>
|
||||||
|
<class>google.registry.model.common.DnsRefreshRequest</class>
|
||||||
<class>google.registry.model.console.User</class>
|
<class>google.registry.model.console.User</class>
|
||||||
<class>google.registry.model.contact.ContactHistory</class>
|
<class>google.registry.model.contact.ContactHistory</class>
|
||||||
<class>google.registry.model.contact.Contact</class>
|
<class>google.registry.model.contact.Contact</class>
|
||||||
|
|
|
@ -27,14 +27,14 @@ import static google.registry.testing.DatabaseHelper.persistDeletedDomain;
|
||||||
import static google.registry.testing.DatabaseHelper.persistDomainAsDeleted;
|
import static google.registry.testing.DatabaseHelper.persistDomainAsDeleted;
|
||||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||||
import static google.registry.testing.DatabaseHelper.persistSimpleResource;
|
import static google.registry.testing.DatabaseHelper.persistSimpleResource;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
|
||||||
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||||
import static org.joda.time.DateTimeZone.UTC;
|
import static org.joda.time.DateTimeZone.UTC;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import google.registry.config.RegistryEnvironment;
|
import google.registry.config.RegistryEnvironment;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.billing.BillingEvent;
|
import google.registry.model.billing.BillingEvent;
|
||||||
import google.registry.model.billing.BillingEvent.Reason;
|
import google.registry.model.billing.BillingEvent.Reason;
|
||||||
|
@ -47,9 +47,8 @@ import google.registry.model.tld.Registry.TldType;
|
||||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.DnsUtilsHelper;
|
||||||
import google.registry.testing.SystemPropertyExtension;
|
import google.registry.testing.SystemPropertyExtension;
|
||||||
import google.registry.testing.TaskQueueExtension;
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import org.joda.money.Money;
|
import org.joda.money.Money;
|
||||||
|
@ -68,13 +67,14 @@ class DeleteProberDataActionTest {
|
||||||
final JpaIntegrationTestExtension jpa =
|
final JpaIntegrationTestExtension jpa =
|
||||||
new JpaTestExtensions.Builder().buildIntegrationTestExtension();
|
new JpaTestExtensions.Builder().buildIntegrationTestExtension();
|
||||||
|
|
||||||
@RegisterExtension TaskQueueExtension taskQueue = new TaskQueueExtension();
|
|
||||||
|
|
||||||
@RegisterExtension
|
@RegisterExtension
|
||||||
final SystemPropertyExtension systemPropertyExtension = new SystemPropertyExtension();
|
final SystemPropertyExtension systemPropertyExtension = new SystemPropertyExtension();
|
||||||
|
|
||||||
private DeleteProberDataAction action;
|
private DeleteProberDataAction action;
|
||||||
|
|
||||||
|
private final DnsUtils dnsUtils = mock(DnsUtils.class);
|
||||||
|
private final DnsUtilsHelper dnsUtilsHelper = new DnsUtilsHelper(dnsUtils);
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
void beforeEach() {
|
void beforeEach() {
|
||||||
// Entities in these two should not be touched.
|
// Entities in these two should not be touched.
|
||||||
|
@ -99,7 +99,7 @@ class DeleteProberDataActionTest {
|
||||||
|
|
||||||
private void resetAction() {
|
private void resetAction() {
|
||||||
action = new DeleteProberDataAction();
|
action = new DeleteProberDataAction();
|
||||||
action.dnsQueue = DnsQueue.createForTesting(new FakeClock());
|
action.dnsUtils = dnsUtils;
|
||||||
action.isDryRun = false;
|
action.isDryRun = false;
|
||||||
action.tlds = ImmutableSet.of();
|
action.tlds = ImmutableSet.of();
|
||||||
action.registryAdminRegistrarId = "TheRegistrar";
|
action.registryAdminRegistrarId = "TheRegistrar";
|
||||||
|
@ -201,7 +201,7 @@ class DeleteProberDataActionTest {
|
||||||
DateTime timeAfterDeletion = DateTime.now(UTC);
|
DateTime timeAfterDeletion = DateTime.now(UTC);
|
||||||
assertThat(loadByForeignKey(Domain.class, "blah.ib-any.test", timeAfterDeletion)).isEmpty();
|
assertThat(loadByForeignKey(Domain.class, "blah.ib-any.test", timeAfterDeletion)).isEmpty();
|
||||||
assertThat(loadByEntity(domain).getDeletionTime()).isLessThan(timeAfterDeletion);
|
assertThat(loadByEntity(domain).getDeletionTime()).isLessThan(timeAfterDeletion);
|
||||||
assertDnsTasksEnqueued("blah.ib-any.test");
|
dnsUtilsHelper.assertDomainDnsRequests("blah.ib-any.test");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -218,7 +218,7 @@ class DeleteProberDataActionTest {
|
||||||
action.run();
|
action.run();
|
||||||
assertThat(loadByForeignKey(Domain.class, "blah.ib-any.test", timeAfterDeletion)).isEmpty();
|
assertThat(loadByForeignKey(Domain.class, "blah.ib-any.test", timeAfterDeletion)).isEmpty();
|
||||||
assertThat(loadByEntity(domain).getDeletionTime()).isLessThan(timeAfterDeletion);
|
assertThat(loadByEntity(domain).getDeletionTime()).isLessThan(timeAfterDeletion);
|
||||||
assertDnsTasksEnqueued("blah.ib-any.test");
|
dnsUtilsHelper.assertDomainDnsRequests("blah.ib-any.test");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
170
core/src/test/java/google/registry/dns/DnsUtilsTest.java
Normal file
170
core/src/test/java/google/registry/dns/DnsUtilsTest.java
Normal file
|
@ -0,0 +1,170 @@
|
||||||
|
// Copyright 2023 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.dns;
|
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat;
|
||||||
|
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||||
|
import static google.registry.testing.DatabaseHelper.createTld;
|
||||||
|
import static google.registry.testing.DatabaseHelper.loadAllOf;
|
||||||
|
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||||
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
|
import static org.mockito.ArgumentMatchers.anyString;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.never;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableSortedMap;
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
|
import com.google.common.collect.Ordering;
|
||||||
|
import google.registry.dns.DnsConstants.TargetType;
|
||||||
|
import google.registry.model.common.DatabaseMigrationStateSchedule;
|
||||||
|
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
|
||||||
|
import google.registry.model.common.DnsRefreshRequest;
|
||||||
|
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||||
|
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
||||||
|
import google.registry.testing.FakeClock;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Duration;
|
||||||
|
import org.junit.jupiter.api.Assertions;
|
||||||
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||||
|
|
||||||
|
/** Unit tests for {@link DnsUtils}. */
|
||||||
|
public class DnsUtilsTest {
|
||||||
|
|
||||||
|
private static final String tld = "tld";
|
||||||
|
private static final String domainName = "test.tld";
|
||||||
|
private static final String hostName = "ns1.test.tld";
|
||||||
|
|
||||||
|
private final DnsQueue dnsQueue = mock(DnsQueue.class);
|
||||||
|
private final DnsUtils dnsUtils = new DnsUtils(dnsQueue);
|
||||||
|
|
||||||
|
FakeClock clock = new FakeClock(DateTime.parse("2020-02-02T01:23:45Z"));
|
||||||
|
|
||||||
|
@RegisterExtension
|
||||||
|
JpaIntegrationTestExtension jpa =
|
||||||
|
new JpaTestExtensions.Builder().withClock(clock).buildIntegrationTestExtension();
|
||||||
|
|
||||||
|
@BeforeAll
|
||||||
|
static void beforeAll() {
|
||||||
|
DatabaseMigrationStateSchedule.useUncachedForTest();
|
||||||
|
}
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
void beforeEach() {
|
||||||
|
createTld(tld);
|
||||||
|
when(dnsQueue.getClock()).thenReturn(clock);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testSuccess_hostRefresh_pullQueue() {
|
||||||
|
dnsUtils.requestHostDnsRefresh(hostName);
|
||||||
|
verify(dnsQueue).addHostRefreshTask(hostName);
|
||||||
|
assertThat(loadAllOf(DnsRefreshRequest.class)).isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testSuccess_domainRefresh_pullQueue() {
|
||||||
|
dnsUtils.requestDomainDnsRefresh(domainName);
|
||||||
|
verify(dnsQueue).addDomainRefreshTask(domainName, Duration.ZERO);
|
||||||
|
assertThat(loadAllOf(DnsRefreshRequest.class)).isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testSuccess_domainRefreshWithDelay_pullQueue() {
|
||||||
|
dnsUtils.requestDomainDnsRefresh(domainName, Duration.standardMinutes(3));
|
||||||
|
verify(dnsQueue).addDomainRefreshTask(domainName, Duration.standardMinutes(3));
|
||||||
|
assertThat(loadAllOf(DnsRefreshRequest.class)).isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testFailure_hostRefresh_unmanagedHost() {
|
||||||
|
String unmanagedHostName = "ns1.another.example";
|
||||||
|
Assertions.assertThrows(
|
||||||
|
IllegalArgumentException.class, () -> dnsUtils.requestHostDnsRefresh(unmanagedHostName));
|
||||||
|
verify(dnsQueue, never()).addHostRefreshTask(anyString());
|
||||||
|
assertThat(loadAllOf(DnsRefreshRequest.class)).isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testFailure_domainRefresh_unmanagedDomain() {
|
||||||
|
String unmanagedDomainName = "another.example";
|
||||||
|
Assertions.assertThrows(
|
||||||
|
IllegalArgumentException.class,
|
||||||
|
() -> dnsUtils.requestDomainDnsRefresh(unmanagedDomainName));
|
||||||
|
verify(dnsQueue, never()).addDomainRefreshTask(anyString(), any(Duration.class));
|
||||||
|
assertThat(loadAllOf(DnsRefreshRequest.class)).isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testSuccess_hostRefresh() {
|
||||||
|
useDnsSql();
|
||||||
|
dnsUtils.requestHostDnsRefresh(hostName);
|
||||||
|
verify(dnsQueue, never()).addHostRefreshTask(anyString());
|
||||||
|
DnsRefreshRequest request = Iterables.getOnlyElement(loadAllOf(DnsRefreshRequest.class));
|
||||||
|
assertRequest(request, TargetType.HOST, hostName, tld, clock.nowUtc());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testSuccess_domainRefresh() {
|
||||||
|
useDnsSql();
|
||||||
|
dnsUtils.requestDomainDnsRefresh(domainName);
|
||||||
|
verify(dnsQueue, never()).addDomainRefreshTask(anyString(), any(Duration.class));
|
||||||
|
DnsRefreshRequest request = Iterables.getOnlyElement(loadAllOf(DnsRefreshRequest.class));
|
||||||
|
assertRequest(request, TargetType.DOMAIN, domainName, tld, clock.nowUtc());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testSuccess_domainRefreshWithDelay() {
|
||||||
|
useDnsSql();
|
||||||
|
dnsUtils.requestDomainDnsRefresh(domainName, Duration.standardMinutes(3));
|
||||||
|
verify(dnsQueue, never()).addDomainRefreshTask(anyString(), any(Duration.class));
|
||||||
|
DnsRefreshRequest request = Iterables.getOnlyElement(loadAllOf(DnsRefreshRequest.class));
|
||||||
|
assertRequest(request, TargetType.DOMAIN, domainName, tld, clock.nowUtc().plusMinutes(3));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void assertRequest(
|
||||||
|
DnsRefreshRequest request, TargetType type, String name, String tld, DateTime requestTime) {
|
||||||
|
assertThat(request.getType()).isEqualTo(type);
|
||||||
|
assertThat(request.getName()).isEqualTo(name);
|
||||||
|
assertThat(request.getTld()).isEqualTo(tld);
|
||||||
|
assertThat(request.getRequestTime()).isEqualTo(requestTime);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void useDnsSql() {
|
||||||
|
DateTime currentTime = clock.nowUtc();
|
||||||
|
clock.setTo(START_OF_TIME);
|
||||||
|
tm().transact(
|
||||||
|
() ->
|
||||||
|
DatabaseMigrationStateSchedule.set(
|
||||||
|
new ImmutableSortedMap.Builder<DateTime, MigrationState>(Ordering.natural())
|
||||||
|
.put(START_OF_TIME, MigrationState.DATASTORE_ONLY)
|
||||||
|
.put(START_OF_TIME.plusMillis(1), MigrationState.DATASTORE_PRIMARY)
|
||||||
|
.put(START_OF_TIME.plusMillis(2), MigrationState.DATASTORE_PRIMARY_NO_ASYNC)
|
||||||
|
.put(
|
||||||
|
START_OF_TIME.plusMillis(3), MigrationState.DATASTORE_PRIMARY_READ_ONLY)
|
||||||
|
.put(START_OF_TIME.plusMillis(4), MigrationState.SQL_PRIMARY_READ_ONLY)
|
||||||
|
.put(START_OF_TIME.plusMillis(5), MigrationState.SQL_PRIMARY)
|
||||||
|
.put(START_OF_TIME.plusMillis(6), MigrationState.SQL_ONLY)
|
||||||
|
.put(START_OF_TIME.plusMillis(7), MigrationState.SEQUENCE_BASED_ALLOCATE_ID)
|
||||||
|
.put(START_OF_TIME.plusMillis(8), MigrationState.NORDN_SQL)
|
||||||
|
.put(START_OF_TIME.plusMillis(9), MigrationState.DNS_SQL)
|
||||||
|
.build()));
|
||||||
|
clock.setTo(currentTime);
|
||||||
|
}
|
||||||
|
}
|
|
@ -54,6 +54,7 @@ import google.registry.request.HttpException.ServiceUnavailableException;
|
||||||
import google.registry.request.lock.LockHandler;
|
import google.registry.request.lock.LockHandler;
|
||||||
import google.registry.testing.CloudTasksHelper;
|
import google.registry.testing.CloudTasksHelper;
|
||||||
import google.registry.testing.CloudTasksHelper.TaskMatcher;
|
import google.registry.testing.CloudTasksHelper.TaskMatcher;
|
||||||
|
import google.registry.testing.DnsUtilsHelper;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
import google.registry.testing.FakeLockHandler;
|
import google.registry.testing.FakeLockHandler;
|
||||||
import google.registry.testing.FakeResponse;
|
import google.registry.testing.FakeResponse;
|
||||||
|
@ -82,7 +83,8 @@ public class PublishDnsUpdatesActionTest {
|
||||||
private final FakeLockHandler lockHandler = new FakeLockHandler(true);
|
private final FakeLockHandler lockHandler = new FakeLockHandler(true);
|
||||||
private final DnsWriter dnsWriter = mock(DnsWriter.class);
|
private final DnsWriter dnsWriter = mock(DnsWriter.class);
|
||||||
private final DnsMetrics dnsMetrics = mock(DnsMetrics.class);
|
private final DnsMetrics dnsMetrics = mock(DnsMetrics.class);
|
||||||
private final DnsQueue dnsQueue = mock(DnsQueue.class);
|
private final DnsUtils dnsUtils = mock(DnsUtils.class);
|
||||||
|
private final DnsUtilsHelper dnsUtilsHelper = new DnsUtilsHelper(dnsUtils);
|
||||||
private final CloudTasksHelper cloudTasksHelper = new CloudTasksHelper();
|
private final CloudTasksHelper cloudTasksHelper = new CloudTasksHelper();
|
||||||
private PublishDnsUpdatesAction action;
|
private PublishDnsUpdatesAction action;
|
||||||
private InternetAddress outgoingRegistry;
|
private InternetAddress outgoingRegistry;
|
||||||
|
@ -162,7 +164,7 @@ public class PublishDnsUpdatesActionTest {
|
||||||
outgoingRegistry,
|
outgoingRegistry,
|
||||||
Optional.ofNullable(retryCount),
|
Optional.ofNullable(retryCount),
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
dnsQueue,
|
dnsUtils,
|
||||||
new DnsWriterProxy(ImmutableMap.of("correctWriter", dnsWriter)),
|
new DnsWriterProxy(ImmutableMap.of("correctWriter", dnsWriter)),
|
||||||
dnsMetrics,
|
dnsMetrics,
|
||||||
lockHandler,
|
lockHandler,
|
||||||
|
@ -196,7 +198,7 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -223,7 +225,7 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -276,7 +278,7 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -496,7 +498,7 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -526,7 +528,7 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -554,7 +556,7 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -580,9 +582,9 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verify(dnsQueue).addDomainRefreshTask("example.com");
|
dnsUtilsHelper.assertDomainDnsRequests("example.com");
|
||||||
verify(dnsQueue).addHostRefreshTask("ns1.example.com");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.com");
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -608,9 +610,9 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verify(dnsQueue).addDomainRefreshTask("example.com");
|
dnsUtilsHelper.assertDomainDnsRequests("example.com");
|
||||||
verify(dnsQueue).addHostRefreshTask("ns1.example.com");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.com");
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -632,11 +634,11 @@ public class PublishDnsUpdatesActionTest {
|
||||||
Duration.standardHours(2),
|
Duration.standardHours(2),
|
||||||
Duration.standardHours(1));
|
Duration.standardHours(1));
|
||||||
verifyNoMoreInteractions(dnsMetrics);
|
verifyNoMoreInteractions(dnsMetrics);
|
||||||
verify(dnsQueue).addDomainRefreshTask("example.com");
|
dnsUtilsHelper.assertDomainDnsRequests("example.com");
|
||||||
verify(dnsQueue).addDomainRefreshTask("example2.com");
|
dnsUtilsHelper.assertDomainDnsRequests("example2.com");
|
||||||
verify(dnsQueue).addHostRefreshTask("ns1.example.com");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.com");
|
||||||
verify(dnsQueue).addHostRefreshTask("ns2.example.com");
|
dnsUtilsHelper.assertHostDnsRequests("ns2.example.com");
|
||||||
verify(dnsQueue).addHostRefreshTask("ns1.example2.com");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example2.com");
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -388,21 +388,6 @@ public class ReadDnsQueueActionTest {
|
||||||
ImmutableMultimap.of("com", "comWriter", "example", "exampleWriter"));
|
ImmutableMultimap.of("com", "comWriter", "example", "exampleWriter"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@RetryingTest(4)
|
|
||||||
void testSuccess_zone_getsIgnored() {
|
|
||||||
dnsQueue.addHostRefreshTask("ns1.domain.com");
|
|
||||||
dnsQueue.addDomainRefreshTask("domain.net");
|
|
||||||
dnsQueue.addZoneRefreshTask("example");
|
|
||||||
|
|
||||||
run();
|
|
||||||
|
|
||||||
TaskQueueHelper.assertNoTasksEnqueued(DNS_PULL_QUEUE_NAME);
|
|
||||||
cloudTasksHelper.assertTasksEnqueued(
|
|
||||||
DNS_PUBLISH_PUSH_QUEUE_NAME,
|
|
||||||
new TaskMatcher().url(PublishDnsUpdatesAction.PATH).param("domains", "domain.net"),
|
|
||||||
new TaskMatcher().url(PublishDnsUpdatesAction.PATH).param("hosts", "ns1.domain.com"));
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String makeCommaSeparatedRange(int from, int to, String format) {
|
private static String makeCommaSeparatedRange(int from, int to, String format) {
|
||||||
return IntStream.range(from, to)
|
return IntStream.range(from, to)
|
||||||
.mapToObj(i -> String.format(format, i))
|
.mapToObj(i -> String.format(format, i))
|
||||||
|
|
|
@ -21,8 +21,6 @@ import static google.registry.testing.DatabaseHelper.persistActiveHost;
|
||||||
import static google.registry.testing.DatabaseHelper.persistActiveSubordinateHost;
|
import static google.registry.testing.DatabaseHelper.persistActiveSubordinateHost;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.verify;
|
|
||||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
|
||||||
|
|
||||||
import google.registry.dns.DnsConstants.TargetType;
|
import google.registry.dns.DnsConstants.TargetType;
|
||||||
import google.registry.model.domain.Domain;
|
import google.registry.model.domain.Domain;
|
||||||
|
@ -30,6 +28,7 @@ import google.registry.persistence.transaction.JpaTestExtensions;
|
||||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
||||||
import google.registry.request.HttpException.BadRequestException;
|
import google.registry.request.HttpException.BadRequestException;
|
||||||
import google.registry.request.HttpException.NotFoundException;
|
import google.registry.request.HttpException.NotFoundException;
|
||||||
|
import google.registry.testing.DnsUtilsHelper;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
@ -42,11 +41,12 @@ public class RefreshDnsActionTest {
|
||||||
final JpaIntegrationTestExtension jpa =
|
final JpaIntegrationTestExtension jpa =
|
||||||
new JpaTestExtensions.Builder().buildIntegrationTestExtension();
|
new JpaTestExtensions.Builder().buildIntegrationTestExtension();
|
||||||
|
|
||||||
private final DnsQueue dnsQueue = mock(DnsQueue.class);
|
private final DnsUtils dnsUtils = mock(DnsUtils.class);
|
||||||
|
private final DnsUtilsHelper dnsUtilsHelper = new DnsUtilsHelper(dnsUtils);
|
||||||
private final FakeClock clock = new FakeClock();
|
private final FakeClock clock = new FakeClock();
|
||||||
|
|
||||||
private void run(TargetType type, String name) {
|
private void run(TargetType type, String name) {
|
||||||
new RefreshDnsAction(name, type, clock, dnsQueue).run();
|
new RefreshDnsAction(name, type, clock, dnsUtils).run();
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
|
@ -59,8 +59,8 @@ public class RefreshDnsActionTest {
|
||||||
Domain domain = persistActiveDomain("example.xn--q9jyb4c");
|
Domain domain = persistActiveDomain("example.xn--q9jyb4c");
|
||||||
persistActiveSubordinateHost("ns1.example.xn--q9jyb4c", domain);
|
persistActiveSubordinateHost("ns1.example.xn--q9jyb4c", domain);
|
||||||
run(TargetType.HOST, "ns1.example.xn--q9jyb4c");
|
run(TargetType.HOST, "ns1.example.xn--q9jyb4c");
|
||||||
verify(dnsQueue).addHostRefreshTask("ns1.example.xn--q9jyb4c");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.xn--q9jyb4c");
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -74,7 +74,7 @@ public class RefreshDnsActionTest {
|
||||||
try {
|
try {
|
||||||
run(TargetType.HOST, "ns1.example.xn--q9jyb4c");
|
run(TargetType.HOST, "ns1.example.xn--q9jyb4c");
|
||||||
} finally {
|
} finally {
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
assertThat(thrown)
|
assertThat(thrown)
|
||||||
|
@ -86,8 +86,8 @@ public class RefreshDnsActionTest {
|
||||||
void testSuccess_domain() {
|
void testSuccess_domain() {
|
||||||
persistActiveDomain("example.xn--q9jyb4c");
|
persistActiveDomain("example.xn--q9jyb4c");
|
||||||
run(TargetType.DOMAIN, "example.xn--q9jyb4c");
|
run(TargetType.DOMAIN, "example.xn--q9jyb4c");
|
||||||
verify(dnsQueue).addDomainRefreshTask("example.xn--q9jyb4c");
|
dnsUtilsHelper.assertDomainDnsRequests("example.xn--q9jyb4c");
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -24,14 +24,13 @@ import static google.registry.testing.DatabaseHelper.persistResource;
|
||||||
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
|
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
|
||||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.verify;
|
|
||||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import google.registry.model.eppcommon.StatusValue;
|
import google.registry.model.eppcommon.StatusValue;
|
||||||
import google.registry.model.host.Host;
|
import google.registry.model.host.Host;
|
||||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
||||||
|
import google.registry.testing.DnsUtilsHelper;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
import google.registry.testing.FakeResponse;
|
import google.registry.testing.FakeResponse;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -43,7 +42,8 @@ import org.junit.jupiter.api.extension.RegisterExtension;
|
||||||
public class RefreshDnsOnHostRenameActionTest {
|
public class RefreshDnsOnHostRenameActionTest {
|
||||||
|
|
||||||
private final FakeClock clock = new FakeClock(DateTime.parse("2015-01-15T11:22:33Z"));
|
private final FakeClock clock = new FakeClock(DateTime.parse("2015-01-15T11:22:33Z"));
|
||||||
private final DnsQueue dnsQueue = mock(DnsQueue.class);
|
private final DnsUtils dnsUtils = mock(DnsUtils.class);
|
||||||
|
private final DnsUtilsHelper dnsUtilsHelper = new DnsUtilsHelper(dnsUtils);
|
||||||
private final FakeResponse response = new FakeResponse();
|
private final FakeResponse response = new FakeResponse();
|
||||||
|
|
||||||
@RegisterExtension
|
@RegisterExtension
|
||||||
|
@ -53,14 +53,7 @@ public class RefreshDnsOnHostRenameActionTest {
|
||||||
private RefreshDnsOnHostRenameAction action;
|
private RefreshDnsOnHostRenameAction action;
|
||||||
|
|
||||||
private void createAction(String hostKey) {
|
private void createAction(String hostKey) {
|
||||||
action = new RefreshDnsOnHostRenameAction(hostKey, response, dnsQueue);
|
action = new RefreshDnsOnHostRenameAction(hostKey, response, dnsUtils);
|
||||||
}
|
|
||||||
|
|
||||||
private void assertDnsTasksEnqueued(String... domains) {
|
|
||||||
for (String domain : domains) {
|
|
||||||
verify(dnsQueue).addDomainRefreshTask(domain);
|
|
||||||
}
|
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
|
@ -82,7 +75,7 @@ public class RefreshDnsOnHostRenameActionTest {
|
||||||
persistDomainAsDeleted(newDomain("deleted.tld", host), clock.nowUtc().minusDays(1));
|
persistDomainAsDeleted(newDomain("deleted.tld", host), clock.nowUtc().minusDays(1));
|
||||||
createAction(host.createVKey().stringify());
|
createAction(host.createVKey().stringify());
|
||||||
action.run();
|
action.run();
|
||||||
assertDnsTasksEnqueued("example.tld", "otherexample.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld", "otherexample.tld");
|
||||||
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -90,7 +83,7 @@ public class RefreshDnsOnHostRenameActionTest {
|
||||||
void testFailure_nonexistentHost() {
|
void testFailure_nonexistentHost() {
|
||||||
createAction("kind:Host@sql:rO0ABXQABGJsYWg");
|
createAction("kind:Host@sql:rO0ABXQABGJsYWg");
|
||||||
action.run();
|
action.run();
|
||||||
assertDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
||||||
assertThat(response.getPayload())
|
assertThat(response.getPayload())
|
||||||
.isEqualTo("Host to refresh does not exist: VKey<Host>(sql:blah)");
|
.isEqualTo("Host to refresh does not exist: VKey<Host>(sql:blah)");
|
||||||
|
@ -102,7 +95,7 @@ public class RefreshDnsOnHostRenameActionTest {
|
||||||
persistResource(newDomain("example.tld", host));
|
persistResource(newDomain("example.tld", host));
|
||||||
createAction(host.createVKey().stringify());
|
createAction(host.createVKey().stringify());
|
||||||
action.run();
|
action.run();
|
||||||
assertDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
||||||
assertThat(response.getPayload())
|
assertThat(response.getPayload())
|
||||||
.isEqualTo("Host to refresh is already deleted: ns1.example.tld");
|
.isEqualTo("Host to refresh is already deleted: ns1.example.tld");
|
||||||
|
|
|
@ -24,6 +24,7 @@ import google.registry.batch.CloudTasksUtils;
|
||||||
import google.registry.config.RegistryConfig.ConfigModule;
|
import google.registry.config.RegistryConfig.ConfigModule;
|
||||||
import google.registry.config.RegistryConfig.ConfigModule.TmchCaMode;
|
import google.registry.config.RegistryConfig.ConfigModule.TmchCaMode;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsQueue;
|
||||||
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.flows.custom.CustomLogicFactory;
|
import google.registry.flows.custom.CustomLogicFactory;
|
||||||
import google.registry.flows.custom.TestCustomLogicFactory;
|
import google.registry.flows.custom.TestCustomLogicFactory;
|
||||||
import google.registry.flows.domain.DomainFlowTmchUtils;
|
import google.registry.flows.domain.DomainFlowTmchUtils;
|
||||||
|
@ -31,6 +32,7 @@ import google.registry.monitoring.whitebox.EppMetric;
|
||||||
import google.registry.request.RequestScope;
|
import google.registry.request.RequestScope;
|
||||||
import google.registry.request.lock.LockHandler;
|
import google.registry.request.lock.LockHandler;
|
||||||
import google.registry.testing.CloudTasksHelper;
|
import google.registry.testing.CloudTasksHelper;
|
||||||
|
import google.registry.testing.DnsUtilsHelper;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
import google.registry.testing.FakeLockHandler;
|
import google.registry.testing.FakeLockHandler;
|
||||||
import google.registry.testing.FakeSleeper;
|
import google.registry.testing.FakeSleeper;
|
||||||
|
@ -59,11 +61,16 @@ public interface EppTestComponent {
|
||||||
private FakeLockHandler lockHandler;
|
private FakeLockHandler lockHandler;
|
||||||
private Sleeper sleeper;
|
private Sleeper sleeper;
|
||||||
private CloudTasksHelper cloudTasksHelper;
|
private CloudTasksHelper cloudTasksHelper;
|
||||||
|
private DnsUtilsHelper dnsUtilsHelper;
|
||||||
|
|
||||||
public CloudTasksHelper getCloudTasksHelper() {
|
public CloudTasksHelper getCloudTasksHelper() {
|
||||||
return cloudTasksHelper;
|
return cloudTasksHelper;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public DnsUtilsHelper getDnsUtilsHelper() {
|
||||||
|
return dnsUtilsHelper;
|
||||||
|
}
|
||||||
|
|
||||||
public EppMetric.Builder getMetricBuilder() {
|
public EppMetric.Builder getMetricBuilder() {
|
||||||
return metricBuilder;
|
return metricBuilder;
|
||||||
}
|
}
|
||||||
|
@ -82,6 +89,7 @@ public interface EppTestComponent {
|
||||||
instance.metricBuilder = EppMetric.builderForRequest(clock);
|
instance.metricBuilder = EppMetric.builderForRequest(clock);
|
||||||
instance.lockHandler = new FakeLockHandler(true);
|
instance.lockHandler = new FakeLockHandler(true);
|
||||||
instance.cloudTasksHelper = cloudTasksHelper;
|
instance.cloudTasksHelper = cloudTasksHelper;
|
||||||
|
instance.dnsUtilsHelper = new DnsUtilsHelper();
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -95,6 +103,11 @@ public interface EppTestComponent {
|
||||||
return cloudTasksHelper.getTestCloudTasksUtils();
|
return cloudTasksHelper.getTestCloudTasksUtils();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Provides
|
||||||
|
DnsUtils provideDnsUtils() {
|
||||||
|
return dnsUtilsHelper.getDnsUtils();
|
||||||
|
}
|
||||||
|
|
||||||
@Provides
|
@Provides
|
||||||
Clock provideClock() {
|
Clock provideClock() {
|
||||||
return clock;
|
return clock;
|
||||||
|
|
|
@ -45,6 +45,7 @@ import google.registry.persistence.transaction.JpaTestExtensions;
|
||||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
||||||
import google.registry.testing.CloudTasksHelper;
|
import google.registry.testing.CloudTasksHelper;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
|
import google.registry.testing.DnsUtilsHelper;
|
||||||
import google.registry.testing.EppLoader;
|
import google.registry.testing.EppLoader;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
import google.registry.testing.FakeHttpSession;
|
import google.registry.testing.FakeHttpSession;
|
||||||
|
@ -84,6 +85,7 @@ public abstract class FlowTestCase<F extends Flow> {
|
||||||
protected TransportCredentials credentials = new PasswordOnlyTransportCredentials();
|
protected TransportCredentials credentials = new PasswordOnlyTransportCredentials();
|
||||||
protected EppRequestSource eppRequestSource = EppRequestSource.UNIT_TEST;
|
protected EppRequestSource eppRequestSource = EppRequestSource.UNIT_TEST;
|
||||||
protected CloudTasksHelper cloudTasksHelper;
|
protected CloudTasksHelper cloudTasksHelper;
|
||||||
|
protected DnsUtilsHelper dnsUtilsHelper;
|
||||||
|
|
||||||
private EppMetric.Builder eppMetricBuilder;
|
private EppMetric.Builder eppMetricBuilder;
|
||||||
|
|
||||||
|
@ -216,6 +218,7 @@ public abstract class FlowTestCase<F extends Flow> {
|
||||||
|
|
||||||
FakesAndMocksModule fakesAndMocksModule = FakesAndMocksModule.create(clock);
|
FakesAndMocksModule fakesAndMocksModule = FakesAndMocksModule.create(clock);
|
||||||
cloudTasksHelper = fakesAndMocksModule.getCloudTasksHelper();
|
cloudTasksHelper = fakesAndMocksModule.getCloudTasksHelper();
|
||||||
|
dnsUtilsHelper = fakesAndMocksModule.getDnsUtilsHelper();
|
||||||
// Run the flow.
|
// Run the flow.
|
||||||
return DaggerEppTestComponent.builder()
|
return DaggerEppTestComponent.builder()
|
||||||
.fakesAndMocksModule(fakesAndMocksModule)
|
.fakesAndMocksModule(fakesAndMocksModule)
|
||||||
|
|
|
@ -54,8 +54,6 @@ import static google.registry.testing.DatabaseHelper.persistReservedList;
|
||||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||||
import static google.registry.testing.DomainSubject.assertAboutDomains;
|
import static google.registry.testing.DomainSubject.assertAboutDomains;
|
||||||
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
||||||
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.assertNoTasksEnqueued;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertTasksEnqueued;
|
import static google.registry.testing.TaskQueueHelper.assertTasksEnqueued;
|
||||||
import static google.registry.tmch.LordnTaskUtils.QUEUE_CLAIMS;
|
import static google.registry.tmch.LordnTaskUtils.QUEUE_CLAIMS;
|
||||||
|
@ -393,7 +391,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
GracePeriod.create(
|
GracePeriod.create(
|
||||||
GracePeriodStatus.ADD, domain.getRepoId(), billingTime, "TheRegistrar", null),
|
GracePeriodStatus.ADD, domain.getRepoId(), billingTime, "TheRegistrar", null),
|
||||||
createBillingEvent));
|
createBillingEvent));
|
||||||
assertDnsTasksEnqueued(getUniqueIdFromCommand());
|
dnsUtilsHelper.assertDomainDnsRequests(getUniqueIdFromCommand());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertNoLordn() throws Exception {
|
private void assertNoLordn() throws Exception {
|
||||||
|
@ -415,7 +413,9 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
.and()
|
.and()
|
||||||
.hasLaunchNotice(null);
|
.hasLaunchNotice(null);
|
||||||
if (DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc())
|
if (DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc())
|
||||||
.equals(MigrationState.NORDN_SQL)) {
|
.equals(MigrationState.NORDN_SQL)
|
||||||
|
|| DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc())
|
||||||
|
.equals(MigrationState.DNS_SQL)) {
|
||||||
assertAboutDomains().that(reloadResourceByForeignKey()).hasLordnPhase(LordnPhase.SUNRISE);
|
assertAboutDomains().that(reloadResourceByForeignKey()).hasLordnPhase(LordnPhase.SUNRISE);
|
||||||
} else {
|
} else {
|
||||||
String expectedPayload =
|
String expectedPayload =
|
||||||
|
@ -441,8 +441,10 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
"tmch",
|
"tmch",
|
||||||
DateTime.parse("2010-08-16T09:00:00.0Z"),
|
DateTime.parse("2010-08-16T09:00:00.0Z"),
|
||||||
DateTime.parse("2009-08-16T09:00:00.0Z")));
|
DateTime.parse("2009-08-16T09:00:00.0Z")));
|
||||||
if (DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc())
|
if ((DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc())
|
||||||
.equals(MigrationState.NORDN_SQL)) {
|
.equals(MigrationState.NORDN_SQL)
|
||||||
|
|| DatabaseMigrationStateSchedule.getValueAtTime(clock.nowUtc())
|
||||||
|
.equals(MigrationState.DNS_SQL))) {
|
||||||
assertAboutDomains().that(reloadResourceByForeignKey()).hasLordnPhase(LordnPhase.CLAIMS);
|
assertAboutDomains().that(reloadResourceByForeignKey()).hasLordnPhase(LordnPhase.CLAIMS);
|
||||||
} else {
|
} else {
|
||||||
TaskMatcher task =
|
TaskMatcher task =
|
||||||
|
@ -944,7 +946,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
persistContactsAndHosts("net");
|
persistContactsAndHosts("net");
|
||||||
runFlowAssertResponse(loadFile("domain_create_response_idn_minna.xml"));
|
runFlowAssertResponse(loadFile("domain_create_response_idn_minna.xml"));
|
||||||
assertSuccessfulCreate("xn--q9jyb4c", ImmutableSet.of());
|
assertSuccessfulCreate("xn--q9jyb4c", ImmutableSet.of());
|
||||||
assertDnsTasksEnqueued("xn--abc-873b2e7eb1k8a4lpjvv.xn--q9jyb4c");
|
dnsUtilsHelper.assertDomainDnsRequests("xn--abc-873b2e7eb1k8a4lpjvv.xn--q9jyb4c");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -953,7 +955,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
persistContactsAndHosts();
|
persistContactsAndHosts();
|
||||||
runFlowAssertResponse(
|
runFlowAssertResponse(
|
||||||
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "example.tld")));
|
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "example.tld")));
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -966,7 +968,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
assertAboutDomains()
|
assertAboutDomains()
|
||||||
.that(reloadResourceByForeignKey())
|
.that(reloadResourceByForeignKey())
|
||||||
.hasRegistrationExpirationTime(clock.nowUtc().plusYears(1));
|
.hasRegistrationExpirationTime(clock.nowUtc().plusYears(1));
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -988,7 +990,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
persistContactsAndHosts();
|
persistContactsAndHosts();
|
||||||
runFlowAssertResponse(loadFile("domain_create_response_claims.xml"));
|
runFlowAssertResponse(loadFile("domain_create_response_claims.xml"));
|
||||||
assertSuccessfulCreate("tld", ImmutableSet.of());
|
assertSuccessfulCreate("tld", ImmutableSet.of());
|
||||||
assertDnsTasksEnqueued("example-one.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example-one.tld");
|
||||||
assertClaimsLordn();
|
assertClaimsLordn();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1021,7 +1023,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
persistContactsAndHosts();
|
persistContactsAndHosts();
|
||||||
runFlowAssertResponse(loadFile("domain_create_response_claims.xml"));
|
runFlowAssertResponse(loadFile("domain_create_response_claims.xml"));
|
||||||
assertSuccessfulCreate("tld", ImmutableSet.of(RESERVED), allocationToken);
|
assertSuccessfulCreate("tld", ImmutableSet.of(RESERVED), allocationToken);
|
||||||
assertDnsTasksEnqueued("example-one.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example-one.tld");
|
||||||
assertClaimsLordn();
|
assertClaimsLordn();
|
||||||
assertAllocationTokenWasRedeemed("abcDEF23456");
|
assertAllocationTokenWasRedeemed("abcDEF23456");
|
||||||
}
|
}
|
||||||
|
@ -1034,7 +1036,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
runFlowAssertResponse(
|
runFlowAssertResponse(
|
||||||
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "example.tld")));
|
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "example.tld")));
|
||||||
assertSuccessfulCreate("tld", ImmutableSet.of());
|
assertSuccessfulCreate("tld", ImmutableSet.of());
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1442,7 +1444,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
persistContactsAndHosts();
|
persistContactsAndHosts();
|
||||||
runFlowAssertResponse(loadFile("domain_create_response_claims.xml"));
|
runFlowAssertResponse(loadFile("domain_create_response_claims.xml"));
|
||||||
assertSuccessfulCreate("tld", ImmutableSet.of(ANCHOR_TENANT), allocationToken);
|
assertSuccessfulCreate("tld", ImmutableSet.of(ANCHOR_TENANT), allocationToken);
|
||||||
assertDnsTasksEnqueued("example-one.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example-one.tld");
|
||||||
assertClaimsLordn();
|
assertClaimsLordn();
|
||||||
assertAllocationTokenWasRedeemed("abcDEF23456");
|
assertAllocationTokenWasRedeemed("abcDEF23456");
|
||||||
}
|
}
|
||||||
|
@ -1526,7 +1528,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
"EXPIRATION_TIME",
|
"EXPIRATION_TIME",
|
||||||
SMD_VALID_TIME.plusYears(2).toString())));
|
SMD_VALID_TIME.plusYears(2).toString())));
|
||||||
assertSuccessfulCreate("tld", ImmutableSet.of(ANCHOR_TENANT, SUNRISE), allocationToken);
|
assertSuccessfulCreate("tld", ImmutableSet.of(ANCHOR_TENANT, SUNRISE), allocationToken);
|
||||||
assertDnsTasksEnqueued("test-validate.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("test-validate.tld");
|
||||||
assertSunriseLordn("test-validate.tld");
|
assertSunriseLordn("test-validate.tld");
|
||||||
assertAllocationTokenWasRedeemed("abcDEF23456");
|
assertAllocationTokenWasRedeemed("abcDEF23456");
|
||||||
}
|
}
|
||||||
|
@ -2128,7 +2130,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
assertSunriseLordn("test-and-validate.tld");
|
assertSunriseLordn("test-and-validate.tld");
|
||||||
|
|
||||||
// Check for SERVER_HOLD status, no DNS tasks enqueued, and collision poll message.
|
// Check for SERVER_HOLD status, no DNS tasks enqueued, and collision poll message.
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
Domain domain = reloadResourceByForeignKey();
|
Domain domain = reloadResourceByForeignKey();
|
||||||
assertThat(domain.getStatusValues()).contains(SERVER_HOLD);
|
assertThat(domain.getStatusValues()).contains(SERVER_HOLD);
|
||||||
assertPollMessagesWithCollisionOneTime(domain);
|
assertPollMessagesWithCollisionOneTime(domain);
|
||||||
|
@ -2145,7 +2147,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
|
||||||
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "badcrash.tld")));
|
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "badcrash.tld")));
|
||||||
|
|
||||||
// Check for SERVER_HOLD status, no DNS tasks enqueued, and collision poll message.
|
// Check for SERVER_HOLD status, no DNS tasks enqueued, and collision poll message.
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
Domain domain = reloadResourceByForeignKey();
|
Domain domain = reloadResourceByForeignKey();
|
||||||
assertThat(domain.getStatusValues()).contains(SERVER_HOLD);
|
assertThat(domain.getStatusValues()).contains(SERVER_HOLD);
|
||||||
assertPollMessagesWithCollisionOneTime(domain);
|
assertPollMessagesWithCollisionOneTime(domain);
|
||||||
|
|
|
@ -52,7 +52,6 @@ import static google.registry.testing.DatabaseHelper.persistResource;
|
||||||
import static google.registry.testing.DomainSubject.assertAboutDomains;
|
import static google.registry.testing.DomainSubject.assertAboutDomains;
|
||||||
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
||||||
import static google.registry.testing.HistoryEntrySubject.assertAboutHistoryEntries;
|
import static google.registry.testing.HistoryEntrySubject.assertAboutHistoryEntries;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
|
||||||
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||||
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||||
import static org.joda.money.CurrencyUnit.USD;
|
import static org.joda.money.CurrencyUnit.USD;
|
||||||
|
@ -101,20 +100,16 @@ import google.registry.model.transfer.TransferResponse;
|
||||||
import google.registry.model.transfer.TransferStatus;
|
import google.registry.model.transfer.TransferStatus;
|
||||||
import google.registry.testing.CloudTasksHelper.TaskMatcher;
|
import google.registry.testing.CloudTasksHelper.TaskMatcher;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
import google.registry.testing.TaskQueueExtension;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import org.joda.money.Money;
|
import org.joda.money.Money;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
|
||||||
|
|
||||||
/** Unit tests for {@link DomainDeleteFlow}. */
|
/** Unit tests for {@link DomainDeleteFlow}. */
|
||||||
class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain> {
|
class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain> {
|
||||||
|
|
||||||
@RegisterExtension final TaskQueueExtension taskQueue = new TaskQueueExtension();
|
|
||||||
|
|
||||||
private Domain domain;
|
private Domain domain;
|
||||||
private DomainHistory earlierHistoryEntry;
|
private DomainHistory earlierHistoryEntry;
|
||||||
|
|
||||||
|
@ -362,7 +357,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
|
||||||
// The add grace period is for a billable action, so it should trigger a cancellation.
|
// The add grace period is for a billable action, so it should trigger a cancellation.
|
||||||
assertAutorenewClosedAndCancellationCreatedFor(
|
assertAutorenewClosedAndCancellationCreatedFor(
|
||||||
graceBillingEvent, getOnlyHistoryEntryOfType(domain, DOMAIN_DELETE, DomainHistory.class));
|
graceBillingEvent, getOnlyHistoryEntryOfType(domain, DOMAIN_DELETE, DomainHistory.class));
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
// There should be no poll messages. The previous autorenew poll message should now be deleted.
|
// There should be no poll messages. The previous autorenew poll message should now be deleted.
|
||||||
assertThat(getPollMessages("TheRegistrar")).isEmpty();
|
assertThat(getPollMessages("TheRegistrar")).isEmpty();
|
||||||
}
|
}
|
||||||
|
@ -752,7 +747,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
|
||||||
.build());
|
.build());
|
||||||
DateTime eventTime = clock.nowUtc();
|
DateTime eventTime = clock.nowUtc();
|
||||||
runFlowAssertResponse(loadFile("generic_success_response.xml"));
|
runFlowAssertResponse(loadFile("generic_success_response.xml"));
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
assertAutorenewClosedAndCancellationCreatedFor(
|
assertAutorenewClosedAndCancellationCreatedFor(
|
||||||
graceBillingEvent,
|
graceBillingEvent,
|
||||||
getOnlyHistoryEntryOfType(domain, DOMAIN_DELETE, DomainHistory.class),
|
getOnlyHistoryEntryOfType(domain, DOMAIN_DELETE, DomainHistory.class),
|
||||||
|
@ -770,7 +765,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
|
||||||
.build());
|
.build());
|
||||||
clock.advanceOneMilli();
|
clock.advanceOneMilli();
|
||||||
runFlowAssertResponse(loadFile("domain_delete_response_pending.xml"));
|
runFlowAssertResponse(loadFile("domain_delete_response_pending.xml"));
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
assertOnlyBillingEventIsClosedAutorenew("TheRegistrar");
|
assertOnlyBillingEventIsClosedAutorenew("TheRegistrar");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -29,7 +29,6 @@ import static google.registry.testing.DatabaseHelper.persistReservedList;
|
||||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||||
import static google.registry.testing.DomainSubject.assertAboutDomains;
|
import static google.registry.testing.DomainSubject.assertAboutDomains;
|
||||||
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
|
||||||
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||||
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||||
import static org.joda.money.CurrencyUnit.EUR;
|
import static org.joda.money.CurrencyUnit.EUR;
|
||||||
|
@ -75,20 +74,16 @@ import google.registry.model.reporting.DomainTransactionRecord.TransactionReport
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.model.tld.Registry;
|
import google.registry.model.tld.Registry;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
import google.registry.testing.TaskQueueExtension;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import org.joda.money.Money;
|
import org.joda.money.Money;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
|
||||||
|
|
||||||
/** Unit tests for {@link DomainRestoreRequestFlow}. */
|
/** Unit tests for {@link DomainRestoreRequestFlow}. */
|
||||||
class DomainRestoreRequestFlowTest extends ResourceFlowTestCase<DomainRestoreRequestFlow, Domain> {
|
class DomainRestoreRequestFlowTest extends ResourceFlowTestCase<DomainRestoreRequestFlow, Domain> {
|
||||||
|
|
||||||
@RegisterExtension final TaskQueueExtension taskQueue = new TaskQueueExtension();
|
|
||||||
|
|
||||||
private static final ImmutableMap<String, String> FEE_06_MAP =
|
private static final ImmutableMap<String, String> FEE_06_MAP =
|
||||||
ImmutableMap.of("FEE_VERSION", "0.6", "FEE_NS", "fee", "CURRENCY", "USD");
|
ImmutableMap.of("FEE_VERSION", "0.6", "FEE_NS", "fee", "CURRENCY", "USD");
|
||||||
private static final ImmutableMap<String, String> FEE_11_MAP =
|
private static final ImmutableMap<String, String> FEE_11_MAP =
|
||||||
|
@ -192,7 +187,7 @@ class DomainRestoreRequestFlowTest extends ResourceFlowTestCase<DomainRestoreReq
|
||||||
.and()
|
.and()
|
||||||
.hasLastEppUpdateRegistrarId("TheRegistrar");
|
.hasLastEppUpdateRegistrarId("TheRegistrar");
|
||||||
assertThat(domain.getGracePeriods()).isEmpty();
|
assertThat(domain.getGracePeriods()).isEmpty();
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
// The poll message for the delete should now be gone. The only poll message should be the new
|
// The poll message for the delete should now be gone. The only poll message should be the new
|
||||||
// autorenew poll message.
|
// autorenew poll message.
|
||||||
assertPollMessages(
|
assertPollMessages(
|
||||||
|
@ -261,7 +256,7 @@ class DomainRestoreRequestFlowTest extends ResourceFlowTestCase<DomainRestoreReq
|
||||||
.and()
|
.and()
|
||||||
.hasLastEppUpdateRegistrarId("TheRegistrar");
|
.hasLastEppUpdateRegistrarId("TheRegistrar");
|
||||||
assertThat(domain.getGracePeriods()).isEmpty();
|
assertThat(domain.getGracePeriods()).isEmpty();
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
// The poll message for the delete should now be gone. The only poll message should be the new
|
// The poll message for the delete should now be gone. The only poll message should be the new
|
||||||
// autorenew poll message.
|
// autorenew poll message.
|
||||||
assertPollMessages(
|
assertPollMessages(
|
||||||
|
|
|
@ -46,8 +46,6 @@ import static google.registry.testing.DatabaseHelper.persistResource;
|
||||||
import static google.registry.testing.DomainSubject.assertAboutDomains;
|
import static google.registry.testing.DomainSubject.assertAboutDomains;
|
||||||
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
||||||
import static google.registry.testing.HistoryEntrySubject.assertAboutHistoryEntries;
|
import static google.registry.testing.HistoryEntrySubject.assertAboutHistoryEntries;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
|
||||||
import static google.registry.testing.TaskQueueHelper.assertNoDnsTasksEnqueued;
|
|
||||||
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||||
import static org.joda.money.CurrencyUnit.USD;
|
import static org.joda.money.CurrencyUnit.USD;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
@ -106,19 +104,15 @@ import google.registry.model.poll.PollMessage;
|
||||||
import google.registry.model.tld.Registry;
|
import google.registry.model.tld.Registry;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
import google.registry.testing.TaskQueueExtension;
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import org.joda.money.Money;
|
import org.joda.money.Money;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
|
||||||
|
|
||||||
/** Unit tests for {@link DomainUpdateFlow}. */
|
/** Unit tests for {@link DomainUpdateFlow}. */
|
||||||
class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain> {
|
class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain> {
|
||||||
|
|
||||||
@RegisterExtension final TaskQueueExtension taskQueue = new TaskQueueExtension();
|
|
||||||
|
|
||||||
private static final DomainDsData SOME_DSDATA =
|
private static final DomainDsData SOME_DSDATA =
|
||||||
DomainDsData.create(
|
DomainDsData.create(
|
||||||
1,
|
1,
|
||||||
|
@ -222,7 +216,7 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
||||||
.and()
|
.and()
|
||||||
.hasNoAutorenewEndTime();
|
.hasNoAutorenewEndTime();
|
||||||
assertNoBillingEvents();
|
assertNoBillingEvents();
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
assertLastHistoryContainsResource(reloadResourceByForeignKey());
|
assertLastHistoryContainsResource(reloadResourceByForeignKey());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -351,7 +345,7 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
||||||
assertThat(domain.getContacts()).hasSize(3);
|
assertThat(domain.getContacts()).hasSize(3);
|
||||||
assertThat(loadByKey(domain.getRegistrant()).getContactId()).isEqualTo("max_test_7");
|
assertThat(loadByKey(domain.getRegistrant()).getContactId()).isEqualTo("max_test_7");
|
||||||
assertNoBillingEvents();
|
assertNoBillingEvents();
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -502,9 +496,9 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
||||||
.map(ds -> ds.cloneWithDomainRepoId(resource.getRepoId()))
|
.map(ds -> ds.cloneWithDomainRepoId(resource.getRepoId()))
|
||||||
.collect(toImmutableSet()));
|
.collect(toImmutableSet()));
|
||||||
if (dnsTaskEnqueued) {
|
if (dnsTaskEnqueued) {
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
} else {
|
} else {
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1765,7 +1759,7 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
||||||
.setStatusValues(ImmutableSet.of(StatusValue.CLIENT_TRANSFER_PROHIBITED))
|
.setStatusValues(ImmutableSet.of(StatusValue.CLIENT_TRANSFER_PROHIBITED))
|
||||||
.build());
|
.build());
|
||||||
runFlowAsSuperuser();
|
runFlowAsSuperuser();
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1781,7 +1775,7 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
||||||
.setStatusValues(ImmutableSet.of(StatusValue.SERVER_HOLD))
|
.setStatusValues(ImmutableSet.of(StatusValue.SERVER_HOLD))
|
||||||
.build());
|
.build());
|
||||||
runFlowAsSuperuser();
|
runFlowAsSuperuser();
|
||||||
assertDnsTasksEnqueued("example.tld");
|
dnsUtilsHelper.assertDomainDnsRequests("example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1797,6 +1791,6 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase<DomainUpdateFlow, Domain
|
||||||
.setStatusValues(ImmutableSet.of(StatusValue.PENDING_DELETE, StatusValue.SERVER_HOLD))
|
.setStatusValues(ImmutableSet.of(StatusValue.PENDING_DELETE, StatusValue.SERVER_HOLD))
|
||||||
.build());
|
.build());
|
||||||
runFlowAsSuperuser();
|
runFlowAsSuperuser();
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,8 +26,6 @@ import static google.registry.testing.DatabaseHelper.persistDeletedHost;
|
||||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||||
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
||||||
import static google.registry.testing.HostSubject.assertAboutHosts;
|
import static google.registry.testing.HostSubject.assertAboutHosts;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
|
||||||
import static google.registry.testing.TaskQueueHelper.assertNoDnsTasksEnqueued;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
|
@ -54,16 +52,12 @@ import google.registry.model.eppcommon.StatusValue;
|
||||||
import google.registry.model.host.Host;
|
import google.registry.model.host.Host;
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
import google.registry.testing.TaskQueueExtension;
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
|
||||||
|
|
||||||
/** Unit tests for {@link HostCreateFlow}. */
|
/** Unit tests for {@link HostCreateFlow}. */
|
||||||
class HostCreateFlowTest extends ResourceFlowTestCase<HostCreateFlow, Host> {
|
class HostCreateFlowTest extends ResourceFlowTestCase<HostCreateFlow, Host> {
|
||||||
|
|
||||||
@RegisterExtension TaskQueueExtension taskQueue = new TaskQueueExtension();
|
|
||||||
|
|
||||||
private void setEppHostCreateInput(String hostName, String hostAddrs) {
|
private void setEppHostCreateInput(String hostName, String hostAddrs) {
|
||||||
setEppInput(
|
setEppInput(
|
||||||
"host_create.xml",
|
"host_create.xml",
|
||||||
|
@ -122,7 +116,7 @@ class HostCreateFlowTest extends ResourceFlowTestCase<HostCreateFlow, Host> {
|
||||||
void testSuccess_externalNeverExisted() throws Exception {
|
void testSuccess_externalNeverExisted() throws Exception {
|
||||||
doSuccessfulTest();
|
doSuccessfulTest();
|
||||||
assertAboutHosts().that(reloadResourceByForeignKey()).hasSuperordinateDomain(null);
|
assertAboutHosts().that(reloadResourceByForeignKey()).hasSuperordinateDomain(null);
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -133,7 +127,7 @@ class HostCreateFlowTest extends ResourceFlowTestCase<HostCreateFlow, Host> {
|
||||||
loadByForeignKey(Domain.class, "example.tld", clock.nowUtc()).get();
|
loadByForeignKey(Domain.class, "example.tld", clock.nowUtc()).get();
|
||||||
assertAboutHosts().that(host).hasSuperordinateDomain(superordinateDomain.createVKey());
|
assertAboutHosts().that(host).hasSuperordinateDomain(superordinateDomain.createVKey());
|
||||||
assertThat(superordinateDomain.getSubordinateHosts()).containsExactly("ns1.example.tld");
|
assertThat(superordinateDomain.getSubordinateHosts()).containsExactly("ns1.example.tld");
|
||||||
assertDnsTasksEnqueued("ns1.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -150,7 +144,7 @@ class HostCreateFlowTest extends ResourceFlowTestCase<HostCreateFlow, Host> {
|
||||||
persistDeletedHost(getUniqueIdFromCommand(), clock.nowUtc().minusDays(1));
|
persistDeletedHost(getUniqueIdFromCommand(), clock.nowUtc().minusDays(1));
|
||||||
doSuccessfulTest();
|
doSuccessfulTest();
|
||||||
assertAboutHosts().that(reloadResourceByForeignKey()).hasSuperordinateDomain(null);
|
assertAboutHosts().that(reloadResourceByForeignKey()).hasSuperordinateDomain(null);
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -162,7 +156,7 @@ class HostCreateFlowTest extends ResourceFlowTestCase<HostCreateFlow, Host> {
|
||||||
loadByForeignKey(Domain.class, "example.tld", clock.nowUtc()).get();
|
loadByForeignKey(Domain.class, "example.tld", clock.nowUtc()).get();
|
||||||
assertAboutHosts().that(host).hasSuperordinateDomain(superordinateDomain.createVKey());
|
assertAboutHosts().that(host).hasSuperordinateDomain(superordinateDomain.createVKey());
|
||||||
assertThat(superordinateDomain.getSubordinateHosts()).containsExactly("ns1.example.tld");
|
assertThat(superordinateDomain.getSubordinateHosts()).containsExactly("ns1.example.tld");
|
||||||
assertDnsTasksEnqueued("ns1.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -24,8 +24,6 @@ import static google.registry.testing.DatabaseHelper.persistDeletedHost;
|
||||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||||
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions;
|
||||||
import static google.registry.testing.HostSubject.assertAboutHosts;
|
import static google.registry.testing.HostSubject.assertAboutHosts;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
|
||||||
import static google.registry.testing.TaskQueueHelper.assertNoDnsTasksEnqueued;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
@ -48,17 +46,13 @@ import google.registry.model.tld.Registry;
|
||||||
import google.registry.model.transfer.DomainTransferData;
|
import google.registry.model.transfer.DomainTransferData;
|
||||||
import google.registry.model.transfer.TransferStatus;
|
import google.registry.model.transfer.TransferStatus;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
import google.registry.testing.TaskQueueExtension;
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
|
||||||
|
|
||||||
/** Unit tests for {@link HostDeleteFlow}. */
|
/** Unit tests for {@link HostDeleteFlow}. */
|
||||||
class HostDeleteFlowTest extends ResourceFlowTestCase<HostDeleteFlow, Host> {
|
class HostDeleteFlowTest extends ResourceFlowTestCase<HostDeleteFlow, Host> {
|
||||||
|
|
||||||
@RegisterExtension TaskQueueExtension taskQueue = new TaskQueueExtension();
|
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
void initFlowTest() {
|
void initFlowTest() {
|
||||||
setEppInput("host_delete.xml", ImmutableMap.of("HOSTNAME", "ns1.example.tld"));
|
setEppInput("host_delete.xml", ImmutableMap.of("HOSTNAME", "ns1.example.tld"));
|
||||||
|
@ -320,10 +314,10 @@ class HostDeleteFlowTest extends ResourceFlowTestCase<HostDeleteFlow, Host> {
|
||||||
.hasType(Type.HOST_DELETE);
|
.hasType(Type.HOST_DELETE);
|
||||||
assertNoBillingEvents();
|
assertNoBillingEvents();
|
||||||
if (isSubordinate) {
|
if (isSubordinate) {
|
||||||
assertDnsTasksEnqueued(deletedHost.getHostName());
|
dnsUtilsHelper.assertHostDnsRequests(deletedHost.getHostName());
|
||||||
assertThat(loadByKey(deletedHost.getSuperordinateDomain()).getSubordinateHosts()).isEmpty();
|
assertThat(loadByKey(deletedHost.getSuperordinateDomain()).getSubordinateHosts()).isEmpty();
|
||||||
} else {
|
} else {
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
assertLastHistoryContainsResource(deletedHost);
|
assertLastHistoryContainsResource(deletedHost);
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,8 +35,6 @@ import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptio
|
||||||
import static google.registry.testing.GenericEppResourceSubject.assertAboutEppResources;
|
import static google.registry.testing.GenericEppResourceSubject.assertAboutEppResources;
|
||||||
import static google.registry.testing.HistoryEntrySubject.assertAboutHistoryEntries;
|
import static google.registry.testing.HistoryEntrySubject.assertAboutHistoryEntries;
|
||||||
import static google.registry.testing.HostSubject.assertAboutHosts;
|
import static google.registry.testing.HostSubject.assertAboutHosts;
|
||||||
import static google.registry.testing.TaskQueueHelper.assertDnsTasksEnqueued;
|
|
||||||
import static google.registry.testing.TaskQueueHelper.assertNoDnsTasksEnqueued;
|
|
||||||
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
import static google.registry.util.DateTimeUtils.END_OF_TIME;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
|
||||||
|
@ -81,17 +79,13 @@ import google.registry.model.transfer.TransferStatus;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.testing.CloudTasksHelper.TaskMatcher;
|
import google.registry.testing.CloudTasksHelper.TaskMatcher;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
import google.registry.testing.TaskQueueExtension;
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
|
||||||
|
|
||||||
/** Unit tests for {@link HostUpdateFlow}. */
|
/** Unit tests for {@link HostUpdateFlow}. */
|
||||||
class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
|
|
||||||
@RegisterExtension TaskQueueExtension taskQueue = new TaskQueueExtension();
|
|
||||||
|
|
||||||
private void setEppHostUpdateInput(
|
private void setEppHostUpdateInput(
|
||||||
String oldHostName, String newHostName, String ipOrStatusToAdd, String ipOrStatusToRem) {
|
String oldHostName, String newHostName, String ipOrStatusToAdd, String ipOrStatusToRem) {
|
||||||
setEppInput(
|
setEppInput(
|
||||||
|
@ -189,7 +183,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
persistActiveSubordinateHost(oldHostName(), persistActiveDomain("example.tld"));
|
persistActiveSubordinateHost(oldHostName(), persistActiveDomain("example.tld"));
|
||||||
Host renamedHost = doSuccessfulTest();
|
Host renamedHost = doSuccessfulTest();
|
||||||
assertThat(renamedHost.isSubordinate()).isTrue();
|
assertThat(renamedHost.isSubordinate()).isTrue();
|
||||||
assertDnsTasksEnqueued("ns1.example.tld", "ns2.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld", "ns2.example.tld");
|
||||||
VKey<Host> oldVKeyAfterRename = ForeignKeyUtils.load(Host.class, oldHostName(), clock.nowUtc());
|
VKey<Host> oldVKeyAfterRename = ForeignKeyUtils.load(Host.class, oldHostName(), clock.nowUtc());
|
||||||
assertThat(oldVKeyAfterRename).isNull();
|
assertThat(oldVKeyAfterRename).isNull();
|
||||||
}
|
}
|
||||||
|
@ -238,7 +232,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
.and()
|
.and()
|
||||||
.hasOnlyOneHistoryEntryWhich()
|
.hasOnlyOneHistoryEntryWhich()
|
||||||
.hasType(HistoryEntry.Type.HOST_UPDATE);
|
.hasType(HistoryEntry.Type.HOST_UPDATE);
|
||||||
assertDnsTasksEnqueued("ns1.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -264,7 +258,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
.and()
|
.and()
|
||||||
.hasOnlyOneHistoryEntryWhich()
|
.hasOnlyOneHistoryEntryWhich()
|
||||||
.hasType(HistoryEntry.Type.HOST_UPDATE);
|
.hasType(HistoryEntry.Type.HOST_UPDATE);
|
||||||
assertDnsTasksEnqueued("ns1.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -298,7 +292,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
.hasLastTransferTime(oneDayAgo);
|
.hasLastTransferTime(oneDayAgo);
|
||||||
Domain reloadedDomain = loadByEntity(domain).cloneProjectedAtTime(now);
|
Domain reloadedDomain = loadByEntity(domain).cloneProjectedAtTime(now);
|
||||||
assertThat(reloadedDomain.getSubordinateHosts()).containsExactly("ns2.example.tld");
|
assertThat(reloadedDomain.getSubordinateHosts()).containsExactly("ns2.example.tld");
|
||||||
assertDnsTasksEnqueued("ns1.example.tld", "ns2.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld", "ns2.example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -333,7 +327,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
assertThat(loadByEntity(foo).cloneProjectedAtTime(now).getSubordinateHosts()).isEmpty();
|
assertThat(loadByEntity(foo).cloneProjectedAtTime(now).getSubordinateHosts()).isEmpty();
|
||||||
assertThat(loadByEntity(example).cloneProjectedAtTime(now).getSubordinateHosts())
|
assertThat(loadByEntity(example).cloneProjectedAtTime(now).getSubordinateHosts())
|
||||||
.containsExactly("ns2.example.tld");
|
.containsExactly("ns2.example.tld");
|
||||||
assertDnsTasksEnqueued("ns2.foo.tld", "ns2.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns2.foo.tld", "ns2.example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -370,7 +364,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
assertThat(reloadedFooDomain.getSubordinateHosts()).isEmpty();
|
assertThat(reloadedFooDomain.getSubordinateHosts()).isEmpty();
|
||||||
Domain reloadedTldDomain = loadByEntity(tldDomain).cloneProjectedAtTime(now);
|
Domain reloadedTldDomain = loadByEntity(tldDomain).cloneProjectedAtTime(now);
|
||||||
assertThat(reloadedTldDomain.getSubordinateHosts()).containsExactly("ns2.example.tld");
|
assertThat(reloadedTldDomain.getSubordinateHosts()).containsExactly("ns2.example.tld");
|
||||||
assertDnsTasksEnqueued("ns1.example.foo", "ns2.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.foo", "ns2.example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -413,7 +407,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
assertThat(renamedHost.getLastTransferTime()).isEqualTo(oneDayAgo);
|
assertThat(renamedHost.getLastTransferTime()).isEqualTo(oneDayAgo);
|
||||||
Domain reloadedDomain = loadByEntity(domain).cloneProjectedAtTime(clock.nowUtc());
|
Domain reloadedDomain = loadByEntity(domain).cloneProjectedAtTime(clock.nowUtc());
|
||||||
assertThat(reloadedDomain.getSubordinateHosts()).isEmpty();
|
assertThat(reloadedDomain.getSubordinateHosts()).isEmpty();
|
||||||
assertDnsTasksEnqueued("ns1.example.foo");
|
dnsUtilsHelper.assertHostDnsRequests("ns1.example.foo");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -425,7 +419,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
persistActiveHost(oldHostName());
|
persistActiveHost(oldHostName());
|
||||||
assertThat(domain.getSubordinateHosts()).isEmpty();
|
assertThat(domain.getSubordinateHosts()).isEmpty();
|
||||||
assertThrows(CannotRenameExternalHostException.class, this::runFlow);
|
assertThrows(CannotRenameExternalHostException.class, this::runFlow);
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -449,7 +443,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
.hasLastTransferTime(null);
|
.hasLastTransferTime(null);
|
||||||
assertThat(loadByEntity(domain).cloneProjectedAtTime(now).getSubordinateHosts())
|
assertThat(loadByEntity(domain).cloneProjectedAtTime(now).getSubordinateHosts())
|
||||||
.containsExactly("ns2.example.tld");
|
.containsExactly("ns2.example.tld");
|
||||||
assertDnsTasksEnqueued("ns2.example.tld");
|
dnsUtilsHelper.assertHostDnsRequests("ns2.example.tld");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -474,7 +468,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase<HostUpdateFlow, Host> {
|
||||||
.hasPersistedCurrentSponsorRegistrarId("TheRegistrar")
|
.hasPersistedCurrentSponsorRegistrarId("TheRegistrar")
|
||||||
.and()
|
.and()
|
||||||
.hasLastTransferTime(null);
|
.hasLastTransferTime(null);
|
||||||
assertNoDnsTasksEnqueued();
|
dnsUtilsHelper.assertNoMoreDnsRequests();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -0,0 +1,93 @@
|
||||||
|
// Copyright 2023 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.model.common;
|
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat;
|
||||||
|
import static google.registry.model.ImmutableObjectSubject.assertAboutImmutableObjects;
|
||||||
|
import static google.registry.testing.DatabaseHelper.insertInDb;
|
||||||
|
import static google.registry.testing.DatabaseHelper.loadAllOf;
|
||||||
|
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import google.registry.dns.DnsConstants.TargetType;
|
||||||
|
import google.registry.model.EntityTestCase;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
|
/** Unit tests for {@link DnsRefreshRequest}. */
|
||||||
|
public class DnsRefreshRequestTest extends EntityTestCase {
|
||||||
|
|
||||||
|
DnsRefreshRequestTest() {
|
||||||
|
super(JpaEntityCoverageCheck.ENABLED);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final DnsRefreshRequest request =
|
||||||
|
new DnsRefreshRequest(TargetType.DOMAIN, "test.example", "example", fakeClock.nowUtc());
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testPersistence() {
|
||||||
|
assertThat(request.getLastProcessTime()).isEqualTo(START_OF_TIME);
|
||||||
|
fakeClock.advanceOneMilli();
|
||||||
|
insertInDb(request);
|
||||||
|
fakeClock.advanceOneMilli();
|
||||||
|
ImmutableList<DnsRefreshRequest> requests = loadAllOf(DnsRefreshRequest.class);
|
||||||
|
assertThat(requests.size()).isEqualTo(1);
|
||||||
|
assertThat(requests.get(0)).isEqualTo(request);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testNullValues() {
|
||||||
|
// type
|
||||||
|
assertThrows(
|
||||||
|
NullPointerException.class,
|
||||||
|
() -> new DnsRefreshRequest(null, "test.example", "example", fakeClock.nowUtc()));
|
||||||
|
// name
|
||||||
|
assertThrows(
|
||||||
|
NullPointerException.class,
|
||||||
|
() -> new DnsRefreshRequest(TargetType.DOMAIN, null, "example", fakeClock.nowUtc()));
|
||||||
|
// tld
|
||||||
|
assertThrows(
|
||||||
|
NullPointerException.class,
|
||||||
|
() -> new DnsRefreshRequest(TargetType.DOMAIN, "test.example", null, fakeClock.nowUtc()));
|
||||||
|
// request time
|
||||||
|
assertThrows(
|
||||||
|
NullPointerException.class,
|
||||||
|
() -> new DnsRefreshRequest(TargetType.DOMAIN, "test.example", "example", null));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testUpdateProcessTime() {
|
||||||
|
assertThat(
|
||||||
|
assertThrows(
|
||||||
|
IllegalArgumentException.class,
|
||||||
|
() -> request.updateProcessTime(fakeClock.nowUtc())))
|
||||||
|
.hasMessageThat()
|
||||||
|
.contains("must be later than request time");
|
||||||
|
|
||||||
|
fakeClock.advanceOneMilli();
|
||||||
|
fakeClock.advanceOneMilli();
|
||||||
|
|
||||||
|
DnsRefreshRequest newRequest = request.updateProcessTime(fakeClock.nowUtc());
|
||||||
|
assertAboutImmutableObjects().that(newRequest).isEqualExceptFields(request, "lastProcessTime");
|
||||||
|
assertThat(newRequest.getLastProcessTime()).isEqualTo(fakeClock.nowUtc());
|
||||||
|
|
||||||
|
assertThat(
|
||||||
|
assertThrows(
|
||||||
|
IllegalArgumentException.class,
|
||||||
|
() -> newRequest.updateProcessTime(fakeClock.nowUtc().minusMillis(1))))
|
||||||
|
.hasMessageThat()
|
||||||
|
.contains("must be later than the old one");
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,6 +18,7 @@ import static com.google.common.truth.Truth.assert_;
|
||||||
|
|
||||||
import google.registry.model.billing.BillingEventTest;
|
import google.registry.model.billing.BillingEventTest;
|
||||||
import google.registry.model.common.CursorTest;
|
import google.registry.model.common.CursorTest;
|
||||||
|
import google.registry.model.common.DnsRefreshRequestTest;
|
||||||
import google.registry.model.console.UserTest;
|
import google.registry.model.console.UserTest;
|
||||||
import google.registry.model.contact.ContactTest;
|
import google.registry.model.contact.ContactTest;
|
||||||
import google.registry.model.domain.DomainSqlTest;
|
import google.registry.model.domain.DomainSqlTest;
|
||||||
|
@ -85,6 +86,7 @@ import org.junit.runner.RunWith;
|
||||||
ContactHistoryTest.class,
|
ContactHistoryTest.class,
|
||||||
ContactTest.class,
|
ContactTest.class,
|
||||||
CursorTest.class,
|
CursorTest.class,
|
||||||
|
DnsRefreshRequestTest.class,
|
||||||
DomainSqlTest.class,
|
DomainSqlTest.class,
|
||||||
DomainHistoryTest.class,
|
DomainHistoryTest.class,
|
||||||
HostHistoryTest.class,
|
HostHistoryTest.class,
|
||||||
|
|
|
@ -0,0 +1,73 @@
|
||||||
|
// Copyright 2023 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.testing;
|
||||||
|
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.never;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||||
|
|
||||||
|
import google.registry.dns.DnsUtils;
|
||||||
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
|
import org.joda.time.Duration;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test helper for {@link DnsUtils}.
|
||||||
|
*
|
||||||
|
* <p>This is a temporary test class that is only used during DNS pull queue migration. Once we are
|
||||||
|
* no longer using the pull queue method, we can just assert on the inserted SQL entry instead in
|
||||||
|
* {@link DatabaseHelper}.
|
||||||
|
*/
|
||||||
|
@DeleteAfterMigration
|
||||||
|
public class DnsUtilsHelper {
|
||||||
|
|
||||||
|
private final DnsUtils dnsUtils;
|
||||||
|
|
||||||
|
public DnsUtilsHelper() {
|
||||||
|
dnsUtils = mock(DnsUtils.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DnsUtilsHelper(DnsUtils dnsUtils) {
|
||||||
|
this.dnsUtils = dnsUtils;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DnsUtils getDnsUtils() {
|
||||||
|
return dnsUtils;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void assertDomainDnsRequests(String... domainNames) {
|
||||||
|
for (String domainName : domainNames) {
|
||||||
|
verify(dnsUtils).requestDomainDnsRefresh(domainName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void assertDomainDnsRequestWithDelay(String domainName, Duration delay) {
|
||||||
|
verify(dnsUtils).requestDomainDnsRefresh(domainName, delay);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void assertNoDomainDnsRequestWithDelay(String domainName, Duration delay) {
|
||||||
|
verify(dnsUtils, never()).requestDomainDnsRefresh(domainName, delay);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void assertHostDnsRequests(String... hostNames) {
|
||||||
|
for (String hostName : hostNames) {
|
||||||
|
verify(dnsUtils).requestHostDnsRefresh(hostName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void assertNoMoreDnsRequests() {
|
||||||
|
verifyNoMoreInteractions(dnsUtils);
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,21 +18,18 @@ import static com.google.common.truth.Truth.assertThat;
|
||||||
import static google.registry.testing.DatabaseHelper.createTld;
|
import static google.registry.testing.DatabaseHelper.createTld;
|
||||||
import static google.registry.testing.DatabaseHelper.persistActiveDomain;
|
import static google.registry.testing.DatabaseHelper.persistActiveDomain;
|
||||||
import static google.registry.testing.DatabaseHelper.persistDeletedDomain;
|
import static google.registry.testing.DatabaseHelper.persistDeletedDomain;
|
||||||
import static org.joda.time.Duration.standardMinutes;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
import static org.mockito.ArgumentMatchers.eq;
|
import static org.mockito.ArgumentMatchers.eq;
|
||||||
import static org.mockito.Mockito.doThrow;
|
import static org.mockito.Mockito.doThrow;
|
||||||
import static org.mockito.Mockito.inOrder;
|
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.never;
|
|
||||||
import static org.mockito.Mockito.verify;
|
|
||||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import google.registry.dns.DnsQueue;
|
import google.registry.dns.DnsUtils;
|
||||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
||||||
|
import google.registry.testing.DnsUtilsHelper;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
import google.registry.testing.FakeResponse;
|
import google.registry.testing.FakeResponse;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
@ -42,14 +39,13 @@ import org.joda.time.Duration;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||||
import org.mockito.ArgumentCaptor;
|
|
||||||
import org.mockito.InOrder;
|
|
||||||
|
|
||||||
/** Unit tests for {@link RefreshDnsForAllDomainsAction}. */
|
/** Unit tests for {@link RefreshDnsForAllDomainsAction}. */
|
||||||
public class RefreshDnsForAllDomainsActionTest {
|
public class RefreshDnsForAllDomainsActionTest {
|
||||||
|
|
||||||
private final FakeClock clock = new FakeClock(DateTime.parse("2020-02-02T02:02:02Z"));
|
private final FakeClock clock = new FakeClock(DateTime.parse("2020-02-02T02:02:02Z"));
|
||||||
private final DnsQueue dnsQueue = mock(DnsQueue.class);
|
private final DnsUtils dnsUtils = mock(DnsUtils.class);
|
||||||
|
private final DnsUtilsHelper dnsUtilsHelper = new DnsUtilsHelper(dnsUtils);
|
||||||
private RefreshDnsForAllDomainsAction action;
|
private RefreshDnsForAllDomainsAction action;
|
||||||
private final FakeResponse response = new FakeResponse();
|
private final FakeResponse response = new FakeResponse();
|
||||||
|
|
||||||
|
@ -64,27 +60,26 @@ public class RefreshDnsForAllDomainsActionTest {
|
||||||
action.random = new Random();
|
action.random = new Random();
|
||||||
action.random.setSeed(123L);
|
action.random.setSeed(123L);
|
||||||
action.clock = clock;
|
action.clock = clock;
|
||||||
action.dnsQueue = dnsQueue;
|
action.dnsUtils = dnsUtils;
|
||||||
action.response = response;
|
action.response = response;
|
||||||
|
|
||||||
createTld("bar");
|
createTld("bar");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void test_runAction_errorEnqueuingToDnsQueue() throws Exception {
|
void test_runAction_errorRequestDnsRefresh() throws Exception {
|
||||||
persistActiveDomain("foo.bar");
|
persistActiveDomain("foo.bar");
|
||||||
persistActiveDomain("baz.bar");
|
persistActiveDomain("baz.bar");
|
||||||
persistActiveDomain("low.bar");
|
persistActiveDomain("low.bar");
|
||||||
action.tlds = ImmutableSet.of("bar");
|
action.tlds = ImmutableSet.of("bar");
|
||||||
doThrow(new RuntimeException("Error enqueuing task."))
|
doThrow(new RuntimeException("Error enqueuing task."))
|
||||||
.when(dnsQueue)
|
.when(dnsUtils)
|
||||||
.addDomainRefreshTask(eq("baz.bar"), any(Duration.class));
|
.requestDomainDnsRefresh(eq("baz.bar"), any(Duration.class));
|
||||||
action.run();
|
action.run();
|
||||||
InOrder inOrder = inOrder(dnsQueue);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("low.bar", Duration.ZERO);
|
||||||
inOrder.verify(dnsQueue).addDomainRefreshTask("low.bar", Duration.ZERO);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("baz.bar", Duration.ZERO);
|
||||||
inOrder.verify(dnsQueue).addDomainRefreshTask("baz.bar", Duration.ZERO);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.ZERO);
|
||||||
inOrder.verify(dnsQueue).addDomainRefreshTask("foo.bar", Duration.ZERO);
|
verifyNoMoreInteractions(dnsUtils);
|
||||||
verifyNoMoreInteractions(dnsQueue);
|
|
||||||
assertThat(response.getStatus()).isEqualTo(HttpStatus.SC_INTERNAL_SERVER_ERROR);
|
assertThat(response.getStatus()).isEqualTo(HttpStatus.SC_INTERNAL_SERVER_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -94,8 +89,8 @@ public class RefreshDnsForAllDomainsActionTest {
|
||||||
persistActiveDomain("low.bar");
|
persistActiveDomain("low.bar");
|
||||||
action.tlds = ImmutableSet.of("bar");
|
action.tlds = ImmutableSet.of("bar");
|
||||||
action.run();
|
action.run();
|
||||||
verify(dnsQueue).addDomainRefreshTask("foo.bar", Duration.ZERO);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.ZERO);
|
||||||
verify(dnsQueue).addDomainRefreshTask("low.bar", Duration.ZERO);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("low.bar", Duration.ZERO);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -105,10 +100,8 @@ public class RefreshDnsForAllDomainsActionTest {
|
||||||
action.tlds = ImmutableSet.of("bar");
|
action.tlds = ImmutableSet.of("bar");
|
||||||
action.smearMinutes = 1000;
|
action.smearMinutes = 1000;
|
||||||
action.run();
|
action.run();
|
||||||
ArgumentCaptor<Duration> captor = ArgumentCaptor.forClass(Duration.class);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.standardMinutes(450));
|
||||||
verify(dnsQueue).addDomainRefreshTask(eq("foo.bar"), captor.capture());
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("low.bar", Duration.standardMinutes(782));
|
||||||
verify(dnsQueue).addDomainRefreshTask(eq("low.bar"), captor.capture());
|
|
||||||
assertThat(captor.getAllValues()).containsExactly(standardMinutes(450), standardMinutes(782));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -117,8 +110,8 @@ public class RefreshDnsForAllDomainsActionTest {
|
||||||
persistDeletedDomain("deleted.bar", clock.nowUtc().minusYears(1));
|
persistDeletedDomain("deleted.bar", clock.nowUtc().minusYears(1));
|
||||||
action.tlds = ImmutableSet.of("bar");
|
action.tlds = ImmutableSet.of("bar");
|
||||||
action.run();
|
action.run();
|
||||||
verify(dnsQueue).addDomainRefreshTask("foo.bar", Duration.ZERO);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.ZERO);
|
||||||
verify(dnsQueue, never()).addDomainRefreshTask("deleted.bar", Duration.ZERO);
|
dnsUtilsHelper.assertNoDomainDnsRequestWithDelay("deleted.bar", Duration.ZERO);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -129,9 +122,9 @@ public class RefreshDnsForAllDomainsActionTest {
|
||||||
persistActiveDomain("ignore.baz");
|
persistActiveDomain("ignore.baz");
|
||||||
action.tlds = ImmutableSet.of("bar");
|
action.tlds = ImmutableSet.of("bar");
|
||||||
action.run();
|
action.run();
|
||||||
verify(dnsQueue).addDomainRefreshTask("foo.bar", Duration.ZERO);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.ZERO);
|
||||||
verify(dnsQueue).addDomainRefreshTask("low.bar", Duration.ZERO);
|
dnsUtilsHelper.assertDomainDnsRequestWithDelay("low.bar", Duration.ZERO);
|
||||||
verify(dnsQueue, never()).addDomainRefreshTask("ignore.baz", Duration.ZERO);
|
dnsUtilsHelper.assertNoDomainDnsRequestWithDelay("ignore.baz", Duration.ZERO);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -255,6 +255,16 @@
|
||||||
primary key (algorithm, digest, digest_type, domain_repo_id, key_tag)
|
primary key (algorithm, digest, digest_type, domain_repo_id, key_tag)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
create table "DnsRefreshRequest" (
|
||||||
|
id bigserial not null,
|
||||||
|
last_process_time timestamptz not null,
|
||||||
|
name text not null,
|
||||||
|
request_time timestamptz not null,
|
||||||
|
tld text not null,
|
||||||
|
type text not null,
|
||||||
|
primary key (id)
|
||||||
|
);
|
||||||
|
|
||||||
create table "Domain" (
|
create table "Domain" (
|
||||||
repo_id text not null,
|
repo_id text not null,
|
||||||
update_timestamp timestamptz,
|
update_timestamp timestamptz,
|
||||||
|
@ -786,6 +796,8 @@ create index IDXhp33wybmb6tbpr1bq7ttwk8je on "ContactHistory" (history_registrar
|
||||||
create index IDX9q53px6r302ftgisqifmc6put on "ContactHistory" (history_type);
|
create index IDX9q53px6r302ftgisqifmc6put on "ContactHistory" (history_type);
|
||||||
create index IDXsudwswtwqnfnx2o1hx4s0k0g5 on "ContactHistory" (history_modification_time);
|
create index IDXsudwswtwqnfnx2o1hx4s0k0g5 on "ContactHistory" (history_modification_time);
|
||||||
create index IDXhlqqd5uy98cjyos72d81x9j95 on "DelegationSignerData" (domain_repo_id);
|
create index IDXhlqqd5uy98cjyos72d81x9j95 on "DelegationSignerData" (domain_repo_id);
|
||||||
|
create index IDX8gtvnbk64yskcvrdp61f5ied3 on "DnsRefreshRequest" (request_time);
|
||||||
|
create index IDXfdk2xpil2x1gh0omt84k2y3o1 on "DnsRefreshRequest" (last_process_time);
|
||||||
create index IDXnb02m43jcx24r64n8rbg22u4q on "Domain" (admin_contact);
|
create index IDXnb02m43jcx24r64n8rbg22u4q on "Domain" (admin_contact);
|
||||||
create index IDXlrq7v63pc21uoh3auq6eybyhl on "Domain" (autorenew_end_time);
|
create index IDXlrq7v63pc21uoh3auq6eybyhl on "Domain" (autorenew_end_time);
|
||||||
create index IDXq9gy8x2xynt9tb16yajn1gcm8 on "Domain" (billing_contact);
|
create index IDXq9gy8x2xynt9tb16yajn1gcm8 on "Domain" (billing_contact);
|
||||||
|
|
Loading…
Add table
Reference in a new issue