Add a beam pipeline to expand recurring billing event (#1881)

This will replace the ExpandRecurringBillingEventsAction, which has a
couple of issues:

1) The action starts with too many Recurrings that are later filtered out
   because their expanded OneTimes are not actually in scope. This is due
   to the Recurrings not recording its latest expanded event time, and
   therefore many Recurrings that are not yet due for renewal get included
   in the initial query.

2) The action works in sequence, which exacerbated the issue in 1) and
   makes it very slow to run if the window of operation is wider than
   one day, which in turn makes it impossible to run any catch-up
   expansions with any significant gap to fill.

3) The action only expands the recurrence when the billing times because
   due, but most of its logic works on event time, which is 45 days
   before billing time, making the code hard to reason about and
   error-prone.  This has led to b/258822640 where a premature
   optimization intended to fix 1) caused some autorenwals to not be
   expanded correctly when subsequent manual renews within the autorenew
   grace period closed the original recurrece.

As a result, the new pipeline addresses the above issues in the
following way:

1) Update the recurrenceLastExpansion field on the Recurring when a new
   expansion occurs, and narrow down the Recurrings in scope for
   expansion by only looking for the ones that have not been expanded for
   more than a year.

2) Make it a Beam pipeline so expansions can happen in parallel. The
   Recurrings are grouped into batches in order to not overwhelm the
   database with writes for each expansion.

3) Create new expansions when the event time, as opposed to billing
   time, is within the operation window. This streamlines the logic and
   makes it clearer and easier to reason about. This also aligns with
   how other (cancelllable) operations for which there are accompanying
   grace periods are handled, when the corresponding data is always
   speculatively created at event time. Lastly, doing this negates the
   need to check if the expansion has finished running before generating
   the monthly invoices, because the billing events are now created not
   just-in-time, but 45 days in advance.

Note that this PR only adds the pipeline. It does not switch the default
behavior to using the pipeline, which is still done by
ExpandRecurringBillingEventsAction. We will first use this pipeline to
generate missing billing events and domain histories caused by
b/258822640. This also allows us to test it in production, as it
backfills data that will not affect ongoing invoice generation. If
anything goes wrong, we can always delete the generated billing events
and domain histories, based on the unique "reason" in them.

This pipeline can only run after we switch to use SQL sequence based ID
allocation, introduced in #1831.
This commit is contained in:
Lai Jiang 2023-01-09 17:41:56 -05:00 committed by GitHub
parent 9789cf3b00
commit 2294c77306
29 changed files with 1194 additions and 27 deletions

View file

@ -12,12 +12,12 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package google.registry.beam.invoicing;
package google.registry.beam.billing;
import static com.google.common.truth.Truth.assertThat;
import google.registry.beam.invoicing.BillingEvent.InvoiceGroupingKey;
import google.registry.beam.invoicing.BillingEvent.InvoiceGroupingKey.InvoiceGroupingKeyCoder;
import google.registry.beam.billing.BillingEvent.InvoiceGroupingKey;
import google.registry.beam.billing.BillingEvent.InvoiceGroupingKey.InvoiceGroupingKeyCoder;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;

View file

