deleteProberData
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 c222d2d8e..12bab5789 100644
--- a/core/src/main/java/google/registry/flows/domain/DomainCreateFlow.java
+++ b/core/src/main/java/google/registry/flows/domain/DomainCreateFlow.java
@@ -86,8 +86,6 @@ import google.registry.model.billing.BillingEvent.Flag;
import google.registry.model.billing.BillingEvent.Reason;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
-import google.registry.model.common.DatabaseMigrationStateSchedule;
-import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainCommand;
import google.registry.model.domain.DomainCommand.Create;
@@ -124,7 +122,6 @@ import google.registry.model.tld.label.ReservationType;
import google.registry.model.tmch.ClaimsList;
import google.registry.model.tmch.ClaimsListDao;
import google.registry.persistence.VKey;
-import google.registry.tmch.LordnTaskUtils;
import google.registry.tmch.LordnTaskUtils.LordnPhase;
import java.util.Map;
import java.util.Optional;
@@ -406,14 +403,9 @@ 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.DNS_SQL))
- ? LordnPhase.NONE
- : hasSignedMarks
- ? LordnPhase.SUNRISE
- : hasClaimsNotice ? LordnPhase.CLAIMS : LordnPhase.NONE);
+ hasSignedMarks
+ ? LordnPhase.SUNRISE
+ : hasClaimsNotice ? LordnPhase.CLAIMS : LordnPhase.NONE);
Domain domain = domainBuilder.build();
if (allocationToken.isPresent()
&& allocationToken.get().getTokenType().equals(TokenType.PACKAGE)) {
@@ -436,8 +428,9 @@ public final class DomainCreateFlow implements TransactionalFlow {
allocationTokenFlowUtils.redeemToken(
allocationToken.get(), domainHistory.getHistoryEntryId()));
}
- enqueueTasks(domain, hasSignedMarks, hasClaimsNotice);
-
+ if (domain.shouldPublishToDns()) {
+ dnsUtils.requestDomainDnsRefresh(domain.getDomainName());
+ }
EntityChanges entityChanges =
flowCustomLogic.beforeSave(
DomainCreateFlowCustomLogic.BeforeSaveParameters.newBuilder()
@@ -710,19 +703,6 @@ public final class DomainCreateFlow implements TransactionalFlow {
.build();
}
- private void enqueueTasks(Domain newDomain, boolean hasSignedMarks, boolean hasClaimsNotice) {
- if (newDomain.shouldPublishToDns()) {
- dnsUtils.requestDomainDnsRefresh(newDomain.getDomainName());
- }
- if (!(DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime())
- .equals(MigrationState.NORDN_SQL)
- || DatabaseMigrationStateSchedule.getValueAtTime(tm().getTransactionTime())
- .equals(MigrationState.DNS_SQL))
- && (hasClaimsNotice || hasSignedMarks)) {
- LordnTaskUtils.enqueueDomainTask(newDomain);
- }
- }
-
/**
* Determines the {@link RenewalPriceBehavior} and the renewal price that needs be stored in the
* {@link Recurring} billing events.
diff --git a/core/src/main/java/google/registry/tmch/LordnTaskUtils.java b/core/src/main/java/google/registry/tmch/LordnTaskUtils.java
index 2c43b3d00..b315b1def 100644
--- a/core/src/main/java/google/registry/tmch/LordnTaskUtils.java
+++ b/core/src/main/java/google/registry/tmch/LordnTaskUtils.java
@@ -14,21 +14,15 @@
package google.registry.tmch;
-import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
import static com.google.common.base.Preconditions.checkState;
-import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
-import com.google.appengine.api.taskqueue.TaskOptions;
-import com.google.appengine.api.taskqueue.TaskOptions.Method;
import com.google.common.base.Joiner;
import google.registry.model.domain.Domain;
import google.registry.model.registrar.Registrar;
import java.util.Optional;
-import org.joda.time.DateTime;
/**
- * Helper methods for creating tasks containing CSV line data in the lordn-sunrise and lordn-claims
- * queues based on {@link Domain} changes.
+ * Helper methods for creating tasks containing CSV line data based on {@link Domain#getLordnPhase}.
*
* Note that, per the
* TMCH RFC, while the application-datetime data is optional (which we never send because there
@@ -36,65 +30,32 @@ import org.joda.time.DateTime;
*/
public final class LordnTaskUtils {
- public static final String QUEUE_SUNRISE = "lordn-sunrise";
- public static final String QUEUE_CLAIMS = "lordn-claims";
public static final String COLUMNS_CLAIMS =
"roid,domain-name,notice-id,registrar-id,"
+ "registration-datetime,ack-datetime,application-datetime";
public static final String COLUMNS_SUNRISE =
"roid,domain-name,SMD-id,registrar-id," + "registration-datetime,application-datetime";
- /** Enqueues a task in the LORDN queue representing a line of CSV for LORDN export. */
- public static void enqueueDomainTask(Domain domain) {
- tm().assertInTransaction();
- // This method needs to use transactionTime as the Domain's creationTime because CreationTime
- // isn't yet populated when this method is called during the resource flow.
- String tld = domain.getTld();
- if (domain.getLaunchNotice() == null) {
- getQueue(QUEUE_SUNRISE)
- .add(
- TaskOptions.Builder.withTag(tld)
- .method(Method.PULL)
- .payload(getCsvLineForSunriseDomain(domain, tm().getTransactionTime())));
- } else {
- getQueue(QUEUE_CLAIMS)
- .add(
- TaskOptions.Builder.withTag(tld)
- .method(Method.PULL)
- .payload(getCsvLineForClaimsDomain(domain, tm().getTransactionTime())));
- }
- }
-
/** Returns the corresponding CSV LORDN line for a sunrise domain. */
public static String getCsvLineForSunriseDomain(Domain domain) {
- return getCsvLineForSunriseDomain(domain, domain.getCreationTime());
- }
-
- // TODO: Merge into the function above after pull queue migration.
- private static String getCsvLineForSunriseDomain(Domain domain, DateTime transactionTime) {
return Joiner.on(',')
.join(
domain.getRepoId(),
domain.getDomainName(),
domain.getSmdId(),
getIanaIdentifier(domain.getCreationRegistrarId()),
- transactionTime); // Used as creation time.
+ domain.getCreationTime()); // Used as creation time.
}
/** Returns the corresponding CSV LORDN line for a claims domain. */
public static String getCsvLineForClaimsDomain(Domain domain) {
- return getCsvLineForClaimsDomain(domain, domain.getCreationTime());
- }
-
- // TODO: Merge into the function above after pull queue migration.
- private static String getCsvLineForClaimsDomain(Domain domain, DateTime transactionTime) {
return Joiner.on(',')
.join(
domain.getRepoId(),
domain.getDomainName(),
domain.getLaunchNotice().getNoticeId().getTcnId(),
getIanaIdentifier(domain.getCreationRegistrarId()),
- transactionTime, // Used as creation time.
+ domain.getCreationTime(), // Used as creation time.
domain.getLaunchNotice().getAcceptedTime());
}
diff --git a/core/src/main/java/google/registry/tmch/NordnUploadAction.java b/core/src/main/java/google/registry/tmch/NordnUploadAction.java
index f550e3bf3..a9374f4c4 100644
--- a/core/src/main/java/google/registry/tmch/NordnUploadAction.java
+++ b/core/src/main/java/google/registry/tmch/NordnUploadAction.java
@@ -14,7 +14,6 @@
package google.registry.tmch;
-import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.net.HttpHeaders.LOCATION;
@@ -43,7 +42,6 @@ import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMultimap;
import com.google.common.collect.ImmutableSortedSet;
-import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.flogger.FluentLogger;
import google.registry.batch.CloudTasksUtils;
@@ -66,7 +64,6 @@ import java.net.URL;
import java.security.GeneralSecurityException;
import java.security.SecureRandom;
import java.util.List;
-import java.util.Optional;
import java.util.Random;
import java.util.concurrent.TimeUnit;
import javax.inject.Inject;
@@ -90,8 +87,6 @@ public final class NordnUploadAction implements Runnable {
static final String PATH = "/_dr/task/nordnUpload";
static final String LORDN_PHASE_PARAM = "lordnPhase";
- // TODO: Delete after migrating off of pull queue.
- static final String PULL_QUEUE_PARAM = "pullQueue";
private static final int BATCH_SIZE = 1000;
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
@@ -122,10 +117,6 @@ public final class NordnUploadAction implements Runnable {
@Parameter(RequestParameters.PARAM_TLD)
String tld;
- @Inject
- @Parameter(PULL_QUEUE_PARAM)
- Optional usePullQueue;
-
@Inject CloudTasksUtils cloudTasksUtils;
@Inject
@@ -146,67 +137,59 @@ public final class NordnUploadAction implements Runnable {
@Override
public void run() {
- if (usePullQueue.orElse(false)) {
- try {
- processLordnTasks();
- } catch (IOException | GeneralSecurityException e) {
- throw new RuntimeException(e);
- }
- } else {
- checkArgument(
- phase.equals(PARAM_LORDN_PHASE_SUNRISE) || phase.equals(PARAM_LORDN_PHASE_CLAIMS),
- "Invalid phase specified to NordnUploadAction: %s.",
- phase);
- tm().transact(
- () -> {
- // Note here that we load all domains pending Nordn in one batch, which should not
- // be a problem for the rate of domain registration that we see. If we anticipate
- // a peak in claims during TLD launch (sunrise is NOT first-come-first-serve, so
- // there should be no expectation of a peak during it), we can consider temporarily
- // increasing the frequency of Nordn upload to reduce the size of each batch.
- //
- // We did not further divide the domains into smaller batches because the
- // read-upload-write operation per small batch needs to be inside a single
- // transaction to prevent race conditions, and running several uploads in rapid
- // sucession will likely overwhelm the MarksDB upload server, which recommands a
- // maximum upload frequency of every 3 hours.
- //
- // See:
- // https://datatracker.ietf.org/doc/html/draft-ietf-regext-tmch-func-spec-01#section-5.2.3.3
- List domains =
- tm().createQueryComposer(Domain.class)
- .where("lordnPhase", EQ, LordnPhase.valueOf(Ascii.toUpperCase(phase)))
- .where("tld", EQ, tld)
- .orderBy("creationTime")
- .list();
- if (domains.isEmpty()) {
- return;
- }
- StringBuilder csv = new StringBuilder();
- ImmutableList.Builder newDomains = new ImmutableList.Builder<>();
+ checkArgument(
+ phase.equals(PARAM_LORDN_PHASE_SUNRISE) || phase.equals(PARAM_LORDN_PHASE_CLAIMS),
+ "Invalid phase specified to NordnUploadAction: %s.",
+ phase);
+ tm().transact(
+ () -> {
+ // Note here that we load all domains pending Nordn in one batch, which should not
+ // be a problem for the rate of domain registration that we see. If we anticipate
+ // a peak in claims during TLD launch (sunrise is NOT first-come-first-serve, so
+ // there should be no expectation of a peak during it), we can consider temporarily
+ // increasing the frequency of Nordn upload to reduce the size of each batch.
+ //
+ // We did not further divide the domains into smaller batches because the
+ // read-upload-write operation per small batch needs to be inside a single
+ // transaction to prevent race conditions, and running several uploads in rapid
+ // sucession will likely overwhelm the MarksDB upload server, which recommands a
+ // maximum upload frequency of every 3 hours.
+ //
+ // See:
+ // https://datatracker.ietf.org/doc/html/draft-ietf-regext-tmch-func-spec-01#section-5.2.3.3
+ List domains =
+ tm().createQueryComposer(Domain.class)
+ .where("lordnPhase", EQ, LordnPhase.valueOf(Ascii.toUpperCase(phase)))
+ .where("tld", EQ, tld)
+ .orderBy("creationTime")
+ .list();
+ if (domains.isEmpty()) {
+ return;
+ }
+ StringBuilder csv = new StringBuilder();
+ ImmutableList.Builder newDomains = new ImmutableList.Builder<>();
- domains.forEach(
- domain -> {
- if (phase.equals(PARAM_LORDN_PHASE_SUNRISE)) {
- csv.append(getCsvLineForSunriseDomain(domain)).append('\n');
- } else {
- csv.append(getCsvLineForClaimsDomain(domain)).append('\n');
- }
- Domain newDomain = domain.asBuilder().setLordnPhase(LordnPhase.NONE).build();
- newDomains.add(newDomain);
- });
- String columns =
- phase.equals(PARAM_LORDN_PHASE_SUNRISE) ? COLUMNS_SUNRISE : COLUMNS_CLAIMS;
- String header =
- String.format("1,%s,%d\n%s\n", clock.nowUtc(), domains.size(), columns);
- try {
- uploadCsvToLordn(String.format("/LORDN/%s/%s", tld, phase), header + csv);
- } catch (IOException | GeneralSecurityException e) {
- throw new RuntimeException(e);
- }
- tm().updateAll(newDomains.build());
- });
- }
+ domains.forEach(
+ domain -> {
+ if (phase.equals(PARAM_LORDN_PHASE_SUNRISE)) {
+ csv.append(getCsvLineForSunriseDomain(domain)).append('\n');
+ } else {
+ csv.append(getCsvLineForClaimsDomain(domain)).append('\n');
+ }
+ Domain newDomain = domain.asBuilder().setLordnPhase(LordnPhase.NONE).build();
+ newDomains.add(newDomain);
+ });
+ String columns =
+ phase.equals(PARAM_LORDN_PHASE_SUNRISE) ? COLUMNS_SUNRISE : COLUMNS_CLAIMS;
+ String header =
+ String.format("1,%s,%d\n%s\n", clock.nowUtc(), domains.size(), columns);
+ try {
+ uploadCsvToLordn(String.format("/LORDN/%s/%s", tld, phase), header + csv);
+ } catch (IOException | GeneralSecurityException e) {
+ throw new RuntimeException(e);
+ }
+ tm().updateAll(newDomains.build());
+ });
}
/**
@@ -249,35 +232,6 @@ public final class NordnUploadAction implements Runnable {
}
}
- private void processLordnTasks() throws IOException, GeneralSecurityException {
- checkArgument(
- phase.equals(PARAM_LORDN_PHASE_SUNRISE) || phase.equals(PARAM_LORDN_PHASE_CLAIMS),
- "Invalid phase specified to NordnUploadAction: %s.",
- phase);
- DateTime now = clock.nowUtc();
- Queue queue =
- getQueue(
- phase.equals(PARAM_LORDN_PHASE_SUNRISE)
- ? LordnTaskUtils.QUEUE_SUNRISE
- : LordnTaskUtils.QUEUE_CLAIMS);
- String columns = phase.equals(PARAM_LORDN_PHASE_SUNRISE) ? COLUMNS_SUNRISE : COLUMNS_CLAIMS;
- List tasks = loadAllTasks(queue, tld);
- // Note: This upload/task deletion isn't done atomically (it's not clear how one would do so
- // anyway). As a result, it is possible that the upload might succeed yet the deletion of
- // enqueued tasks might fail. If so, this would result in the same lines being uploaded to NORDN
- // across multiple uploads. This is probably OK; all that we really cannot have is a missing
- // line.
- if (!tasks.isEmpty()) {
- String csvData = convertTasksToCsv(tasks, now, columns);
- uploadCsvToLordn(String.format("/LORDN/%s/%s", tld, phase), csvData);
- Lists.partition(tasks, BATCH_SIZE)
- .forEach(
- batch ->
- retrier.callWithRetry(
- () -> queue.deleteTask(batch), TransientFailureException.class));
- }
- }
-
/**
* Upload LORDN file to MarksDB.
*
diff --git a/core/src/main/java/google/registry/tmch/TmchModule.java b/core/src/main/java/google/registry/tmch/TmchModule.java
index 9769663ff..ba2c106b9 100644
--- a/core/src/main/java/google/registry/tmch/TmchModule.java
+++ b/core/src/main/java/google/registry/tmch/TmchModule.java
@@ -16,7 +16,6 @@ package google.registry.tmch;
import static com.google.common.io.Resources.asByteSource;
import static com.google.common.io.Resources.getResource;
-import static google.registry.request.RequestParameters.extractOptionalBooleanParameter;
import static google.registry.request.RequestParameters.extractRequiredParameter;
import dagger.Module;
@@ -26,7 +25,6 @@ import google.registry.request.HttpException.BadRequestException;
import google.registry.request.Parameter;
import java.net.MalformedURLException;
import java.net.URL;
-import java.util.Optional;
import javax.servlet.http.HttpServletRequest;
import org.bouncycastle.openpgp.PGPPublicKey;
@@ -66,10 +64,4 @@ public final class TmchModule {
static String provideNordnLogId(HttpServletRequest req) {
return extractRequiredParameter(req, NordnVerifyAction.NORDN_LOG_ID_PARAM);
}
-
- @Provides
- @Parameter(NordnUploadAction.PULL_QUEUE_PARAM)
- static Optional provideUsePullQueue(HttpServletRequest req) {
- return extractOptionalBooleanParameter(req, NordnUploadAction.PULL_QUEUE_PARAM);
- }
}
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 f27fa2493..4c5e5fee9 100644
--- a/core/src/test/java/google/registry/flows/domain/DomainCreateFlowTest.java
+++ b/core/src/test/java/google/registry/flows/domain/DomainCreateFlowTest.java
@@ -54,10 +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.assertNoTasksEnqueued;
-import static google.registry.testing.TaskQueueHelper.assertTasksEnqueued;
-import static google.registry.tmch.LordnTaskUtils.QUEUE_CLAIMS;
-import static google.registry.tmch.LordnTaskUtils.QUEUE_SUNRISE;
import static google.registry.util.DateTimeUtils.END_OF_TIME;
import static google.registry.util.DateTimeUtils.START_OF_TIME;
import static org.joda.money.CurrencyUnit.JPY;
@@ -153,8 +149,6 @@ import google.registry.model.billing.BillingEvent;
import google.registry.model.billing.BillingEvent.Flag;
import google.registry.model.billing.BillingEvent.Reason;
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
-import google.registry.model.common.DatabaseMigrationStateSchedule;
-import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.GracePeriod;
@@ -181,7 +175,6 @@ import google.registry.monitoring.whitebox.EppMetric;
import google.registry.persistence.VKey;
import google.registry.testing.DatabaseHelper;
import google.registry.testing.TaskQueueExtension;
-import google.registry.testing.TaskQueueHelper.TaskMatcher;
import google.registry.tmch.LordnTaskUtils.LordnPhase;
import google.registry.tmch.SmdrlCsvParser;
import google.registry.tmch.TmchData;
@@ -193,12 +186,9 @@ import javax.annotation.Nullable;
import org.joda.money.Money;
import org.joda.time.DateTime;
import org.joda.time.Duration;
-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;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
import org.junitpioneer.jupiter.cartesian.CartesianTest;
import org.junitpioneer.jupiter.cartesian.CartesianTest.Values;
@@ -228,11 +218,6 @@ class DomainCreateFlowTest extends ResourceFlowTestCase
- DatabaseMigrationStateSchedule.set(
- new ImmutableSortedMap.Builder(Ordering.natural())
- .put(START_OF_TIME, MigrationState.DATASTORE_ONLY)
- .put(START_OF_TIME.plusMillis(1), MigrationState.DATASTORE_PRIMARY)
- .build()));
- 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)
- .build()));
- 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)
- .build()));
- 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)
- .build()));
- 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)
- .build()));
- 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)
- .build()));
- 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)
- .build()));
- 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)
- .build()));
- }
}
diff --git a/core/src/test/java/google/registry/testing/DatabaseHelper.java b/core/src/test/java/google/registry/testing/DatabaseHelper.java
index df498f758..d4ebf1af6 100644
--- a/core/src/test/java/google/registry/testing/DatabaseHelper.java
+++ b/core/src/test/java/google/registry/testing/DatabaseHelper.java
@@ -104,7 +104,6 @@ import google.registry.model.transfer.DomainTransferData;
import google.registry.model.transfer.TransferData;
import google.registry.model.transfer.TransferStatus;
import google.registry.persistence.VKey;
-import google.registry.tmch.LordnTaskUtils;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
@@ -312,17 +311,6 @@ public final class DatabaseHelper {
return persistResource(domain.asBuilder().setDeletionTime(deletionTime).build());
}
- // TODO: delete after pull queue migration.
- /** Persists a domain and enqueues a LORDN task of the appropriate type for it. */
- public static Domain persistDomainAndEnqueueLordn(final Domain domain) {
- final Domain persistedDomain = persistResource(domain);
- // Calls {@link LordnTaskUtils#enqueueDomainTask} wrapped in a transaction so that the
- // transaction time is set correctly.
- tm().transact(() -> LordnTaskUtils.enqueueDomainTask(persistedDomain));
- maybeAdvanceClock();
- return persistedDomain;
- }
-
/** Persists a {@link Recurring} and {@link HistoryEntry} for a domain that already exists. */
public static Domain persistBillingRecurrenceForDomain(
Domain domain, RenewalPriceBehavior renewalPriceBehavior, @Nullable Money renewalPrice) {
diff --git a/core/src/test/java/google/registry/tmch/LordnTaskUtilsTest.java b/core/src/test/java/google/registry/tmch/LordnTaskUtilsTest.java
deleted file mode 100644
index be59a804a..000000000
--- a/core/src/test/java/google/registry/tmch/LordnTaskUtilsTest.java
+++ /dev/null
@@ -1,111 +0,0 @@
-// Copyright 2017 The Nomulus Authors. All Rights Reserved.
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package google.registry.tmch;
-
-import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
-import static google.registry.testing.DatabaseHelper.createTld;
-import static google.registry.testing.DatabaseHelper.loadRegistrar;
-import static google.registry.testing.DatabaseHelper.persistActiveContact;
-import static google.registry.testing.DatabaseHelper.persistDomainAndEnqueueLordn;
-import static google.registry.testing.TaskQueueHelper.assertTasksEnqueued;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-import google.registry.model.domain.Domain;
-import google.registry.model.domain.launch.LaunchNotice;
-import google.registry.model.registrar.Registrar.Type;
-import google.registry.persistence.transaction.JpaTestExtensions;
-import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
-import google.registry.testing.FakeClock;
-import google.registry.testing.TaskQueueExtension;
-import google.registry.testing.TaskQueueHelper.TaskMatcher;
-import google.registry.util.Clock;
-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 LordnTaskUtils}. */
-public class LordnTaskUtilsTest {
-
- private static final Clock clock = new FakeClock(DateTime.parse("2010-05-01T10:11:12Z"));
-
- @RegisterExtension
- final JpaIntegrationTestExtension jpa =
- new JpaTestExtensions.Builder().withClock(clock).buildIntegrationTestExtension();
-
- @RegisterExtension final TaskQueueExtension taskQueue = new TaskQueueExtension();
-
- @BeforeEach
- void beforeEach() {
- createTld("example");
- }
-
- private static Domain.Builder newDomainBuilder() {
- return new Domain.Builder()
- .setDomainName("fleece.example")
- .setPersistedCurrentSponsorRegistrarId("TheRegistrar")
- .setCreationRegistrarId("TheRegistrar")
- .setRegistrant(persistActiveContact("jd1234").createVKey())
- .setSmdId("smdzzzz")
- .setCreationRegistrarId("TheRegistrar");
- }
-
- @Test
- void test_enqueueDomainTask_sunrise() {
- persistDomainAndEnqueueLordn(newDomainBuilder().setRepoId("A-EXAMPLE").build());
- String expectedPayload =
- "A-EXAMPLE,fleece.example,smdzzzz,1,2010-05-01T10:11:12.000Z";
- assertTasksEnqueued(
- "lordn-sunrise", new TaskMatcher().payload(expectedPayload).tag("example"));
- }
-
- @Test
- void test_enqueueDomainTask_claims() {
- Domain domain =
- newDomainBuilder()
- .setRepoId("11-EXAMPLE")
- .setLaunchNotice(
- LaunchNotice.create(
- "landrush1tcn", null, null, DateTime.parse("2010-05-01T09:11:12Z")))
- .build();
- persistDomainAndEnqueueLordn(domain);
- String expectedPayload = "11-EXAMPLE,fleece.example,landrush1tcn,1,2010-05-01T10:11:12.000Z,"
- + "2010-05-01T09:11:12.000Z";
- assertTasksEnqueued("lordn-claims", new TaskMatcher().payload(expectedPayload).tag("example"));
- }
-
- @Test
- void test_oteRegistrarWithNullIanaId() {
- tm().transact(
- () ->
- tm().put(
- loadRegistrar("TheRegistrar")
- .asBuilder()
- .setType(Type.OTE)
- .setIanaIdentifier(null)
- .build()));
- persistDomainAndEnqueueLordn(newDomainBuilder().setRepoId("3-EXAMPLE").build());
- String expectedPayload = "3-EXAMPLE,fleece.example,smdzzzz,null,2010-05-01T10:11:12.000Z";
- assertTasksEnqueued(
- "lordn-sunrise", new TaskMatcher().payload(expectedPayload).tag("example"));
- }
-
- @Test
- void test_enqueueDomainTask_throwsNpeOnNullDomain() {
- assertThrows(
- NullPointerException.class,
- () -> tm().transact(() -> LordnTaskUtils.enqueueDomainTask(null)));
- }
-}
diff --git a/core/src/test/java/google/registry/tmch/NordnUploadActionTest.java b/core/src/test/java/google/registry/tmch/NordnUploadActionTest.java
index f803b1e7c..52bf3bf56 100644
--- a/core/src/test/java/google/registry/tmch/NordnUploadActionTest.java
+++ b/core/src/test/java/google/registry/tmch/NordnUploadActionTest.java
@@ -14,7 +14,6 @@
package google.registry.tmch;
-import static com.google.appengine.api.taskqueue.QueueFactory.getQueue;
import static com.google.common.net.HttpHeaders.AUTHORIZATION;
import static com.google.common.net.HttpHeaders.CONTENT_TYPE;
import static com.google.common.net.HttpHeaders.LOCATION;
@@ -25,7 +24,6 @@ import static google.registry.testing.DatabaseHelper.createTld;
import static google.registry.testing.DatabaseHelper.loadByKey;
import static google.registry.testing.DatabaseHelper.loadRegistrar;
import static google.registry.testing.DatabaseHelper.newDomain;
-import static google.registry.testing.DatabaseHelper.persistDomainAndEnqueueLordn;
import static google.registry.testing.DatabaseHelper.persistResource;
import static java.nio.charset.StandardCharsets.UTF_8;
import static javax.servlet.http.HttpServletResponse.SC_ACCEPTED;
@@ -63,7 +61,6 @@ import google.registry.testing.CloudTasksHelper.TaskMatcher;
import google.registry.testing.FakeClock;
import google.registry.testing.FakeSleeper;
import google.registry.testing.FakeUrlConnectionService;
-import google.registry.testing.TaskQueueExtension;
import google.registry.tmch.LordnTaskUtils.LordnPhase;
import google.registry.util.Retrier;
import google.registry.util.UrlConnectionException;
@@ -74,14 +71,11 @@ import java.net.URL;
import java.security.SecureRandom;
import java.util.List;
import java.util.Optional;
-import java.util.concurrent.TimeUnit;
import org.joda.time.DateTime;
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.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
/** Unit tests for {@link NordnUploadAction}. */
class NordnUploadActionTest {
@@ -111,8 +105,6 @@ class NordnUploadActionTest {
final JpaIntegrationTestExtension jpa =
new JpaTestExtensions.Builder().withClock(clock).buildIntegrationTestExtension();
- @RegisterExtension final TaskQueueExtension taskQueueExtension = new TaskQueueExtension();
-
private final LordnRequestInitializer lordnRequestInitializer =
new LordnRequestInitializer(Optional.of("attack"));
private final NordnUploadAction action = new NordnUploadAction();
@@ -145,7 +137,6 @@ class NordnUploadActionTest {
action.tmchMarksdbUrl = "http://127.0.0.1";
action.random = new SecureRandom();
action.retrier = new Retrier(new FakeSleeper(clock), 3);
- action.usePullQueue = Optional.empty();
}
@Test
@@ -229,23 +220,21 @@ class NordnUploadActionTest {
cloudTasksHelper.assertNoTasksEnqueued(NordnVerifyAction.QUEUE);
}
- @ParameterizedTest
- @ValueSource(booleans = {true, false})
- void testSuccess_claimsMode(boolean usePullQueue) throws Exception {
- testRun("claims", "claims-landrush1.tld", "claims-landrush2.tld", CLAIMS_CSV, usePullQueue);
+ @Test
+ void testSuccess_claimsMode() throws Exception {
+ testRun("claims", "claims-landrush1.tld", "claims-landrush2.tld", CLAIMS_CSV);
}
- @ParameterizedTest
- @ValueSource(booleans = {true, false})
- void testSuccess_sunriseMode(boolean usePullQueue) throws Exception {
- testRun("sunrise", "sunrise1.tld", "sunrise2.tld", SUNRISE_CSV, usePullQueue);
+ @Test
+ void testSuccess_sunriseMode() throws Exception {
+ testRun("sunrise", "sunrise1.tld", "sunrise2.tld", SUNRISE_CSV);
}
@Test
void testSuccess_noResponseContent_stillWorksNormally() throws Exception {
// Returning null only affects logging.
when(httpUrlConnection.getInputStream()).thenReturn(new ByteArrayInputStream(new byte[] {}));
- testRun("claims", "claims-landrush1.tld", "claims-landrush2.tld", CLAIMS_CSV, false);
+ testRun("claims", "claims-landrush1.tld", "claims-landrush2.tld", CLAIMS_CSV);
}
@Test
@@ -268,7 +257,7 @@ class NordnUploadActionTest {
}
private void persistClaimsModeDomain() {
- persistDomainAndEnqueueLordn(
+ persistResource(
newDomain("claims-landrush2.tld")
.asBuilder()
.setCreationTimeForTest(clock.nowUtc())
@@ -278,7 +267,7 @@ class NordnUploadActionTest {
.setLordnPhase(LordnPhase.CLAIMS)
.build());
clock.advanceBy(Duration.standardDays(1));
- persistDomainAndEnqueueLordn(
+ persistResource(
newDomain("claims-landrush1.tld")
.asBuilder()
.setCreationTimeForTest(clock.nowUtc())
@@ -289,7 +278,7 @@ class NordnUploadActionTest {
}
private void persistSunriseModeDomain() {
- persistDomainAndEnqueueLordn(
+ persistResource(
newDomain("sunrise2.tld")
.asBuilder()
.setCreationTimeForTest(clock.nowUtc())
@@ -298,7 +287,7 @@ class NordnUploadActionTest {
.setLordnPhase(LordnPhase.SUNRISE)
.build());
clock.advanceBy(Duration.standardDays(1));
- persistDomainAndEnqueueLordn(
+ persistResource(
newDomain("sunrise1.tld")
.asBuilder()
.setCreationTimeForTest(clock.nowUtc())
@@ -320,10 +309,7 @@ class NordnUploadActionTest {
assertThat(domain.getLordnPhase()).isEqualTo(LordnPhase.NONE);
}
- private void testRun(
- String phase, String domain1, String domain2, String csv, boolean usePullQueue)
- throws Exception {
- action.usePullQueue = Optional.of(usePullQueue);
+ private void testRun(String phase, String domain1, String domain2, String csv) throws Exception {
action.phase = phase;
action.run();
verify(httpUrlConnection)
@@ -335,18 +321,8 @@ class NordnUploadActionTest {
assertThat(httpUrlConnection.getURL())
.isEqualTo(new URL("http://127.0.0.1/LORDN/tld/" + phase));
assertThat(connectionOutputStream.toString(UTF_8)).contains(csv);
- if (!usePullQueue) {
- verifyColumnCleared(domain1);
- verifyColumnCleared(domain2);
- } else {
- assertThat(
- getQueue("lordn-" + phase)
- .leaseTasks(
- LeaseOptions.Builder.withTag("tld")
- .leasePeriod(1, TimeUnit.HOURS)
- .countLimit(100)))
- .isEmpty();
- }
+ verifyColumnCleared(domain1);
+ verifyColumnCleared(domain2);
cloudTasksHelper.assertTasksEnqueued(
NordnVerifyAction.QUEUE,
new TaskMatcher()
diff --git a/core/src/test/java/google/registry/tmch/TmchTestSuite.java b/core/src/test/java/google/registry/tmch/TmchTestSuite.java
index 8d9facd3b..883eff818 100644
--- a/core/src/test/java/google/registry/tmch/TmchTestSuite.java
+++ b/core/src/test/java/google/registry/tmch/TmchTestSuite.java
@@ -21,7 +21,6 @@ import org.junit.runners.Suite.SuiteClasses;
/** Convenience class to run all TMCH tests inside IDE with one keystroke. */
@RunWith(Suite.class)
@SuiteClasses({
- LordnTaskUtilsTest.class,
NordnUploadAction.class,
NordnVerifyAction.class,
SmdrlCsvParserTest.class,