From 3d809e762b941e72afcb81f0e72f393b7d3fd1a1 Mon Sep 17 00:00:00 2001 From: Lai Jiang Date: Thu, 16 Mar 2023 13:02:20 -0400 Subject: [PATCH] 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. --- .../batch/DeleteProberDataAction.java | 6 +- .../google/registry/dns/DnsConstants.java | 5 +- .../java/google/registry/dns/DnsQueue.java | 17 +- .../java/google/registry/dns/DnsUtils.java | 74 ++++++++ .../registry/dns/PublishDnsUpdatesAction.java | 10 +- .../google/registry/dns/RefreshDnsAction.java | 10 +- .../dns/RefreshDnsOnHostRenameAction.java | 8 +- .../flows/domain/DomainCreateFlow.java | 19 +- .../flows/domain/DomainDeleteFlow.java | 6 +- .../domain/DomainRestoreRequestFlow.java | 6 +- .../flows/domain/DomainUpdateFlow.java | 6 +- .../registry/flows/host/HostCreateFlow.java | 6 +- .../registry/flows/host/HostDeleteFlow.java | 6 +- .../registry/flows/host/HostUpdateFlow.java | 8 +- .../DatabaseMigrationStateSchedule.java | 11 +- .../model/common/DnsRefreshRequest.java | 133 ++++++++++++++ .../server/RefreshDnsForAllDomainsAction.java | 6 +- .../main/resources/META-INF/persistence.xml | 1 + .../batch/DeleteProberDataActionTest.java | 18 +- .../google/registry/dns/DnsUtilsTest.java | 170 ++++++++++++++++++ .../dns/PublishDnsUpdatesActionTest.java | 42 ++--- .../registry/dns/ReadDnsQueueActionTest.java | 15 -- .../registry/dns/RefreshDnsActionTest.java | 18 +- .../dns/RefreshDnsOnHostRenameActionTest.java | 21 +-- .../registry/flows/EppTestComponent.java | 13 ++ .../google/registry/flows/FlowTestCase.java | 3 + .../flows/domain/DomainCreateFlowTest.java | 34 ++-- .../flows/domain/DomainDeleteFlowTest.java | 11 +- .../domain/DomainRestoreRequestFlowTest.java | 9 +- .../flows/domain/DomainUpdateFlowTest.java | 20 +-- .../flows/host/HostCreateFlowTest.java | 14 +- .../flows/host/HostDeleteFlowTest.java | 10 +- .../flows/host/HostUpdateFlowTest.java | 26 ++- .../model/common/DnsRefreshRequestTest.java | 93 ++++++++++ .../integration/SqlIntegrationTestSuite.java | 2 + .../registry/testing/DnsUtilsHelper.java | 73 ++++++++ .../RefreshDnsForAllDomainsActionTest.java | 49 +++-- .../sql/schema/db-schema.sql.generated | 12 ++ 38 files changed, 760 insertions(+), 231 deletions(-) create mode 100644 core/src/main/java/google/registry/dns/DnsUtils.java create mode 100644 core/src/main/java/google/registry/model/common/DnsRefreshRequest.java create mode 100644 core/src/test/java/google/registry/dns/DnsUtilsTest.java create mode 100644 core/src/test/java/google/registry/model/common/DnsRefreshRequestTest.java create mode 100644 core/src/test/java/google/registry/testing/DnsUtilsHelper.java diff --git a/core/src/main/java/google/registry/batch/DeleteProberDataAction.java b/core/src/main/java/google/registry/batch/DeleteProberDataAction.java index a924d1c5e..4370c9e7f 100644 --- a/core/src/main/java/google/registry/batch/DeleteProberDataAction.java +++ b/core/src/main/java/google/registry/batch/DeleteProberDataAction.java @@ -32,7 +32,7 @@ import com.google.common.collect.Sets; import com.google.common.flogger.FluentLogger; import google.registry.config.RegistryConfig.Config; import google.registry.config.RegistryEnvironment; -import google.registry.dns.DnsQueue; +import google.registry.dns.DnsUtils; import google.registry.model.CreateAutoTimestamp; import google.registry.model.EppResourceUtils; 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. */ private static final int BATCH_SIZE = 1000; - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; @Inject @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 // anyway. tm().putAll(ImmutableList.of(deletedDomain, historyEntry)); - dnsQueue.addDomainRefreshTask(deletedDomain.getDomainName()); + dnsUtils.requestDomainDnsRefresh(deletedDomain.getDomainName()); } } diff --git a/core/src/main/java/google/registry/dns/DnsConstants.java b/core/src/main/java/google/registry/dns/DnsConstants.java index 37d1ec377..a08e7e9bb 100644 --- a/core/src/main/java/google/registry/dns/DnsConstants.java +++ b/core/src/main/java/google/registry/dns/DnsConstants.java @@ -34,5 +34,8 @@ public class DnsConstants { public static final String DNS_TARGET_CREATE_TIME_PARAM = "Create-Time"; /** The possible values of the {@code DNS_TARGET_TYPE_PARAM} parameter. */ - public enum TargetType { DOMAIN, HOST, ZONE } + public enum TargetType { + DOMAIN, + HOST + } } diff --git a/core/src/main/java/google/registry/dns/DnsQueue.java b/core/src/main/java/google/registry/dns/DnsQueue.java index dfa61208a..36a39fca5 100644 --- a/core/src/main/java/google/registry/dns/DnsQueue.java +++ b/core/src/main/java/google/registry/dns/DnsQueue.java @@ -77,11 +77,15 @@ public class DnsQueue { private static final RateLimiter rateLimiter = RateLimiter.create(9); @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.clock = clock; } + Clock getClock() { + return clock; + } + @VisibleForTesting public static DnsQueue createForTesting(Clock 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. */ - public TaskHandle addHostRefreshTask(String hostName) { + TaskHandle addHostRefreshTask(String hostName) { Optional tld = Registries.findTldForName(InternetDomainName.from(hostName)); checkArgument( 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. */ - public TaskHandle addDomainRefreshTask(String domainName) { + TaskHandle addDomainRefreshTask(String domainName) { return addDomainRefreshTask(domainName, Duration.ZERO); } /** 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( TargetType.DOMAIN, domainName, @@ -129,11 +133,6 @@ public class DnsQueue { 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}. * diff --git a/core/src/main/java/google/registry/dns/DnsUtils.java b/core/src/main/java/google/registry/dns/DnsUtils.java new file mode 100644 index 000000000..f14b1950e --- /dev/null +++ b/core/src/main/java/google/registry/dns/DnsUtils.java @@ -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); + } +} diff --git a/core/src/main/java/google/registry/dns/PublishDnsUpdatesAction.java b/core/src/main/java/google/registry/dns/PublishDnsUpdatesAction.java index 1ee910823..bb3b7666a 100644 --- a/core/src/main/java/google/registry/dns/PublishDnsUpdatesAction.java +++ b/core/src/main/java/google/registry/dns/PublishDnsUpdatesAction.java @@ -85,7 +85,7 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable { private static final FluentLogger logger = FluentLogger.forEnclosingClass(); - private final DnsQueue dnsQueue; + private final DnsUtils dnsUtils; private final DnsWriterProxy dnsWriterProxy; private final DnsMetrics dnsMetrics; private final Duration timeout; @@ -139,7 +139,7 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable { @Config("gSuiteOutgoingEmailAddress") InternetAddress gSuiteOutgoingEmailAddress, @Header(APP_ENGINE_RETRY_HEADER) Optional appEngineRetryCount, @Header(CLOUD_TASKS_RETRY_HEADER) Optional cloudTasksRetryCount, - DnsQueue dnsQueue, + DnsUtils dnsUtils, DnsWriterProxy dnsWriterProxy, DnsMetrics dnsMetrics, LockHandler lockHandler, @@ -147,7 +147,7 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable { CloudTasksUtils cloudTasksUtils, SendEmailService sendEmailService, Response response) { - this.dnsQueue = dnsQueue; + this.dnsUtils = dnsUtils; this.dnsWriterProxy = dnsWriterProxy; this.dnsMetrics = dnsMetrics; this.timeout = timeout; @@ -356,10 +356,10 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable { private void requeueBatch() { logger.atInfo().log("Requeueing batch for retry."); for (String domain : nullToEmpty(domains)) { - dnsQueue.addDomainRefreshTask(domain); + dnsUtils.requestDomainDnsRefresh(domain); } for (String host : nullToEmpty(hosts)) { - dnsQueue.addHostRefreshTask(host); + dnsUtils.requestHostDnsRefresh(host); } } diff --git a/core/src/main/java/google/registry/dns/RefreshDnsAction.java b/core/src/main/java/google/registry/dns/RefreshDnsAction.java index a0d9311bb..3cb81c6ce 100644 --- a/core/src/main/java/google/registry/dns/RefreshDnsAction.java +++ b/core/src/main/java/google/registry/dns/RefreshDnsAction.java @@ -39,7 +39,7 @@ import javax.inject.Inject; public final class RefreshDnsAction implements Runnable { private final Clock clock; - private final DnsQueue dnsQueue; + private final DnsUtils dnsUtils; private final String domainOrHostName; private final TargetType type; @@ -48,11 +48,11 @@ public final class RefreshDnsAction implements Runnable { @Parameter("domainOrHostName") String domainOrHostName, @Parameter("type") TargetType type, Clock clock, - DnsQueue dnsQueue) { + DnsUtils dnsUtils) { this.domainOrHostName = domainOrHostName; this.type = type; this.clock = clock; - this.dnsQueue = dnsQueue; + this.dnsUtils = dnsUtils; } @Override @@ -63,11 +63,11 @@ public final class RefreshDnsAction implements Runnable { switch (type) { case DOMAIN: loadAndVerifyExistence(Domain.class, domainOrHostName); - dnsQueue.addDomainRefreshTask(domainOrHostName); + dnsUtils.requestDomainDnsRefresh(domainOrHostName); break; case HOST: verifyHostIsSubordinate(loadAndVerifyExistence(Host.class, domainOrHostName)); - dnsQueue.addHostRefreshTask(domainOrHostName); + dnsUtils.requestHostDnsRefresh(domainOrHostName); break; default: throw new BadRequestException("Unsupported type: " + type); diff --git a/core/src/main/java/google/registry/dns/RefreshDnsOnHostRenameAction.java b/core/src/main/java/google/registry/dns/RefreshDnsOnHostRenameAction.java index 7e27281a0..573349ae6 100644 --- a/core/src/main/java/google/registry/dns/RefreshDnsOnHostRenameAction.java +++ b/core/src/main/java/google/registry/dns/RefreshDnsOnHostRenameAction.java @@ -45,14 +45,14 @@ public class RefreshDnsOnHostRenameAction implements Runnable { private final VKey hostKey; private final Response response; - private final DnsQueue dnsQueue; + private final DnsUtils dnsUtils; @Inject 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.response = response; - this.dnsQueue = dnsQueue; + this.dnsUtils = dnsUtils; } @Override @@ -76,7 +76,7 @@ public class RefreshDnsOnHostRenameAction implements Runnable { .stream() .map(domainKey -> tm().loadByKey(domainKey)) .filter(Domain::shouldPublishToDns) - .forEach(domain -> dnsQueue.addDomainRefreshTask(domain.getDomainName())); + .forEach(domain -> dnsUtils.requestDomainDnsRefresh(domain.getDomainName())); } if (!hostValid) { diff --git a/core/src/main/java/google/registry/flows/domain/DomainCreateFlow.java b/core/src/main/java/google/registry/flows/domain/DomainCreateFlow.java index 29526c206..c222d2d8e 100644 --- a/core/src/main/java/google/registry/flows/domain/DomainCreateFlow.java +++ b/core/src/main/java/google/registry/flows/domain/DomainCreateFlow.java @@ -62,7 +62,7 @@ import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; 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.AssociationProhibitsOperationException; import google.registry.flows.EppException.CommandUseErrorException; @@ -235,7 +235,8 @@ public final class DomainCreateFlow implements TransactionalFlow { @Inject DomainCreateFlowCustomLogic flowCustomLogic; @Inject DomainFlowTmchUtils tmchUtils; @Inject DomainPricingLogic pricingLogic; - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; + @Inject DomainCreateFlow() {} @Override @@ -405,8 +406,10 @@ public final class DomainCreateFlow implements TransactionalFlow { .addGracePeriod( GracePeriod.forBillingEvent(GracePeriodStatus.ADD, repoId, createBillingEvent)) .setLordnPhase( - !DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime()) - .equals(MigrationState.NORDN_SQL) + !(DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime()) + .equals(MigrationState.NORDN_SQL) + || DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime()) + .equals(MigrationState.DNS_SQL)) ? LordnPhase.NONE : hasSignedMarks ? LordnPhase.SUNRISE @@ -709,10 +712,12 @@ public final class DomainCreateFlow implements TransactionalFlow { private void enqueueTasks(Domain newDomain, boolean hasSignedMarks, boolean hasClaimsNotice) { if (newDomain.shouldPublishToDns()) { - dnsQueue.addDomainRefreshTask(newDomain.getDomainName()); + dnsUtils.requestDomainDnsRefresh(newDomain.getDomainName()); } - if (!DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime()) - .equals(MigrationState.NORDN_SQL) + if (!(DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime()) + .equals(MigrationState.NORDN_SQL) + || DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime()) + .equals(MigrationState.DNS_SQL)) && (hasClaimsNotice || hasSignedMarks)) { LordnTaskUtils.enqueueDomainTask(newDomain); } diff --git a/core/src/main/java/google/registry/flows/domain/DomainDeleteFlow.java b/core/src/main/java/google/registry/flows/domain/DomainDeleteFlow.java index 3d6bc07c6..f3be54dda 100644 --- a/core/src/main/java/google/registry/flows/domain/DomainDeleteFlow.java +++ b/core/src/main/java/google/registry/flows/domain/DomainDeleteFlow.java @@ -44,7 +44,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Sets; 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.AssociationProhibitsOperationException; import google.registry.flows.ExtensionManager; @@ -129,7 +129,7 @@ public final class DomainDeleteFlow implements TransactionalFlow { @Inject @TargetId String targetId; @Inject @Superuser boolean isSuperuser; @Inject DomainHistory.Builder historyBuilder; - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; @Inject Trid trid; @Inject AsyncTaskEnqueuer asyncTaskEnqueuer; @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 // event and poll message will already have been deleted in // ResourceDeleteFlow since it's listed in serverApproveEntities. - dnsQueue.addDomainRefreshTask(existingDomain.getDomainName()); + dnsUtils.requestDomainDnsRefresh(existingDomain.getDomainName()); entitiesToSave.add(newDomain, domainHistory); EntityChanges entityChanges = diff --git a/core/src/main/java/google/registry/flows/domain/DomainRestoreRequestFlow.java b/core/src/main/java/google/registry/flows/domain/DomainRestoreRequestFlow.java index dcffe4d24..e928b1131 100644 --- a/core/src/main/java/google/registry/flows/domain/DomainRestoreRequestFlow.java +++ b/core/src/main/java/google/registry/flows/domain/DomainRestoreRequestFlow.java @@ -34,7 +34,7 @@ import static google.registry.util.DateTimeUtils.END_OF_TIME; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; 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.CommandUseErrorException; import google.registry.flows.EppException.StatusProhibitsOperationException; @@ -122,7 +122,7 @@ public final class DomainRestoreRequestFlow implements TransactionalFlow { @Inject @TargetId String targetId; @Inject @Superuser boolean isSuperuser; @Inject DomainHistory.Builder historyBuilder; - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; @Inject EppResponse.Builder responseBuilder; @Inject DomainPricingLogic pricingLogic; @Inject DomainRestoreRequestFlow() {} @@ -186,7 +186,7 @@ public final class DomainRestoreRequestFlow implements TransactionalFlow { entitiesToSave.add(newDomain, domainHistory, autorenewEvent, autorenewPollMessage); tm().putAll(entitiesToSave.build()); tm().delete(existingDomain.getDeletePollMessage()); - dnsQueue.addDomainRefreshTask(existingDomain.getDomainName()); + dnsUtils.requestDomainDnsRefresh(existingDomain.getDomainName()); return responseBuilder .setExtensions(createResponseExtensions(feesAndCredits, feeUpdate, isExpired)) .build(); diff --git a/core/src/main/java/google/registry/flows/domain/DomainUpdateFlow.java b/core/src/main/java/google/registry/flows/domain/DomainUpdateFlow.java index caeaa76d1..101e43807 100644 --- a/core/src/main/java/google/registry/flows/domain/DomainUpdateFlow.java +++ b/core/src/main/java/google/registry/flows/domain/DomainUpdateFlow.java @@ -49,7 +49,7 @@ import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; 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.ExtensionManager; import google.registry.flows.FlowModule.RegistrarId; @@ -156,7 +156,7 @@ public final class DomainUpdateFlow implements TransactionalFlow { @Inject @Superuser boolean isSuperuser; @Inject Trid trid; @Inject DomainHistory.Builder historyBuilder; - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; @Inject EppResponse.Builder responseBuilder; @Inject DomainUpdateFlowCustomLogic flowCustomLogic; @Inject DomainPricingLogic pricingLogic; @@ -183,7 +183,7 @@ public final class DomainUpdateFlow implements TransactionalFlow { historyBuilder.setType(DOMAIN_UPDATE).setDomain(newDomain).build(); validateNewState(newDomain); if (requiresDnsUpdate(existingDomain, newDomain)) { - dnsQueue.addDomainRefreshTask(targetId); + dnsUtils.requestDomainDnsRefresh(targetId); } ImmutableSet.Builder entitiesToSave = new ImmutableSet.Builder<>(); entitiesToSave.add(newDomain, domainHistory); diff --git a/core/src/main/java/google/registry/flows/host/HostCreateFlow.java b/core/src/main/java/google/registry/flows/host/HostCreateFlow.java index f1be9a2b1..63bbcfbf3 100644 --- a/core/src/main/java/google/registry/flows/host/HostCreateFlow.java +++ b/core/src/main/java/google/registry/flows/host/HostCreateFlow.java @@ -28,7 +28,7 @@ import static google.registry.util.CollectionUtils.isNullOrEmpty; import com.google.common.collect.ImmutableSet; 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.ParameterValueRangeErrorException; import google.registry.flows.EppException.RequiredParameterMissingException; @@ -85,7 +85,7 @@ public final class HostCreateFlow implements TransactionalFlow { @Inject @RegistrarId String registrarId; @Inject @TargetId String targetId; @Inject HostHistory.Builder historyBuilder; - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; @Inject EppResponse.Builder responseBuilder; @Inject @@ -138,7 +138,7 @@ public final class HostCreateFlow implements TransactionalFlow { .build()); // 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. - dnsQueue.addHostRefreshTask(targetId); + dnsUtils.requestHostDnsRefresh(targetId); } tm().insertAll(entitiesToSave); return responseBuilder.setResData(HostCreateData.create(targetId, now)).build(); diff --git a/core/src/main/java/google/registry/flows/host/HostDeleteFlow.java b/core/src/main/java/google/registry/flows/host/HostDeleteFlow.java index 0b5160eef..07b1a647a 100644 --- a/core/src/main/java/google/registry/flows/host/HostDeleteFlow.java +++ b/core/src/main/java/google/registry/flows/host/HostDeleteFlow.java @@ -24,7 +24,7 @@ import static google.registry.model.eppoutput.Result.Code.SUCCESS; import static google.registry.persistence.transaction.TransactionManagerFactory.tm; 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.ExtensionManager; import google.registry.flows.FlowModule.RegistrarId; @@ -71,7 +71,7 @@ public final class HostDeleteFlow implements TransactionalFlow { StatusValue.PENDING_DELETE, StatusValue.SERVER_DELETE_PROHIBITED); - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; @Inject ExtensionManager extensionManager; @Inject @RegistrarId String registrarId; @Inject @TargetId String targetId; @@ -104,7 +104,7 @@ public final class HostDeleteFlow implements TransactionalFlow { } Host newHost = existingHost.asBuilder().setStatusValues(null).setDeletionTime(now).build(); if (existingHost.isSubordinate()) { - dnsQueue.addHostRefreshTask(existingHost.getHostName()); + dnsUtils.requestHostDnsRefresh(existingHost.getHostName()); tm().update( tm().loadByKey(existingHost.getSuperordinateDomain()) .asBuilder() diff --git a/core/src/main/java/google/registry/flows/host/HostUpdateFlow.java b/core/src/main/java/google/registry/flows/host/HostUpdateFlow.java index 7ab4dd862..b459d5318 100644 --- a/core/src/main/java/google/registry/flows/host/HostUpdateFlow.java +++ b/core/src/main/java/google/registry/flows/host/HostUpdateFlow.java @@ -37,7 +37,7 @@ import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.ImmutableSet; import google.registry.batch.AsyncTaskEnqueuer; import google.registry.batch.CloudTasksUtils; -import google.registry.dns.DnsQueue; +import google.registry.dns.DnsUtils; import google.registry.dns.RefreshDnsOnHostRenameAction; import google.registry.flows.EppException; import google.registry.flows.EppException.ObjectAlreadyExistsException; @@ -124,7 +124,7 @@ public final class HostUpdateFlow implements TransactionalFlow { @Inject @Superuser boolean isSuperuser; @Inject HostHistory.Builder historyBuilder; @Inject AsyncTaskEnqueuer asyncTaskEnqueuer; - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; @Inject EppResponse.Builder responseBuilder; @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 // are only written as NS records from the referencing domain. 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. if (((Update) resourceCommand).getInnerChange().getHostName() != null) { // If the renamed host is also subordinate, then we must enqueue an update to write the new // glue. 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 // that their NS records can be updated to point at the new name. diff --git a/core/src/main/java/google/registry/model/common/DatabaseMigrationStateSchedule.java b/core/src/main/java/google/registry/model/common/DatabaseMigrationStateSchedule.java index 93b867bb2..4f1375ca8 100644 --- a/core/src/main/java/google/registry/model/common/DatabaseMigrationStateSchedule.java +++ b/core/src/main/java/google/registry/model/common/DatabaseMigrationStateSchedule.java @@ -90,7 +90,10 @@ public class DatabaseMigrationStateSchedule extends CrossTldSingleton { SEQUENCE_BASED_ALLOCATE_ID(PrimaryDatabase.CLOUD_SQL, false, ReplayDirection.NO_REPLAY), /** 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 boolean isReadOnly; @@ -171,7 +174,11 @@ public class DatabaseMigrationStateSchedule extends CrossTldSingleton { MigrationState.SQL_PRIMARY) .putAll(MigrationState.SQL_ONLY, MigrationState.SEQUENCE_BASED_ALLOCATE_ID) .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). Arrays.stream(MigrationState.values()).forEach(state -> builder.put(state, state)); diff --git a/core/src/main/java/google/registry/model/common/DnsRefreshRequest.java b/core/src/main/java/google/registry/model/common/DnsRefreshRequest.java new file mode 100644 index 000000000..8d38cc311 --- /dev/null +++ b/core/src/main/java/google/registry/model/common/DnsRefreshRequest.java @@ -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. + * + *

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. + * + *

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); + } +} diff --git a/core/src/main/java/google/registry/tools/server/RefreshDnsForAllDomainsAction.java b/core/src/main/java/google/registry/tools/server/RefreshDnsForAllDomainsAction.java index 417883744..b7fa2af7a 100644 --- a/core/src/main/java/google/registry/tools/server/RefreshDnsForAllDomainsAction.java +++ b/core/src/main/java/google/registry/tools/server/RefreshDnsForAllDomainsAction.java @@ -21,7 +21,7 @@ import static google.registry.request.RequestParameters.PARAM_TLDS; import com.google.common.collect.ImmutableSet; 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.Parameter; import google.registry.request.Response; @@ -66,7 +66,7 @@ public class RefreshDnsForAllDomainsAction implements Runnable { @Parameter("smearMinutes") int smearMinutes; - @Inject DnsQueue dnsQueue; + @Inject DnsUtils dnsUtils; @Inject Clock clock; @Inject Random random; @@ -91,7 +91,7 @@ public class RefreshDnsForAllDomainsAction implements Runnable { domainName -> { try { // Smear the task execution time over the next N minutes. - dnsQueue.addDomainRefreshTask( + dnsUtils.requestDomainDnsRefresh( domainName, Duration.standardMinutes(random.nextInt(smearMinutes))); } catch (Throwable t) { logger.atSevere().withCause(t).log( diff --git a/core/src/main/resources/META-INF/persistence.xml b/core/src/main/resources/META-INF/persistence.xml index 9dbc95e7f..cf5b0055a 100644 --- a/core/src/main/resources/META-INF/persistence.xml +++ b/core/src/main/resources/META-INF/persistence.xml @@ -43,6 +43,7 @@ google.registry.model.billing.BillingEvent$Recurring google.registry.model.common.Cursor google.registry.model.common.DatabaseMigrationStateSchedule + google.registry.model.common.DnsRefreshRequest google.registry.model.console.User google.registry.model.contact.ContactHistory google.registry.model.contact.Contact diff --git a/core/src/test/java/google/registry/batch/DeleteProberDataActionTest.java b/core/src/test/java/google/registry/batch/DeleteProberDataActionTest.java index d36155138..db492982e 100644 --- a/core/src/test/java/google/registry/batch/DeleteProberDataActionTest.java +++ b/core/src/test/java/google/registry/batch/DeleteProberDataActionTest.java @@ -27,14 +27,14 @@ import static google.registry.testing.DatabaseHelper.persistDeletedDomain; import static google.registry.testing.DatabaseHelper.persistDomainAsDeleted; import static google.registry.testing.DatabaseHelper.persistResource; 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 org.joda.time.DateTimeZone.UTC; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; import com.google.common.collect.ImmutableSet; import google.registry.config.RegistryEnvironment; -import google.registry.dns.DnsQueue; +import google.registry.dns.DnsUtils; import google.registry.model.ImmutableObject; import google.registry.model.billing.BillingEvent; 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.JpaIntegrationTestExtension; import google.registry.testing.DatabaseHelper; -import google.registry.testing.FakeClock; +import google.registry.testing.DnsUtilsHelper; import google.registry.testing.SystemPropertyExtension; -import google.registry.testing.TaskQueueExtension; import java.util.Optional; import java.util.Set; import org.joda.money.Money; @@ -68,13 +67,14 @@ class DeleteProberDataActionTest { final JpaIntegrationTestExtension jpa = new JpaTestExtensions.Builder().buildIntegrationTestExtension(); - @RegisterExtension TaskQueueExtension taskQueue = new TaskQueueExtension(); - @RegisterExtension final SystemPropertyExtension systemPropertyExtension = new SystemPropertyExtension(); private DeleteProberDataAction action; + private final DnsUtils dnsUtils = mock(DnsUtils.class); + private final DnsUtilsHelper dnsUtilsHelper = new DnsUtilsHelper(dnsUtils); + @BeforeEach void beforeEach() { // Entities in these two should not be touched. @@ -99,7 +99,7 @@ class DeleteProberDataActionTest { private void resetAction() { action = new DeleteProberDataAction(); - action.dnsQueue = DnsQueue.createForTesting(new FakeClock()); + action.dnsUtils = dnsUtils; action.isDryRun = false; action.tlds = ImmutableSet.of(); action.registryAdminRegistrarId = "TheRegistrar"; @@ -201,7 +201,7 @@ class DeleteProberDataActionTest { DateTime timeAfterDeletion = DateTime.now(UTC); assertThat(loadByForeignKey(Domain.class, "blah.ib-any.test", timeAfterDeletion)).isEmpty(); assertThat(loadByEntity(domain).getDeletionTime()).isLessThan(timeAfterDeletion); - assertDnsTasksEnqueued("blah.ib-any.test"); + dnsUtilsHelper.assertDomainDnsRequests("blah.ib-any.test"); } @Test @@ -218,7 +218,7 @@ class DeleteProberDataActionTest { action.run(); assertThat(loadByForeignKey(Domain.class, "blah.ib-any.test", timeAfterDeletion)).isEmpty(); assertThat(loadByEntity(domain).getDeletionTime()).isLessThan(timeAfterDeletion); - assertDnsTasksEnqueued("blah.ib-any.test"); + dnsUtilsHelper.assertDomainDnsRequests("blah.ib-any.test"); } @Test diff --git a/core/src/test/java/google/registry/dns/DnsUtilsTest.java b/core/src/test/java/google/registry/dns/DnsUtilsTest.java new file mode 100644 index 000000000..8469cbaa9 --- /dev/null +++ b/core/src/test/java/google/registry/dns/DnsUtilsTest.java @@ -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(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); + } +} diff --git a/core/src/test/java/google/registry/dns/PublishDnsUpdatesActionTest.java b/core/src/test/java/google/registry/dns/PublishDnsUpdatesActionTest.java index 81f493a87..ba20118ac 100644 --- a/core/src/test/java/google/registry/dns/PublishDnsUpdatesActionTest.java +++ b/core/src/test/java/google/registry/dns/PublishDnsUpdatesActionTest.java @@ -54,6 +54,7 @@ import google.registry.request.HttpException.ServiceUnavailableException; import google.registry.request.lock.LockHandler; import google.registry.testing.CloudTasksHelper; import google.registry.testing.CloudTasksHelper.TaskMatcher; +import google.registry.testing.DnsUtilsHelper; import google.registry.testing.FakeClock; import google.registry.testing.FakeLockHandler; import google.registry.testing.FakeResponse; @@ -82,7 +83,8 @@ public class PublishDnsUpdatesActionTest { private final FakeLockHandler lockHandler = new FakeLockHandler(true); private final DnsWriter dnsWriter = mock(DnsWriter.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 PublishDnsUpdatesAction action; private InternetAddress outgoingRegistry; @@ -162,7 +164,7 @@ public class PublishDnsUpdatesActionTest { outgoingRegistry, Optional.ofNullable(retryCount), Optional.empty(), - dnsQueue, + dnsUtils, new DnsWriterProxy(ImmutableMap.of("correctWriter", dnsWriter)), dnsMetrics, lockHandler, @@ -196,7 +198,7 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertNoMoreDnsRequests(); assertThat(response.getStatus()).isEqualTo(SC_OK); } @@ -223,7 +225,7 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertNoMoreDnsRequests(); assertThat(response.getStatus()).isEqualTo(SC_OK); } @@ -276,7 +278,7 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -496,7 +498,7 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -526,7 +528,7 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -554,7 +556,7 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -580,9 +582,9 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verify(dnsQueue).addDomainRefreshTask("example.com"); - verify(dnsQueue).addHostRefreshTask("ns1.example.com"); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertDomainDnsRequests("example.com"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.com"); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -608,9 +610,9 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verify(dnsQueue).addDomainRefreshTask("example.com"); - verify(dnsQueue).addHostRefreshTask("ns1.example.com"); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertDomainDnsRequests("example.com"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.com"); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -632,11 +634,11 @@ public class PublishDnsUpdatesActionTest { Duration.standardHours(2), Duration.standardHours(1)); verifyNoMoreInteractions(dnsMetrics); - verify(dnsQueue).addDomainRefreshTask("example.com"); - verify(dnsQueue).addDomainRefreshTask("example2.com"); - verify(dnsQueue).addHostRefreshTask("ns1.example.com"); - verify(dnsQueue).addHostRefreshTask("ns2.example.com"); - verify(dnsQueue).addHostRefreshTask("ns1.example2.com"); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertDomainDnsRequests("example.com"); + dnsUtilsHelper.assertDomainDnsRequests("example2.com"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.com"); + dnsUtilsHelper.assertHostDnsRequests("ns2.example.com"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example2.com"); + dnsUtilsHelper.assertNoMoreDnsRequests(); } } diff --git a/core/src/test/java/google/registry/dns/ReadDnsQueueActionTest.java b/core/src/test/java/google/registry/dns/ReadDnsQueueActionTest.java index 329f6fe01..1fe4aaaab 100644 --- a/core/src/test/java/google/registry/dns/ReadDnsQueueActionTest.java +++ b/core/src/test/java/google/registry/dns/ReadDnsQueueActionTest.java @@ -388,21 +388,6 @@ public class ReadDnsQueueActionTest { 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) { return IntStream.range(from, to) .mapToObj(i -> String.format(format, i)) diff --git a/core/src/test/java/google/registry/dns/RefreshDnsActionTest.java b/core/src/test/java/google/registry/dns/RefreshDnsActionTest.java index 8c65e6f9e..7c4144c2a 100644 --- a/core/src/test/java/google/registry/dns/RefreshDnsActionTest.java +++ b/core/src/test/java/google/registry/dns/RefreshDnsActionTest.java @@ -21,8 +21,6 @@ import static google.registry.testing.DatabaseHelper.persistActiveHost; import static google.registry.testing.DatabaseHelper.persistActiveSubordinateHost; import static org.junit.jupiter.api.Assertions.assertThrows; 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.model.domain.Domain; @@ -30,6 +28,7 @@ import google.registry.persistence.transaction.JpaTestExtensions; import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension; import google.registry.request.HttpException.BadRequestException; import google.registry.request.HttpException.NotFoundException; +import google.registry.testing.DnsUtilsHelper; import google.registry.testing.FakeClock; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -42,11 +41,12 @@ public class RefreshDnsActionTest { final JpaIntegrationTestExtension jpa = 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 void run(TargetType type, String name) { - new RefreshDnsAction(name, type, clock, dnsQueue).run(); + new RefreshDnsAction(name, type, clock, dnsUtils).run(); } @BeforeEach @@ -59,8 +59,8 @@ public class RefreshDnsActionTest { Domain domain = persistActiveDomain("example.xn--q9jyb4c"); persistActiveSubordinateHost("ns1.example.xn--q9jyb4c", domain); run(TargetType.HOST, "ns1.example.xn--q9jyb4c"); - verify(dnsQueue).addHostRefreshTask("ns1.example.xn--q9jyb4c"); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.xn--q9jyb4c"); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -74,7 +74,7 @@ public class RefreshDnsActionTest { try { run(TargetType.HOST, "ns1.example.xn--q9jyb4c"); } finally { - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertNoMoreDnsRequests(); } }); assertThat(thrown) @@ -86,8 +86,8 @@ public class RefreshDnsActionTest { void testSuccess_domain() { persistActiveDomain("example.xn--q9jyb4c"); run(TargetType.DOMAIN, "example.xn--q9jyb4c"); - verify(dnsQueue).addDomainRefreshTask("example.xn--q9jyb4c"); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertDomainDnsRequests("example.xn--q9jyb4c"); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test diff --git a/core/src/test/java/google/registry/dns/RefreshDnsOnHostRenameActionTest.java b/core/src/test/java/google/registry/dns/RefreshDnsOnHostRenameActionTest.java index 8233d66e3..df691a573 100644 --- a/core/src/test/java/google/registry/dns/RefreshDnsOnHostRenameActionTest.java +++ b/core/src/test/java/google/registry/dns/RefreshDnsOnHostRenameActionTest.java @@ -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_OK; 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 google.registry.model.eppcommon.StatusValue; import google.registry.model.host.Host; import google.registry.persistence.transaction.JpaTestExtensions; import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension; +import google.registry.testing.DnsUtilsHelper; import google.registry.testing.FakeClock; import google.registry.testing.FakeResponse; import org.joda.time.DateTime; @@ -43,7 +42,8 @@ import org.junit.jupiter.api.extension.RegisterExtension; public class RefreshDnsOnHostRenameActionTest { 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(); @RegisterExtension @@ -53,14 +53,7 @@ public class RefreshDnsOnHostRenameActionTest { private RefreshDnsOnHostRenameAction action; private void createAction(String hostKey) { - action = new RefreshDnsOnHostRenameAction(hostKey, response, dnsQueue); - } - - private void assertDnsTasksEnqueued(String... domains) { - for (String domain : domains) { - verify(dnsQueue).addDomainRefreshTask(domain); - } - verifyNoMoreInteractions(dnsQueue); + action = new RefreshDnsOnHostRenameAction(hostKey, response, dnsUtils); } @BeforeEach @@ -82,7 +75,7 @@ public class RefreshDnsOnHostRenameActionTest { persistDomainAsDeleted(newDomain("deleted.tld", host), clock.nowUtc().minusDays(1)); createAction(host.createVKey().stringify()); action.run(); - assertDnsTasksEnqueued("example.tld", "otherexample.tld"); + dnsUtilsHelper.assertDomainDnsRequests("example.tld", "otherexample.tld"); assertThat(response.getStatus()).isEqualTo(SC_OK); } @@ -90,7 +83,7 @@ public class RefreshDnsOnHostRenameActionTest { void testFailure_nonexistentHost() { createAction("kind:Host@sql:rO0ABXQABGJsYWg"); action.run(); - assertDnsTasksEnqueued(); + dnsUtilsHelper.assertNoMoreDnsRequests(); assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT); assertThat(response.getPayload()) .isEqualTo("Host to refresh does not exist: VKey(sql:blah)"); @@ -102,7 +95,7 @@ public class RefreshDnsOnHostRenameActionTest { persistResource(newDomain("example.tld", host)); createAction(host.createVKey().stringify()); action.run(); - assertDnsTasksEnqueued(); + dnsUtilsHelper.assertNoMoreDnsRequests(); assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT); assertThat(response.getPayload()) .isEqualTo("Host to refresh is already deleted: ns1.example.tld"); diff --git a/core/src/test/java/google/registry/flows/EppTestComponent.java b/core/src/test/java/google/registry/flows/EppTestComponent.java index b2146438e..d3c5c8359 100644 --- a/core/src/test/java/google/registry/flows/EppTestComponent.java +++ b/core/src/test/java/google/registry/flows/EppTestComponent.java @@ -24,6 +24,7 @@ import google.registry.batch.CloudTasksUtils; import google.registry.config.RegistryConfig.ConfigModule; import google.registry.config.RegistryConfig.ConfigModule.TmchCaMode; import google.registry.dns.DnsQueue; +import google.registry.dns.DnsUtils; import google.registry.flows.custom.CustomLogicFactory; import google.registry.flows.custom.TestCustomLogicFactory; 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.lock.LockHandler; import google.registry.testing.CloudTasksHelper; +import google.registry.testing.DnsUtilsHelper; import google.registry.testing.FakeClock; import google.registry.testing.FakeLockHandler; import google.registry.testing.FakeSleeper; @@ -59,11 +61,16 @@ public interface EppTestComponent { private FakeLockHandler lockHandler; private Sleeper sleeper; private CloudTasksHelper cloudTasksHelper; + private DnsUtilsHelper dnsUtilsHelper; public CloudTasksHelper getCloudTasksHelper() { return cloudTasksHelper; } + public DnsUtilsHelper getDnsUtilsHelper() { + return dnsUtilsHelper; + } + public EppMetric.Builder getMetricBuilder() { return metricBuilder; } @@ -82,6 +89,7 @@ public interface EppTestComponent { instance.metricBuilder = EppMetric.builderForRequest(clock); instance.lockHandler = new FakeLockHandler(true); instance.cloudTasksHelper = cloudTasksHelper; + instance.dnsUtilsHelper = new DnsUtilsHelper(); return instance; } @@ -95,6 +103,11 @@ public interface EppTestComponent { return cloudTasksHelper.getTestCloudTasksUtils(); } + @Provides + DnsUtils provideDnsUtils() { + return dnsUtilsHelper.getDnsUtils(); + } + @Provides Clock provideClock() { return clock; diff --git a/core/src/test/java/google/registry/flows/FlowTestCase.java b/core/src/test/java/google/registry/flows/FlowTestCase.java index f17d5187e..70ef50889 100644 --- a/core/src/test/java/google/registry/flows/FlowTestCase.java +++ b/core/src/test/java/google/registry/flows/FlowTestCase.java @@ -45,6 +45,7 @@ import google.registry.persistence.transaction.JpaTestExtensions; import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension; import google.registry.testing.CloudTasksHelper; import google.registry.testing.DatabaseHelper; +import google.registry.testing.DnsUtilsHelper; import google.registry.testing.EppLoader; import google.registry.testing.FakeClock; import google.registry.testing.FakeHttpSession; @@ -84,6 +85,7 @@ public abstract class FlowTestCase { protected TransportCredentials credentials = new PasswordOnlyTransportCredentials(); protected EppRequestSource eppRequestSource = EppRequestSource.UNIT_TEST; protected CloudTasksHelper cloudTasksHelper; + protected DnsUtilsHelper dnsUtilsHelper; private EppMetric.Builder eppMetricBuilder; @@ -216,6 +218,7 @@ public abstract class FlowTestCase { FakesAndMocksModule fakesAndMocksModule = FakesAndMocksModule.create(clock); cloudTasksHelper = fakesAndMocksModule.getCloudTasksHelper(); + dnsUtilsHelper = fakesAndMocksModule.getDnsUtilsHelper(); // Run the flow. return DaggerEppTestComponent.builder() .fakesAndMocksModule(fakesAndMocksModule) diff --git a/core/src/test/java/google/registry/flows/domain/DomainCreateFlowTest.java b/core/src/test/java/google/registry/flows/domain/DomainCreateFlowTest.java index e8f0cdfe8..f27fa2493 100644 --- a/core/src/test/java/google/registry/flows/domain/DomainCreateFlowTest.java +++ b/core/src/test/java/google/registry/flows/domain/DomainCreateFlowTest.java @@ -54,8 +54,6 @@ import static google.registry.testing.DatabaseHelper.persistReservedList; import static google.registry.testing.DatabaseHelper.persistResource; import static google.registry.testing.DomainSubject.assertAboutDomains; 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.assertTasksEnqueued; import static google.registry.tmch.LordnTaskUtils.QUEUE_CLAIMS; @@ -393,7 +391,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase { - @RegisterExtension final TaskQueueExtension taskQueue = new TaskQueueExtension(); - private Domain domain; private DomainHistory earlierHistoryEntry; @@ -362,7 +357,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase { - @RegisterExtension final TaskQueueExtension taskQueue = new TaskQueueExtension(); - private static final ImmutableMap FEE_06_MAP = ImmutableMap.of("FEE_VERSION", "0.6", "FEE_NS", "fee", "CURRENCY", "USD"); private static final ImmutableMap FEE_11_MAP = @@ -192,7 +187,7 @@ class DomainRestoreRequestFlowTest extends ResourceFlowTestCase { - @RegisterExtension final TaskQueueExtension taskQueue = new TaskQueueExtension(); - private static final DomainDsData SOME_DSDATA = DomainDsData.create( 1, @@ -222,7 +216,7 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase ds.cloneWithDomainRepoId(resource.getRepoId())) .collect(toImmutableSet())); if (dnsTaskEnqueued) { - assertDnsTasksEnqueued("example.tld"); + dnsUtilsHelper.assertDomainDnsRequests("example.tld"); } else { - assertNoDnsTasksEnqueued(); + dnsUtilsHelper.assertNoMoreDnsRequests(); } } @@ -1765,7 +1759,7 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase { - @RegisterExtension TaskQueueExtension taskQueue = new TaskQueueExtension(); - private void setEppHostCreateInput(String hostName, String hostAddrs) { setEppInput( "host_create.xml", @@ -122,7 +116,7 @@ class HostCreateFlowTest extends ResourceFlowTestCase { void testSuccess_externalNeverExisted() throws Exception { doSuccessfulTest(); assertAboutHosts().that(reloadResourceByForeignKey()).hasSuperordinateDomain(null); - assertNoDnsTasksEnqueued(); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -133,7 +127,7 @@ class HostCreateFlowTest extends ResourceFlowTestCase { loadByForeignKey(Domain.class, "example.tld", clock.nowUtc()).get(); assertAboutHosts().that(host).hasSuperordinateDomain(superordinateDomain.createVKey()); assertThat(superordinateDomain.getSubordinateHosts()).containsExactly("ns1.example.tld"); - assertDnsTasksEnqueued("ns1.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld"); } @Test @@ -150,7 +144,7 @@ class HostCreateFlowTest extends ResourceFlowTestCase { persistDeletedHost(getUniqueIdFromCommand(), clock.nowUtc().minusDays(1)); doSuccessfulTest(); assertAboutHosts().that(reloadResourceByForeignKey()).hasSuperordinateDomain(null); - assertNoDnsTasksEnqueued(); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -162,7 +156,7 @@ class HostCreateFlowTest extends ResourceFlowTestCase { loadByForeignKey(Domain.class, "example.tld", clock.nowUtc()).get(); assertAboutHosts().that(host).hasSuperordinateDomain(superordinateDomain.createVKey()); assertThat(superordinateDomain.getSubordinateHosts()).containsExactly("ns1.example.tld"); - assertDnsTasksEnqueued("ns1.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld"); } @Test diff --git a/core/src/test/java/google/registry/flows/host/HostDeleteFlowTest.java b/core/src/test/java/google/registry/flows/host/HostDeleteFlowTest.java index f2ff0f9a5..2187e9a55 100644 --- a/core/src/test/java/google/registry/flows/host/HostDeleteFlowTest.java +++ b/core/src/test/java/google/registry/flows/host/HostDeleteFlowTest.java @@ -24,8 +24,6 @@ import static google.registry.testing.DatabaseHelper.persistDeletedHost; import static google.registry.testing.DatabaseHelper.persistResource; import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptions; 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 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.TransferStatus; import google.registry.testing.DatabaseHelper; -import google.registry.testing.TaskQueueExtension; import org.joda.time.DateTime; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; /** Unit tests for {@link HostDeleteFlow}. */ class HostDeleteFlowTest extends ResourceFlowTestCase { - @RegisterExtension TaskQueueExtension taskQueue = new TaskQueueExtension(); - @BeforeEach void initFlowTest() { setEppInput("host_delete.xml", ImmutableMap.of("HOSTNAME", "ns1.example.tld")); @@ -320,10 +314,10 @@ class HostDeleteFlowTest extends ResourceFlowTestCase { .hasType(Type.HOST_DELETE); assertNoBillingEvents(); if (isSubordinate) { - assertDnsTasksEnqueued(deletedHost.getHostName()); + dnsUtilsHelper.assertHostDnsRequests(deletedHost.getHostName()); assertThat(loadByKey(deletedHost.getSuperordinateDomain()).getSubordinateHosts()).isEmpty(); } else { - assertNoDnsTasksEnqueued(); + dnsUtilsHelper.assertNoMoreDnsRequests(); } assertLastHistoryContainsResource(deletedHost); } diff --git a/core/src/test/java/google/registry/flows/host/HostUpdateFlowTest.java b/core/src/test/java/google/registry/flows/host/HostUpdateFlowTest.java index b70024281..87475a5eb 100644 --- a/core/src/test/java/google/registry/flows/host/HostUpdateFlowTest.java +++ b/core/src/test/java/google/registry/flows/host/HostUpdateFlowTest.java @@ -35,8 +35,6 @@ import static google.registry.testing.EppExceptionSubject.assertAboutEppExceptio import static google.registry.testing.GenericEppResourceSubject.assertAboutEppResources; import static google.registry.testing.HistoryEntrySubject.assertAboutHistoryEntries; 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 org.junit.jupiter.api.Assertions.assertThrows; @@ -81,17 +79,13 @@ import google.registry.model.transfer.TransferStatus; import google.registry.persistence.VKey; import google.registry.testing.CloudTasksHelper.TaskMatcher; import google.registry.testing.DatabaseHelper; -import google.registry.testing.TaskQueueExtension; import javax.annotation.Nullable; import org.joda.time.DateTime; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; /** Unit tests for {@link HostUpdateFlow}. */ class HostUpdateFlowTest extends ResourceFlowTestCase { - @RegisterExtension TaskQueueExtension taskQueue = new TaskQueueExtension(); - private void setEppHostUpdateInput( String oldHostName, String newHostName, String ipOrStatusToAdd, String ipOrStatusToRem) { setEppInput( @@ -189,7 +183,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { persistActiveSubordinateHost(oldHostName(), persistActiveDomain("example.tld")); Host renamedHost = doSuccessfulTest(); assertThat(renamedHost.isSubordinate()).isTrue(); - assertDnsTasksEnqueued("ns1.example.tld", "ns2.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld", "ns2.example.tld"); VKey oldVKeyAfterRename = ForeignKeyUtils.load(Host.class, oldHostName(), clock.nowUtc()); assertThat(oldVKeyAfterRename).isNull(); } @@ -238,7 +232,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { .and() .hasOnlyOneHistoryEntryWhich() .hasType(HistoryEntry.Type.HOST_UPDATE); - assertDnsTasksEnqueued("ns1.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld"); } @Test @@ -264,7 +258,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { .and() .hasOnlyOneHistoryEntryWhich() .hasType(HistoryEntry.Type.HOST_UPDATE); - assertDnsTasksEnqueued("ns1.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld"); } @Test @@ -298,7 +292,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { .hasLastTransferTime(oneDayAgo); Domain reloadedDomain = loadByEntity(domain).cloneProjectedAtTime(now); assertThat(reloadedDomain.getSubordinateHosts()).containsExactly("ns2.example.tld"); - assertDnsTasksEnqueued("ns1.example.tld", "ns2.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.tld", "ns2.example.tld"); } @Test @@ -333,7 +327,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { assertThat(loadByEntity(foo).cloneProjectedAtTime(now).getSubordinateHosts()).isEmpty(); assertThat(loadByEntity(example).cloneProjectedAtTime(now).getSubordinateHosts()) .containsExactly("ns2.example.tld"); - assertDnsTasksEnqueued("ns2.foo.tld", "ns2.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns2.foo.tld", "ns2.example.tld"); } @Test @@ -370,7 +364,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { assertThat(reloadedFooDomain.getSubordinateHosts()).isEmpty(); Domain reloadedTldDomain = loadByEntity(tldDomain).cloneProjectedAtTime(now); assertThat(reloadedTldDomain.getSubordinateHosts()).containsExactly("ns2.example.tld"); - assertDnsTasksEnqueued("ns1.example.foo", "ns2.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.foo", "ns2.example.tld"); } @Test @@ -413,7 +407,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { assertThat(renamedHost.getLastTransferTime()).isEqualTo(oneDayAgo); Domain reloadedDomain = loadByEntity(domain).cloneProjectedAtTime(clock.nowUtc()); assertThat(reloadedDomain.getSubordinateHosts()).isEmpty(); - assertDnsTasksEnqueued("ns1.example.foo"); + dnsUtilsHelper.assertHostDnsRequests("ns1.example.foo"); } @Test @@ -425,7 +419,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { persistActiveHost(oldHostName()); assertThat(domain.getSubordinateHosts()).isEmpty(); assertThrows(CannotRenameExternalHostException.class, this::runFlow); - assertNoDnsTasksEnqueued(); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test @@ -449,7 +443,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { .hasLastTransferTime(null); assertThat(loadByEntity(domain).cloneProjectedAtTime(now).getSubordinateHosts()) .containsExactly("ns2.example.tld"); - assertDnsTasksEnqueued("ns2.example.tld"); + dnsUtilsHelper.assertHostDnsRequests("ns2.example.tld"); } @Test @@ -474,7 +468,7 @@ class HostUpdateFlowTest extends ResourceFlowTestCase { .hasPersistedCurrentSponsorRegistrarId("TheRegistrar") .and() .hasLastTransferTime(null); - assertNoDnsTasksEnqueued(); + dnsUtilsHelper.assertNoMoreDnsRequests(); } @Test diff --git a/core/src/test/java/google/registry/model/common/DnsRefreshRequestTest.java b/core/src/test/java/google/registry/model/common/DnsRefreshRequestTest.java new file mode 100644 index 000000000..94fee6f03 --- /dev/null +++ b/core/src/test/java/google/registry/model/common/DnsRefreshRequestTest.java @@ -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 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"); + } +} diff --git a/core/src/test/java/google/registry/schema/integration/SqlIntegrationTestSuite.java b/core/src/test/java/google/registry/schema/integration/SqlIntegrationTestSuite.java index d8e5e2d6d..16e705e77 100644 --- a/core/src/test/java/google/registry/schema/integration/SqlIntegrationTestSuite.java +++ b/core/src/test/java/google/registry/schema/integration/SqlIntegrationTestSuite.java @@ -18,6 +18,7 @@ import static com.google.common.truth.Truth.assert_; import google.registry.model.billing.BillingEventTest; import google.registry.model.common.CursorTest; +import google.registry.model.common.DnsRefreshRequestTest; import google.registry.model.console.UserTest; import google.registry.model.contact.ContactTest; import google.registry.model.domain.DomainSqlTest; @@ -85,6 +86,7 @@ import org.junit.runner.RunWith; ContactHistoryTest.class, ContactTest.class, CursorTest.class, + DnsRefreshRequestTest.class, DomainSqlTest.class, DomainHistoryTest.class, HostHistoryTest.class, diff --git a/core/src/test/java/google/registry/testing/DnsUtilsHelper.java b/core/src/test/java/google/registry/testing/DnsUtilsHelper.java new file mode 100644 index 000000000..e4bbce268 --- /dev/null +++ b/core/src/test/java/google/registry/testing/DnsUtilsHelper.java @@ -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}. + * + *

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); + } +} diff --git a/core/src/test/java/google/registry/tools/server/RefreshDnsForAllDomainsActionTest.java b/core/src/test/java/google/registry/tools/server/RefreshDnsForAllDomainsActionTest.java index 5f5dac655..75bb48b6f 100644 --- a/core/src/test/java/google/registry/tools/server/RefreshDnsForAllDomainsActionTest.java +++ b/core/src/test/java/google/registry/tools/server/RefreshDnsForAllDomainsActionTest.java @@ -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.persistActiveDomain; 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.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.inOrder; 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 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.JpaIntegrationTestExtension; +import google.registry.testing.DnsUtilsHelper; import google.registry.testing.FakeClock; import google.registry.testing.FakeResponse; import java.util.Random; @@ -42,14 +39,13 @@ import org.joda.time.Duration; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; -import org.mockito.ArgumentCaptor; -import org.mockito.InOrder; /** Unit tests for {@link RefreshDnsForAllDomainsAction}. */ public class RefreshDnsForAllDomainsActionTest { 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 final FakeResponse response = new FakeResponse(); @@ -64,27 +60,26 @@ public class RefreshDnsForAllDomainsActionTest { action.random = new Random(); action.random.setSeed(123L); action.clock = clock; - action.dnsQueue = dnsQueue; + action.dnsUtils = dnsUtils; action.response = response; createTld("bar"); } @Test - void test_runAction_errorEnqueuingToDnsQueue() throws Exception { + void test_runAction_errorRequestDnsRefresh() throws Exception { persistActiveDomain("foo.bar"); persistActiveDomain("baz.bar"); persistActiveDomain("low.bar"); action.tlds = ImmutableSet.of("bar"); doThrow(new RuntimeException("Error enqueuing task.")) - .when(dnsQueue) - .addDomainRefreshTask(eq("baz.bar"), any(Duration.class)); + .when(dnsUtils) + .requestDomainDnsRefresh(eq("baz.bar"), any(Duration.class)); action.run(); - InOrder inOrder = inOrder(dnsQueue); - inOrder.verify(dnsQueue).addDomainRefreshTask("low.bar", Duration.ZERO); - inOrder.verify(dnsQueue).addDomainRefreshTask("baz.bar", Duration.ZERO); - inOrder.verify(dnsQueue).addDomainRefreshTask("foo.bar", Duration.ZERO); - verifyNoMoreInteractions(dnsQueue); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("low.bar", Duration.ZERO); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("baz.bar", Duration.ZERO); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.ZERO); + verifyNoMoreInteractions(dnsUtils); assertThat(response.getStatus()).isEqualTo(HttpStatus.SC_INTERNAL_SERVER_ERROR); } @@ -94,8 +89,8 @@ public class RefreshDnsForAllDomainsActionTest { persistActiveDomain("low.bar"); action.tlds = ImmutableSet.of("bar"); action.run(); - verify(dnsQueue).addDomainRefreshTask("foo.bar", Duration.ZERO); - verify(dnsQueue).addDomainRefreshTask("low.bar", Duration.ZERO); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.ZERO); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("low.bar", Duration.ZERO); } @Test @@ -105,10 +100,8 @@ public class RefreshDnsForAllDomainsActionTest { action.tlds = ImmutableSet.of("bar"); action.smearMinutes = 1000; action.run(); - ArgumentCaptor captor = ArgumentCaptor.forClass(Duration.class); - verify(dnsQueue).addDomainRefreshTask(eq("foo.bar"), captor.capture()); - verify(dnsQueue).addDomainRefreshTask(eq("low.bar"), captor.capture()); - assertThat(captor.getAllValues()).containsExactly(standardMinutes(450), standardMinutes(782)); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.standardMinutes(450)); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("low.bar", Duration.standardMinutes(782)); } @Test @@ -117,8 +110,8 @@ public class RefreshDnsForAllDomainsActionTest { persistDeletedDomain("deleted.bar", clock.nowUtc().minusYears(1)); action.tlds = ImmutableSet.of("bar"); action.run(); - verify(dnsQueue).addDomainRefreshTask("foo.bar", Duration.ZERO); - verify(dnsQueue, never()).addDomainRefreshTask("deleted.bar", Duration.ZERO); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.ZERO); + dnsUtilsHelper.assertNoDomainDnsRequestWithDelay("deleted.bar", Duration.ZERO); } @Test @@ -129,9 +122,9 @@ public class RefreshDnsForAllDomainsActionTest { persistActiveDomain("ignore.baz"); action.tlds = ImmutableSet.of("bar"); action.run(); - verify(dnsQueue).addDomainRefreshTask("foo.bar", Duration.ZERO); - verify(dnsQueue).addDomainRefreshTask("low.bar", Duration.ZERO); - verify(dnsQueue, never()).addDomainRefreshTask("ignore.baz", Duration.ZERO); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("foo.bar", Duration.ZERO); + dnsUtilsHelper.assertDomainDnsRequestWithDelay("low.bar", Duration.ZERO); + dnsUtilsHelper.assertNoDomainDnsRequestWithDelay("ignore.baz", Duration.ZERO); } @Test diff --git a/db/src/main/resources/sql/schema/db-schema.sql.generated b/db/src/main/resources/sql/schema/db-schema.sql.generated index 8c5e33804..1ffc5b4e8 100644 --- a/db/src/main/resources/sql/schema/db-schema.sql.generated +++ b/db/src/main/resources/sql/schema/db-schema.sql.generated @@ -255,6 +255,16 @@ 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" ( repo_id text not null, update_timestamp timestamptz, @@ -786,6 +796,8 @@ create index IDXhp33wybmb6tbpr1bq7ttwk8je on "ContactHistory" (history_registrar create index IDX9q53px6r302ftgisqifmc6put on "ContactHistory" (history_type); create index IDXsudwswtwqnfnx2o1hx4s0k0g5 on "ContactHistory" (history_modification_time); 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 IDXlrq7v63pc21uoh3auq6eybyhl on "Domain" (autorenew_end_time); create index IDXq9gy8x2xynt9tb16yajn1gcm8 on "Domain" (billing_contact);