@ -0,0 +1,549 @@
// Copyright 2022 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.beam.billing;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.truth.Truth.assertThat;
import static google.registry.model.EppResourceUtils.loadByForeignKey;
import static google.registry.model.ImmutableObjectSubject.immutableObjectCorrespondence;
import static google.registry.model.common.Cursor.CursorType.RECURRING_BILLING;
import static google.registry.model.domain.Period.Unit.YEARS;
import static google.registry.model.reporting.HistoryEntry.Type.DOMAIN_AUTORENEW;
import static google.registry.model.reporting.HistoryEntry.Type.DOMAIN_CREATE;
import static google.registry.model.reporting.HistoryEntryDao.loadHistoryObjectsForResource;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import static google.registry.testing.DatabaseHelper.assertBillingEventsForResource;
import static google.registry.testing.DatabaseHelper.createTld;
import static google.registry.testing.DatabaseHelper.getOnlyHistoryEntryOfType;
import static google.registry.testing.DatabaseHelper.persistActiveDomain;
import static google.registry.testing.DatabaseHelper.persistPremiumList;
import static google.registry.testing.DatabaseHelper.persistResource;
import static google.registry.util.DateTimeUtils.END_OF_TIME;
import static org.joda.money.CurrencyUnit.USD;
import static org.junit.jupiter.api.Assertions.assertThrows;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import google.registry.beam.TestPipelineExtension;
import google.registry.model.billing.BillingEvent;
import google.registry.model.billing.BillingEvent.Flag;
import google.registry.model.billing.BillingEvent.OneTime;
import google.registry.model.billing.BillingEvent.Reason;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.common.Cursor;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.Period;
import google.registry.model.reporting.DomainTransactionRecord;
import google.registry.model.reporting.DomainTransactionRecord.TransactionReportField;
import google.registry.model.tld.Registry;
import google.registry.persistence.PersistenceModule.TransactionIsolationLevel;
import google.registry.persistence.transaction.JpaTestExtensions;
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
import google.registry.testing.FakeClock;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import org.apache.beam.runners.direct.DirectOptions;
import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.hibernate.cfg.AvailableSettings;
import org.joda.money.Money;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
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 ExpandRecurringBillingEventsPipeline}. */
public class ExpandRecurringBillingEventsPipelineTest {
private static final DateTimeFormatter DATE_TIME_FORMATTER =
DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
private final FakeClock clock = new FakeClock(DateTime.parse("2021-02-02T00:00:05Z"));
private final DateTime startTime = DateTime.parse("2021-02-01TZ");
private DateTime endTime = DateTime.parse("2021-02-02TZ");
private final Cursor cursor = Cursor.createGlobal(RECURRING_BILLING, startTime);
private Domain domain;
private Recurring recurring;
private final TestOptions options = PipelineOptionsFactory.create().as(TestOptions.class);
@RegisterExtension
final JpaIntegrationTestExtension jpa =
new JpaTestExtensions.Builder()
.withClock(clock)
.withProperty(
AvailableSettings.ISOLATION,
TransactionIsolationLevel.TRANSACTION_SERIALIZABLE.name())
.buildIntegrationTestExtension();
@RegisterExtension
final TestPipelineExtension pipeline =
TestPipelineExtension.create().enableAbandonedNodeEnforcement(true);
@BeforeEach
void beforeEach() {
// Set up the pipeline.
options.setStartTime(DATE_TIME_FORMATTER.print(startTime));
options.setEndTime(DATE_TIME_FORMATTER.print(endTime));
options.setIsDryRun(false);
options.setAdvanceCursor(true);
tm().transact(() -> tm().put(cursor));
// Set up the database.
createTld("tld");
recurring = createDomainAtTime("example.tld", startTime.minusYears(1).plusHours(12));
domain = loadByForeignKey(Domain.class, "example.tld", clock.nowUtc()).get();
}
@Test
void testFailure_endTimeAfterNow() {
options.setEndTime(DATE_TIME_FORMATTER.print(clock.nowUtc().plusMillis(1)));
IllegalArgumentException thrown =
assertThrows(IllegalArgumentException.class, this::runPipeline);
assertThat(thrown)
.hasMessageThat()
.contains("End time 2021-02-02T00:00:05.001Z must be on or before now");
}
@Test
void testFailure_endTimeBeforeStartTime() {
options.setEndTime(DATE_TIME_FORMATTER.print(startTime.minusMillis(1)));
IllegalArgumentException thrown =
assertThrows(IllegalArgumentException.class, this::runPipeline);
assertThat(thrown)
.hasMessageThat()
.contains("[2021-02-01T00:00:00.000Z, 2021-01-31T23:59:59.999Z)");
}
@Test
void testSuccess_expandSingleEvent() {
runPipeline();
// Assert about DomainHistory.
assertAutoRenewDomainHistories(defaultDomainHistory());
// Assert about BillingEvents.
assertBillingEventsForResource(
domain,
defaultOneTime(getOnlyAutoRenewHistory()),
recurring
.asBuilder()
.setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1))
.build());
// Assert about Cursor.
assertCursorAt(endTime);
}
@Test
void testSuccess_expandSingleEvent_deletedDuringGracePeriod() {
domain = persistResource(domain.asBuilder().setDeletionTime(endTime.minusHours(2)).build());
recurring =
persistResource(recurring.asBuilder().setRecurrenceEndTime(endTime.minusHours(2)).build());
runPipeline();
// Assert about DomainHistory, no transaction record should have been written.
assertAutoRenewDomainHistories(
defaultDomainHistory().asBuilder().setDomainTransactionRecords(ImmutableSet.of()).build());
// Assert about BillingEvents.
assertBillingEventsForResource(
domain,
defaultOneTime(getOnlyAutoRenewHistory()),
recurring
.asBuilder()
.setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1))
.build());
// Assert about Cursor.
assertCursorAt(endTime);
}
@Test
void testFailure_expandSingleEvent_cursorNotAtStartTime() {
tm().transact(() -> tm().put(Cursor.createGlobal(RECURRING_BILLING, startTime.plusMillis(1))));
PipelineExecutionException thrown =
assertThrows(PipelineExecutionException.class, this::runPipeline);
assertThat(thrown).hasCauseThat().hasMessageThat().contains("Current cursor position");
// Assert about DomainHistory.
assertAutoRenewDomainHistories(defaultDomainHistory());
// Assert about BillingEvents.
assertBillingEventsForResource(
domain,
defaultOneTime(getOnlyAutoRenewHistory()),
recurring
.asBuilder()
.setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1))
.build());
// Assert that the cursor did not change.
assertCursorAt(startTime.plusMillis(1));
}
@Test
void testSuccess_noExpansion_recurrenceClosedBeforeEventTime() {
recurring =
persistResource(
recurring
.asBuilder()
.setRecurrenceEndTime(recurring.getEventTime().minusDays(1))
.build());
runPipeline();
assertNoExpansionsHappened();
}
@Test
void testSuccess_noExpansion_recurrenceClosedBeforeStartTime() {
recurring =
persistResource(recurring.asBuilder().setRecurrenceEndTime(startTime.minusDays(1)).build());
runPipeline();
assertNoExpansionsHappened();
}
@Test
void testSuccess_noExpansion_recurrenceClosedBeforeNextExpansion() {
recurring =
persistResource(
recurring
.asBuilder()
.setEventTime(recurring.getEventTime().minusYears(1))
.setRecurrenceEndTime(startTime.plusHours(6))
.build());
runPipeline();
assertNoExpansionsHappened();
}
@Test
void testSuccess_noExpansion_eventTimeAfterEndTime() {
recurring = persistResource(recurring.asBuilder().setEventTime(endTime.plusDays(1)).build());
runPipeline();
assertNoExpansionsHappened();
}
@Test
void testSuccess_noExpansion_LastExpansionLessThanAYearAgo() {
recurring =
persistResource(
recurring
.asBuilder()
.setRecurrenceLastExpansion(startTime.minusYears(1).plusDays(1))
.build());
runPipeline();
assertNoExpansionsHappened();
}
@Test
void testSuccess_noExpansion_oneTimeAlreadyExists() {
DomainHistory history = persistResource(defaultDomainHistory());
OneTime oneTime = persistResource(defaultOneTime(history));
runPipeline();
// Assert about DomainHistory.
assertAutoRenewDomainHistories(history);
// Assert about BillingEvents. No expansion happened, so last recurrence expansion time is
// unchanged.
assertBillingEventsForResource(domain, oneTime, recurring);
// Assert about Cursor.
assertCursorAt(endTime);
}
@Test
void testSuccess_expandSingleEvent_dryRun() {
options.setIsDryRun(true);
runPipeline();
assertNoExpansionsHappened(true);
}
@Test
void testSuccess_expandSingleEvent_doesNotAdvanceCursor() {
options.setAdvanceCursor(false);
runPipeline();
// Assert about DomainHistory.
assertAutoRenewDomainHistories(defaultDomainHistory());
// Assert about BillingEvents.
assertBillingEventsForResource(
domain,
defaultOneTime(getOnlyAutoRenewHistory()),
recurring
.asBuilder()
.setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1))
.build());
// Assert that the cursor did not move.
assertCursorAt(startTime);
}
// We control the number of threads used in the pipeline to test if the batching behavior works
// properly. When two threads are used, the two recurrings are processed in different workers and
// should be processed in parallel.
@ParameterizedTest
@ValueSource(ints = {1, 2})
void testSuccess_expandMultipleEvents_multipleDomains(int numOfThreads) {
createTld("test");
persistResource(
Registry.get("test")
.asBuilder()
.setPremiumList(persistPremiumList("premium", USD, "other,USD 100"))
.build());
DateTime otherCreateTime = startTime.minusYears(1).plusHours(5);
Recurring otherRecurring = createDomainAtTime("other.test", otherCreateTime);
Domain otherDomain = loadByForeignKey(Domain.class, "other.test", clock.nowUtc()).get();
options.setTargetParallelism(numOfThreads);
runPipeline();
// Assert about DomainHistory.
DomainHistory history = defaultDomainHistory();
DomainHistory otherHistory = defaultDomainHistory(otherDomain);
assertAutoRenewDomainHistories(domain, history);
assertAutoRenewDomainHistories(otherDomain, otherHistory);
// Assert about BillingEvents.
assertBillingEventsForResource(
domain,
defaultOneTime(getOnlyAutoRenewHistory()),
recurring
.asBuilder()
.setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1))
.build());
assertBillingEventsForResource(
otherDomain,
defaultOneTime(otherDomain, getOnlyAutoRenewHistory(otherDomain), otherRecurring, 100),
otherRecurring
.asBuilder()
.setRecurrenceLastExpansion(otherDomain.getCreationTime().plusYears(1))
.build());
// Assert about Cursor.
assertCursorAt(endTime);
}
@Test
void testSuccess_expandMultipleEvents_multipleEventTime() {
clock.advanceBy(Duration.standardDays(365));
endTime = endTime.plusYears(1);
options.setEndTime(DATE_TIME_FORMATTER.print(endTime));
runPipeline();
// Assert about DomainHistory.
assertAutoRenewDomainHistories(
defaultDomainHistory(),
defaultDomainHistory()
.asBuilder()
.setDomainTransactionRecords(
ImmutableSet.of(
DomainTransactionRecord.create(
domain.getTld(),
// We report this when the autorenew grace period ends.
domain
.getCreationTime()
.plusYears(2)
.plus(Registry.DEFAULT_AUTO_RENEW_GRACE_PERIOD),
TransactionReportField.netRenewsFieldFromYears(1),
1)))
.build());
// Assert about BillingEvents.
ImmutableList<DomainHistory> histories =
loadHistoryObjectsForResource(domain.createVKey(), DomainHistory.class).stream()
.filter(domainHistory -> DOMAIN_AUTORENEW.equals(domainHistory.getType()))
.sorted(
Comparator.comparing(
h ->
h.getDomainTransactionRecords().stream()
.findFirst()
.get()
.getReportingTime()))
.collect(toImmutableList());
assertBillingEventsForResource(
domain,
defaultOneTime(histories.get(0)),
defaultOneTime(histories.get(1))
.asBuilder()
.setEventTime(domain.getCreationTime().plusYears(2))
.setBillingTime(
domain
.getCreationTime()
.plusYears(2)
.plus(Registry.DEFAULT_AUTO_RENEW_GRACE_PERIOD))
.build(),
recurring
.asBuilder()
.setRecurrenceLastExpansion(domain.getCreationTime().plusYears(2))
.build());
// Assert about Cursor.
assertCursorAt(endTime);
}
private void runPipeline() {
ExpandRecurringBillingEventsPipeline expandRecurringBillingEventsPipeline =
new ExpandRecurringBillingEventsPipeline(options, clock);
expandRecurringBillingEventsPipeline.setupPipeline(pipeline);
pipeline.run(options).waitUntilFinish();
}
void assertNoExpansionsHappened() {
assertNoExpansionsHappened(false);
}
void assertNoExpansionsHappened(boolean dryRun) {
// Only the original domain create history entry is present.
List<DomainHistory> persistedHistory =
loadHistoryObjectsForResource(domain.createVKey(), DomainHistory.class);
assertThat(persistedHistory.size()).isEqualTo(1);
assertThat(persistedHistory.get(0).getType()).isEqualTo(DOMAIN_CREATE);
// Only the original recurrence is present.
assertBillingEventsForResource(domain, recurring);
// If this is not a dry run, the cursor should still be moved even though expansions happened,
// because we still successfully processed all the needed expansions (none in this case) in the
// window. Therefore,
// the cursor should be up-to-date as of end time.
assertCursorAt(dryRun ? startTime : endTime);
}
private DomainHistory defaultDomainHistory() {
return defaultDomainHistory(domain);
}
private DomainHistory defaultDomainHistory(Domain domain) {
return new DomainHistory.Builder()
.setBySuperuser(false)
.setRegistrarId("TheRegistrar")
.setModificationTime(clock.nowUtc())
.setDomain(domain)
.setPeriod(Period.create(1, YEARS))
.setReason("Domain autorenewal by ExpandRecurringBillingEventsPipeline")
.setRequestedByRegistrar(false)
.setType(DOMAIN_AUTORENEW)
.setDomainTransactionRecords(
ImmutableSet.of(
DomainTransactionRecord.create(
domain.getTld(),
// We report this when the autorenew grace period ends.
domain
.getCreationTime()
.plusYears(1)
.plus(Registry.DEFAULT_AUTO_RENEW_GRACE_PERIOD),
TransactionReportField.netRenewsFieldFromYears(1),
1)))
.build();
}
private OneTime defaultOneTime(DomainHistory history) {
return defaultOneTime(domain, history, recurring, 11);
}
private OneTime defaultOneTime(
Domain domain, DomainHistory history, Recurring recurring, int cost) {
return new BillingEvent.OneTime.Builder()
.setBillingTime(
domain.getCreationTime().plusYears(1).plus(Registry.DEFAULT_AUTO_RENEW_GRACE_PERIOD))
.setRegistrarId("TheRegistrar")
.setCost(Money.of(USD, cost))
.setEventTime(domain.getCreationTime().plusYears(1))
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW, Flag.SYNTHETIC))
.setPeriodYears(1)
.setReason(Reason.RENEW)
.setSyntheticCreationTime(endTime)
.setCancellationMatchingBillingEvent(recurring)
.setTargetId(domain.getDomainName())
.setDomainHistory(history)
.build();
}
private void assertAutoRenewDomainHistories(DomainHistory... expected) {
assertAutoRenewDomainHistories(domain, expected);
}
private static void assertAutoRenewDomainHistories(Domain domain, DomainHistory... expected) {
ImmutableList<DomainHistory> actuals =
loadHistoryObjectsForResource(domain.createVKey(), DomainHistory.class).stream()
.filter(domainHistory -> DOMAIN_AUTORENEW.equals(domainHistory.getType()))
.collect(toImmutableList());
assertThat(actuals)
.comparingElementsUsing(immutableObjectCorrespondence("resource", "revisionId"))
.containsExactlyElementsIn(Arrays.asList(expected));
assertThat(
actuals.stream()
.map(history -> history.getDomainBase().get())
.collect(toImmutableList()))
.comparingElementsUsing(immutableObjectCorrespondence("nsHosts", "updateTimestamp"))
.containsExactlyElementsIn(
Arrays.stream(expected)
.map(history -> history.getDomainBase().get())
.collect(toImmutableList()));
}
private static DomainHistory getOnlyAutoRenewHistory(Domain domain) {
return getOnlyHistoryEntryOfType(domain, DOMAIN_AUTORENEW, DomainHistory.class);
}
private DomainHistory getOnlyAutoRenewHistory() {
return getOnlyAutoRenewHistory(domain);
}
private static void assertCursorAt(DateTime expectedCursorTime) {
Cursor cursor = tm().transact(() -> tm().loadByKey(Cursor.createGlobalVKey(RECURRING_BILLING)));
assertThat(cursor).isNotNull();
assertThat(cursor.getCursorTime()).isEqualTo(expectedCursorTime);
}
private static Recurring createDomainAtTime(String domainName, DateTime createTime) {
Domain domain = persistActiveDomain(domainName, createTime);
DomainHistory domainHistory =
persistResource(
new DomainHistory.Builder()
.setRegistrarId(domain.getCreationRegistrarId())
.setType(DOMAIN_CREATE)
.setModificationTime(createTime)
.setDomain(domain)
.build());
return persistResource(
new Recurring.Builder()
.setDomainHistory(domainHistory)
.setRegistrarId(domain.getCreationRegistrarId())
.setEventTime(createTime.plusYears(1))
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setReason(Reason.RENEW)
.setRecurrenceEndTime(END_OF_TIME)
.setTargetId(domain.getDomainName())
.build());
}
public interface TestOptions extends ExpandRecurringBillingEventsPipelineOptions, DirectOptions {}
}

View file

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package google.registry.beam.invoicing;
package google.registry.beam.billing;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static com.google.common.truth.Truth.assertThat;

View file

@ -36,12 +36,14 @@ import google.registry.model.contact.Contact;
import google.registry.model.domain.Domain;
import google.registry.model.domain.GracePeriod;
import google.registry.model.eppcommon.StatusValue;
import google.registry.persistence.PersistenceModule.TransactionIsolationLevel;
import google.registry.persistence.transaction.JpaTestExtensions;
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
import google.registry.persistence.transaction.JpaTransactionManager;
import google.registry.persistence.transaction.TransactionManagerFactory;
import google.registry.testing.FakeClock;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.hibernate.cfg.Environment;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.junit.jupiter.api.BeforeEach;
@ -60,7 +62,11 @@ public class ResaveAllEppResourcesPipelineTest {
@RegisterExtension
final JpaIntegrationTestExtension database =
new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationTestExtension();
new JpaTestExtensions.Builder()
.withClock(fakeClock)
.withProperty(
Environment.ISOLATION, TransactionIsolationLevel.TRANSACTION_REPEATABLE_READ.name())
.buildIntegrationTestExtension();
private final ResaveAllEppResourcesPipelineOptions options =
PipelineOptionsFactory.create().as(ResaveAllEppResourcesPipelineOptions.class);

View file

@ -302,6 +302,8 @@ class DomainTransferApproveFlowTest
getGainingClientAutorenewEvent()
.asBuilder()
.setEventTime(domain.getRegistrationExpirationTime())
.setRecurrenceLastExpansion(
domain.getRegistrationExpirationTime().minusYears(1))
.setDomainHistory(historyEntryTransferApproved)
.build()))
.toArray(BillingEvent[]::new));
@ -338,6 +340,8 @@ class DomainTransferApproveFlowTest
getGainingClientAutorenewEvent()
.asBuilder()
.setEventTime(domain.getRegistrationExpirationTime())
.setRecurrenceLastExpansion(
domain.getRegistrationExpirationTime().minusYears(1))
.setDomainHistory(historyEntryTransferApproved)
.build()))
.toArray(BillingEvent[]::new));
@ -835,7 +839,7 @@ class DomainTransferApproveFlowTest
"tld",
"domain_transfer_approve.xml",
"domain_transfer_approve_response_zero_period.xml",
domain.getRegistrationExpirationTime().plusYears(0));
domain.getRegistrationExpirationTime());
assertHistoryEntriesDoNotContainTransferBillingEventsOrGracePeriods();
}

View file

@ -296,7 +296,11 @@ class DomainTransferRequestFlowTest
.setRecurrenceEndTime(implicitTransferTime)
.build();
BillingEvent.Recurring gainingClientAutorenew =
getGainingClientAutorenewEvent().asBuilder().setEventTime(expectedExpirationTime).build();
getGainingClientAutorenewEvent()
.asBuilder()
.setEventTime(expectedExpirationTime)
.setRecurrenceLastExpansion(expectedExpirationTime.minusYears(1))
.build();
// Construct extra billing events expected by the specific test.
ImmutableSet<BillingEvent> extraBillingEvents =
Stream.of(extraExpectedBillingEvents)

View file

@ -774,6 +774,7 @@ public class BillingEventTest extends EntityTestCase {
.setRecurrenceEndTime(END_OF_TIME)));
assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.SPECIFIED);
assertThat(recurringEvent.getRenewalPrice()).hasValue(Money.of(USD, 100));
assertThat(recurringEvent.getRecurrenceLastExpansion()).isEqualTo(now);
}
@Test

View file

@ -147,7 +147,7 @@ public class JpaTestExtensions {
}
/** Adds the specified property to those used to initialize the transaction manager. */
Builder withProperty(String name, String value) {
public Builder withProperty(String name, String value) {
this.userProperties.put(name, value);
return this;
}