From 668a7a47a2dc34cff7e6623817c9ac97927e7b2c Mon Sep 17 00:00:00 2001 From: gbrodman Date: Fri, 28 Apr 2023 14:27:37 -0400 Subject: [PATCH] Refactor / rename Billing object classes (#1993) This includes renaming the billing classes to match the SQL table names, as well as splitting them out into their own separate top-level classes. The rest of the changes are mostly renaming variables and comments etc. We now use `BillingBase` as the name of the common billing superclass, because one-time events are called BillingEvents --- core/build.gradle | 4 +- .../google/registry/batch/BatchModule.java | 13 +- ...va => ExpandBillingRecurrencesAction.java} | 22 +- .../registry/beam/billing/BillingEvent.java | 2 +- ... => ExpandBillingRecurrencesPipeline.java} | 152 ++-- ...andBillingRecurrencesPipelineOptions.java} | 2 +- .../beam/billing/InvoicingPipeline.java | 70 +- .../default/WEB-INF/cloud-scheduler-tasks.xml | 11 +- .../env/common/backend/WEB-INF/web.xml | 4 +- .../default/WEB-INF/cloud-scheduler-tasks.xml | 13 +- .../default/WEB-INF/cloud-scheduler-tasks.xml | 11 +- .../flows/domain/DomainCheckFlow.java | 18 +- .../flows/domain/DomainCreateFlow.java | 32 +- .../flows/domain/DomainDeleteFlow.java | 36 +- .../flows/domain/DomainFlowUtils.java | 32 +- .../flows/domain/DomainPricingLogic.java | 20 +- .../flows/domain/DomainRenewFlow.java | 25 +- .../domain/DomainRestoreRequestFlow.java | 16 +- .../domain/DomainTransferApproveFlow.java | 41 +- .../domain/DomainTransferCancelFlow.java | 7 +- .../domain/DomainTransferRejectFlow.java | 7 +- .../domain/DomainTransferRequestFlow.java | 11 +- .../flows/domain/DomainTransferUtils.java | 42 +- .../flows/domain/DomainUpdateFlow.java | 8 +- .../token/AllocationTokenFlowUtils.java | 14 +- .../registry/model/billing/BillingBase.java | 264 ++++++ .../model/billing/BillingCancellation.java | 166 ++++ .../registry/model/billing/BillingEvent.java | 801 +++--------------- .../model/billing/BillingRecurrence.java | 199 +++++ .../registry/model/domain/DomainBase.java | 10 +- .../registry/model/domain/GracePeriod.java | 50 +- .../model/domain/GracePeriodBase.java | 25 +- .../model/domain/token/AllocationToken.java | 2 +- .../model/reporting/HistoryEntry.java | 3 +- .../model/transfer/DomainTransferData.java | 21 +- .../backend/BackendRequestComponent.java | 4 +- .../BillingEventFlagSetConverter.java | 2 +- .../registry/tools/DomainLockUtils.java | 8 +- .../GenerateAllocationTokensCommand.java | 4 +- .../google/registry/tools/RegistryTool.java | 6 +- .../registry/tools/RegistryToolComponent.java | 3 - .../registry/tools/UnrenewDomainCommand.java | 9 +- .../tools/UpdateAllocationTokensCommand.java | 2 +- .../tools/UpdateRecurrenceCommand.java | 64 +- ...CancellationsForBillingEventsCommand.java} | 73 +- .../main/resources/META-INF/persistence.xml | 12 +- .../billing/sql/cloud_sql_billing_events.sql | 4 +- ...illing_recurrences_pipeline_metadata.json} | 0 .../CheckPackagesComplianceActionTest.java | 2 +- .../batch/DeleteExpiredDomainsActionTest.java | 13 +- .../batch/DeleteProberDataActionTest.java | 6 +- ...> ExpandBillingRecurrencesActionTest.java} | 17 +- ...ExpandBillingRecurrencesPipelineTest.java} | 95 ++- .../beam/billing/InvoicingPipelineTest.java | 133 +-- .../flows/EppLifecycleDomainTest.java | 44 +- .../google/registry/flows/EppTestCase.java | 57 +- .../google/registry/flows/FlowTestCase.java | 17 +- .../flows/domain/DomainCheckFlowTest.java | 16 +- .../flows/domain/DomainCreateFlowTest.java | 74 +- .../flows/domain/DomainDeleteFlowTest.java | 38 +- .../flows/domain/DomainInfoFlowTest.java | 24 +- .../flows/domain/DomainPricingLogicTest.java | 108 +-- .../flows/domain/DomainRenewFlowTest.java | 59 +- .../domain/DomainRestoreRequestFlowTest.java | 20 +- .../domain/DomainTransferApproveFlowTest.java | 37 +- .../domain/DomainTransferFlowTestCase.java | 15 +- .../domain/DomainTransferRequestFlowTest.java | 65 +- .../flows/domain/DomainUpdateFlowTest.java | 4 +- ...ingEventTest.java => BillingBaseTest.java} | 352 ++++---- .../registry/model/domain/DomainSqlTest.java | 2 +- .../registry/model/domain/DomainTest.java | 73 +- .../model/domain/GracePeriodTest.java | 36 +- .../domain/token/AllocationTokenTest.java | 2 +- .../domain/token/PackagePromotionTest.java | 2 +- .../model/transfer/TransferDataTest.java | 21 +- .../rde/DomainToXjcConverterTest.java | 15 +- .../java/google/registry/rde/RdeFixtures.java | 15 +- .../integration/SqlIntegrationTestSuite.java | 4 +- .../registry/testing/DatabaseHelper.java | 71 +- .../CreatePackagePromotionCommandTest.java | 2 +- .../registry/tools/DomainLockUtilsTest.java | 7 +- .../registry/tools/EppLifecycleToolsTest.java | 12 +- .../GenerateAllocationTokensCommandTest.java | 4 +- .../tools/GetPackagePromotionCommandTest.java | 2 +- .../tools/UnrenewDomainCommandTest.java | 8 +- .../UpdateAllocationTokensCommandTest.java | 6 +- ...> UpdateBillingRecurrenceCommandTest.java} | 72 +- .../tools/UpdateDomainCommandTest.java | 12 +- .../UpdatePackagePromotionCommandTest.java | 2 +- ...ellationsForBillingEventsCommandTest.java} | 81 +- .../RegistryLockVerifyActionTest.java | 4 +- .../module/backend/backend_routing.txt | 2 +- .../sql/er_diagram/brief_er_diagram.html | 4 +- .../sql/er_diagram/full_er_diagram.html | 4 +- .../sql/schema/db-schema.sql.generated | 2 +- docs/code-structure.md | 22 +- release/cloudbuild-nomulus.yaml | 4 +- 97 files changed, 2086 insertions(+), 1946 deletions(-) rename core/src/main/java/google/registry/batch/{ExpandRecurringBillingEventsAction.java => ExpandBillingRecurrencesAction.java} (87%) rename core/src/main/java/google/registry/beam/billing/{ExpandRecurringBillingEventsPipeline.java => ExpandBillingRecurrencesPipeline.java} (80%) rename core/src/main/java/google/registry/beam/billing/{ExpandRecurringBillingEventsPipelineOptions.java => ExpandBillingRecurrencesPipelineOptions.java} (94%) create mode 100644 core/src/main/java/google/registry/model/billing/BillingBase.java create mode 100644 core/src/main/java/google/registry/model/billing/BillingCancellation.java create mode 100644 core/src/main/java/google/registry/model/billing/BillingRecurrence.java rename core/src/main/java/google/registry/tools/javascrap/{CreateCancellationsForOneTimesCommand.java => CreateCancellationsForBillingEventsCommand.java} (52%) rename core/src/main/resources/google/registry/beam/{expand_recurring_billing_events_pipeline_metadata.json => expand_billing_recurrences_pipeline_metadata.json} (100%) rename core/src/test/java/google/registry/batch/{ExpandRecurringBillingEventsActionTest.java => ExpandBillingRecurrencesActionTest.java} (91%) rename core/src/test/java/google/registry/beam/billing/{ExpandRecurringBillingEventsPipelineTest.java => ExpandBillingRecurrencesPipelineTest.java} (87%) rename core/src/test/java/google/registry/model/billing/{BillingEventTest.java => BillingBaseTest.java} (72%) rename core/src/test/java/google/registry/tools/{UpdateRecurrenceCommandTest.java => UpdateBillingRecurrenceCommandTest.java} (72%) rename core/src/test/java/google/registry/tools/javascrap/{CreateCancellationsForOneTimesCommandTest.java => CreateCancellationsForBillingEventsCommandTest.java} (56%) diff --git a/core/build.gradle b/core/build.gradle index d13fd4b44..777d1cebc 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -749,8 +749,8 @@ if (environment == 'alpha') { ], expandBilling : [ - mainClass: 'google.registry.beam.billing.ExpandRecurringBillingEventsPipeline', - metaData : 'google/registry/beam/expand_recurring_billing_events_pipeline_metadata.json' + mainClass: 'google.registry.beam.billing.ExpandBillingRecurrencesPipeline', + metaData : 'google/registry/beam/expand_billing_recurrences_pipeline_metadata.json' ], rde : [ diff --git a/core/src/main/java/google/registry/batch/BatchModule.java b/core/src/main/java/google/registry/batch/BatchModule.java index b7471b178..95b64a88a 100644 --- a/core/src/main/java/google/registry/batch/BatchModule.java +++ b/core/src/main/java/google/registry/batch/BatchModule.java @@ -105,16 +105,15 @@ public class BatchModule { } @Provides - @Parameter(ExpandRecurringBillingEventsAction.PARAM_START_TIME) + @Parameter(ExpandBillingRecurrencesAction.PARAM_START_TIME) static Optional provideStartTime(HttpServletRequest req) { - return extractOptionalDatetimeParameter( - req, ExpandRecurringBillingEventsAction.PARAM_START_TIME); + return extractOptionalDatetimeParameter(req, ExpandBillingRecurrencesAction.PARAM_START_TIME); } @Provides - @Parameter(ExpandRecurringBillingEventsAction.PARAM_END_TIME) + @Parameter(ExpandBillingRecurrencesAction.PARAM_END_TIME) static Optional provideEndTime(HttpServletRequest req) { - return extractOptionalDatetimeParameter(req, ExpandRecurringBillingEventsAction.PARAM_END_TIME); + return extractOptionalDatetimeParameter(req, ExpandBillingRecurrencesAction.PARAM_END_TIME); } @Provides @@ -124,9 +123,9 @@ public class BatchModule { } @Provides - @Parameter(ExpandRecurringBillingEventsAction.PARAM_ADVANCE_CURSOR) + @Parameter(ExpandBillingRecurrencesAction.PARAM_ADVANCE_CURSOR) static boolean provideAdvanceCursor(HttpServletRequest req) { - return extractBooleanParameter(req, ExpandRecurringBillingEventsAction.PARAM_ADVANCE_CURSOR); + return extractBooleanParameter(req, ExpandBillingRecurrencesAction.PARAM_ADVANCE_CURSOR); } @Provides diff --git a/core/src/main/java/google/registry/batch/ExpandRecurringBillingEventsAction.java b/core/src/main/java/google/registry/batch/ExpandBillingRecurrencesAction.java similarity index 87% rename from core/src/main/java/google/registry/batch/ExpandRecurringBillingEventsAction.java rename to core/src/main/java/google/registry/batch/ExpandBillingRecurrencesAction.java index a5e8068de..107ade955 100644 --- a/core/src/main/java/google/registry/batch/ExpandRecurringBillingEventsAction.java +++ b/core/src/main/java/google/registry/batch/ExpandBillingRecurrencesAction.java @@ -29,11 +29,11 @@ import com.google.api.services.dataflow.model.LaunchFlexTemplateRequest; import com.google.api.services.dataflow.model.LaunchFlexTemplateResponse; import com.google.common.collect.ImmutableMap; import com.google.common.flogger.FluentLogger; -import google.registry.beam.billing.ExpandRecurringBillingEventsPipeline; +import google.registry.beam.billing.ExpandBillingRecurrencesPipeline; import google.registry.config.RegistryConfig.Config; import google.registry.config.RegistryEnvironment; -import google.registry.model.billing.BillingEvent.OneTime; -import google.registry.model.billing.BillingEvent.Recurring; +import google.registry.model.billing.BillingEvent; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.common.Cursor; import google.registry.request.Action; import google.registry.request.Parameter; @@ -46,20 +46,20 @@ import javax.inject.Inject; import org.joda.time.DateTime; /** - * An action that kicks off a {@link ExpandRecurringBillingEventsPipeline} dataflow job to expand - * {@link Recurring} billing events into synthetic {@link OneTime} events. + * An action that kicks off a {@link ExpandBillingRecurrencesPipeline} dataflow job to expand {@link + * BillingRecurrence} billing events into synthetic {@link BillingEvent} events. */ @Action( service = Action.Service.BACKEND, - path = "/_dr/task/expandRecurringBillingEvents", + path = "/_dr/task/expandBillingRecurrences", auth = Auth.AUTH_INTERNAL_OR_ADMIN) -public class ExpandRecurringBillingEventsAction implements Runnable { +public class ExpandBillingRecurrencesAction implements Runnable { public static final String PARAM_START_TIME = "startTime"; public static final String PARAM_END_TIME = "endTime"; public static final String PARAM_ADVANCE_CURSOR = "advanceCursor"; - private static final String PIPELINE_NAME = "expand_recurring_billing_events_pipeline"; + private static final String PIPELINE_NAME = "expand_billing_recurrences_pipeline"; private static final FluentLogger logger = FluentLogger.forEnclosingClass(); @Inject Clock clock; @@ -97,7 +97,7 @@ public class ExpandRecurringBillingEventsAction implements Runnable { @Inject Response response; @Inject - ExpandRecurringBillingEventsAction() {} + ExpandBillingRecurrencesAction() {} @Override public void run() { @@ -133,7 +133,7 @@ public class ExpandRecurringBillingEventsAction implements Runnable { .put("advanceCursor", Boolean.toString(advanceCursor)) .build()); logger.atInfo().log( - "Launching recurring billing event expansion pipeline for event time range [%s, %s)%s.", + "Launching billing recurrence expansion pipeline for event time range [%s, %s)%s.", startTime, endTime, isDryRun ? " in dry run mode" : advanceCursor ? "" : " without advancing the cursor"); @@ -152,7 +152,7 @@ public class ExpandRecurringBillingEventsAction implements Runnable { response.setStatus(SC_OK); response.setPayload( String.format( - "Launched recurring billing event expansion pipeline: %s", + "Launched billing recurrence expansion pipeline: %s", launchResponse.getJob().getId())); } catch (IOException e) { logger.atWarning().withCause(e).log("Pipeline Launch failed"); diff --git a/core/src/main/java/google/registry/beam/billing/BillingEvent.java b/core/src/main/java/google/registry/beam/billing/BillingEvent.java index be07a87dd..7158d78fb 100644 --- a/core/src/main/java/google/registry/beam/billing/BillingEvent.java +++ b/core/src/main/java/google/registry/beam/billing/BillingEvent.java @@ -64,7 +64,7 @@ public abstract class BillingEvent implements Serializable { "amount", "flags"); - /** Returns the unique ID for the {@code OneTime} associated with this event. */ + /** Returns the unique ID for the {@code BillingEvent} associated with this event. */ abstract long id(); /** Returns the UTC DateTime this event becomes billable. */ diff --git a/core/src/main/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipeline.java b/core/src/main/java/google/registry/beam/billing/ExpandBillingRecurrencesPipeline.java similarity index 80% rename from core/src/main/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipeline.java rename to core/src/main/java/google/registry/beam/billing/ExpandBillingRecurrencesPipeline.java index 9b95549e5..7c903139f 100644 --- a/core/src/main/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipeline.java +++ b/core/src/main/java/google/registry/beam/billing/ExpandBillingRecurrencesPipeline.java @@ -38,10 +38,10 @@ import google.registry.flows.custom.CustomLogicModule; import google.registry.flows.domain.DomainPricingLogic; import google.registry.flows.domain.DomainPricingLogic.AllocationTokenInvalidForPremiumNameException; import google.registry.model.ImmutableObject; -import google.registry.model.billing.BillingEvent.Cancellation; -import google.registry.model.billing.BillingEvent.Flag; -import google.registry.model.billing.BillingEvent.OneTime; -import google.registry.model.billing.BillingEvent.Recurring; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingCancellation; +import google.registry.model.billing.BillingEvent; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.common.Cursor; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; @@ -77,48 +77,49 @@ import org.apache.beam.sdk.values.PDone; import org.joda.time.DateTime; /** - * Definition of a Dataflow Flex pipeline template, which expands {@link Recurring} to {@link - * OneTime} when an autorenew occurs within the given time frame. + * Definition of a Dataflow Flex pipeline template, which expands {@link BillingRecurrence} to + * {@link BillingEvent} when an autorenew occurs within the given time frame. * *

This pipeline works in three stages: * *

    - *
  • Gather the {@link Recurring}s that are in scope for expansion. The exact condition of - * {@link Recurring}s to include can be found in {@link #getRecurringsInScope(Pipeline)}. - *
  • Expand the {@link Recurring}s to {@link OneTime} (and corresponding {@link DomainHistory}) - * that fall within the [{@link #startTime}, {@link #endTime}) window, excluding those that - * are already present (to make this pipeline idempotent when running with the same parameters - * multiple times, either in parallel or in sequence). The {@link Recurring} is also updated - * with the information on when it was last expanded, so it would not be in scope for - * expansion until at least a year later. + *
  • Gather the {@link BillingRecurrence}s that are in scope for expansion. The exact condition + * of {@link BillingRecurrence}s to include can be found in {@link + * #getRecurrencesInScope(Pipeline)}. + *
  • Expand the {@link BillingRecurrence}s to {@link BillingEvent} (and corresponding {@link + * DomainHistory}) that fall within the [{@link #startTime}, {@link #endTime}) window, + * excluding those that are already present (to make this pipeline idempotent when running + * with the same parameters multiple times, either in parallel or in sequence). The {@link + * BillingRecurrence} is also updated with the information on when it was last expanded, so it + * would not be in scope for expansion until at least a year later. *
  • If the cursor for billing events should be advanced, advance it to {@link #endTime} after * all of the expansions in the previous step is done, only when it is currently at {@link * #startTime}. *
* - *

Note that the creation of new {@link OneTime} and {@link DomainHistory} is done speculatively - * as soon as its event time is in scope for expansion (i.e. within the window of operation). If a - * domain is subsequently cancelled during the autorenew grace period, a {@link Cancellation} would - * have been created to cancel the {@link OneTime} out. Similarly, a {@link DomainHistory} for the - * delete will be created which negates the effect of the speculatively created {@link - * DomainHistory}, specifically for the transaction records. Both the {@link OneTime} and {@link - * DomainHistory} will only be used (and cancelled out) when the billing time becomes effective, - * which is after the grace period, when the cancellations would have been written, if need be. This - * is no different from what we do with manual renewals or normal creates, where entities are always - * created for the action regardless of whether their effects will be negated later due to - * subsequent actions within respective grace periods. + *

Note that the creation of new {@link BillingEvent} and {@link DomainHistory} is done + * speculatively as soon as its event time is in scope for expansion (i.e. within the window of + * operation). If a domain is subsequently cancelled during the autorenew grace period, a {@link + * BillingCancellation} would have been created to cancel the {@link BillingEvent} out. Similarly, a + * {@link DomainHistory} for the delete will be created which negates the effect of the + * speculatively created {@link DomainHistory}, specifically for the transaction records. Both the + * {@link BillingEvent} and {@link DomainHistory} will only be used (and cancelled out) when the + * billing time becomes effective, which is after the grace period, when the cancellations would + * have been written, if need be. This is no different from what we do with manual renewals or + * normal creates, where entities are always created for the action regardless of whether their + * effects will be negated later due to subsequent actions within respective grace periods. * *

To stage this template locally, run {@code ./nom_build :core:sBP --environment=alpha \ * --pipeline=expandBilling}. * *

Then, you can run the staged template via the API client library, gCloud or a raw REST call. * - * @see Cancellation#forGracePeriod + * @see BillingCancellation#forGracePeriod * @see google.registry.flows.domain.DomainFlowUtils#createCancelingRecords * @see Using * Flex Templates */ -public class ExpandRecurringBillingEventsPipeline implements Serializable { +public class ExpandBillingRecurrencesPipeline implements Serializable { private static final long serialVersionUID = -5827984301386630194L; @@ -128,7 +129,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { static { PipelineComponent pipelineComponent = - DaggerExpandRecurringBillingEventsPipeline_PipelineComponent.create(); + DaggerExpandBillingRecurrencesPipeline_PipelineComponent.create(); domainPricingLogic = pipelineComponent.domainPricingLogic(); batchSize = pipelineComponent.batchSize(); } @@ -139,8 +140,8 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { private final DateTime endTime; private final boolean isDryRun; private final boolean advanceCursor; - private final Counter recurringsInScopeCounter = - Metrics.counter("ExpandBilling", "Recurrings in scope for expansion"); + private final Counter recurrencesInScopeCounter = + Metrics.counter("ExpandBilling", "Recurrences in scope for expansion"); // Note that this counter is only accurate when running in dry run mode. Because SQL persistence // is a side effect and not idempotent, a transaction to save OneTimes could be successful but the // transform that contains it could be still be retried, rolling back the counter increment. The @@ -150,8 +151,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { private final Counter oneTimesToExpandCounter = Metrics.counter("ExpandBilling", "OneTimes that would be expanded"); - ExpandRecurringBillingEventsPipeline( - ExpandRecurringBillingEventsPipelineOptions options, Clock clock) { + ExpandBillingRecurrencesPipeline(ExpandBillingRecurrencesPipelineOptions options, Clock clock) { startTime = DateTime.parse(options.getStartTime()); endTime = DateTime.parse(options.getEndTime()); checkArgument( @@ -170,16 +170,16 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { } void setupPipeline(Pipeline pipeline) { - PCollection> recurringIds = getRecurringsInScope(pipeline); - PCollection expanded = expandRecurrings(recurringIds); + PCollection> recurrenceIds = getRecurrencesInScope(pipeline); + PCollection expanded = expandRecurrences(recurrenceIds); if (!isDryRun && advanceCursor) { advanceCursor(expanded); } } - PCollection> getRecurringsInScope(Pipeline pipeline) { + PCollection> getRecurrencesInScope(Pipeline pipeline) { return pipeline.apply( - "Read all Recurrings in scope", + "Read all Recurrences in scope", // Use native query because JPQL does not support timestamp arithmetics. RegistryJpaIO.read( "SELECT billing_recurrence_id " @@ -203,7 +203,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { endTime.minusYears(1)), true, (BigInteger id) -> { - recurringsInScopeCounter.inc(); + recurrencesInScopeCounter.inc(); // Note that because all elements are mapped to the same dummy key, the next // batching transform will effectively be serial. This however does not matter for // our use case because the elements were obtained from a SQL read query, which @@ -222,13 +222,13 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { .withCoder(KvCoder.of(VarIntCoder.of(), VarLongCoder.of()))); } - private PCollection expandRecurrings(PCollection> recurringIds) { - return recurringIds + private PCollection expandRecurrences(PCollection> recurrenceIds) { + return recurrenceIds .apply( "Group into batches", GroupIntoBatches.ofSize(batchSize).withShardedKey()) .apply( - "Expand and save Recurrings into OneTimes and corresponding DomainHistories", + "Expand and save Recurrences into OneTimes and corresponding DomainHistories", MapElements.into(voids()) .via( element -> { @@ -237,7 +237,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { () -> { ImmutableSet.Builder results = new ImmutableSet.Builder<>(); - ids.forEach(id -> expandOneRecurring(id, results)); + ids.forEach(id -> expandOneRecurrence(id, results)); if (!isDryRun) { tm().putAll(results.build()); } @@ -246,14 +246,16 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { })); } - private void expandOneRecurring(Long recurringId, ImmutableSet.Builder results) { - Recurring recurring = tm().loadByKey(Recurring.createVKey(recurringId)); + private void expandOneRecurrence( + Long recurrenceId, ImmutableSet.Builder results) { + BillingRecurrence billingRecurrence = + tm().loadByKey(BillingRecurrence.createVKey(recurrenceId)); - // Determine the complete set of EventTimes this recurring event should expand to within + // Determine the complete set of EventTimes this recurrence event should expand to within // [max(recurrenceLastExpansion + 1 yr, startTime), min(recurrenceEndTime, endTime)). // - // This range should always be legal for recurrings that are returned from the query. However, - // it is possible that the recurring has changed between when the read transformation occurred + // This range should always be legal for recurrences that are returned from the query. However, + // it is possible that the recurrence has changed between when the read transformation occurred // and now. This could be caused by some out-of-process mutations (such as a domain deletion // closing out a previously open-ended recurrence), or more subtly, Beam could execute the same // work multiple times due to transient communication issues between workers and the scheduler. @@ -264,23 +266,25 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { // to work on the same batch. The second worker would see a new recurrence_last_expansion that // causes the range to be illegal. // - // The best way to handle any unexpected behavior is to simply drop the recurring from + // The best way to handle any unexpected behavior is to simply drop the recurrence from // expansion, if its new state still calls for an expansion, it would be picked up the next time // the pipeline runs. ImmutableSet eventTimes; try { eventTimes = ImmutableSet.copyOf( - recurring + billingRecurrence .getRecurrenceTimeOfYear() .getInstancesInRange( Range.closedOpen( - latestOf(recurring.getRecurrenceLastExpansion().plusYears(1), startTime), - earliestOf(recurring.getRecurrenceEndTime(), endTime)))); + latestOf( + billingRecurrence.getRecurrenceLastExpansion().plusYears(1), + startTime), + earliestOf(billingRecurrence.getRecurrenceEndTime(), endTime)))); } catch (IllegalArgumentException e) { return; } - Domain domain = tm().loadByKey(Domain.createVKey(recurring.getDomainRepoId())); + Domain domain = tm().loadByKey(Domain.createVKey(billingRecurrence.getDomainRepoId())); Tld tld = Tld.get(domain.getTld()); // Find the times for which the OneTime billing event are already created, making this expansion @@ -292,7 +296,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { "SELECT eventTime FROM BillingEvent WHERE cancellationMatchingBillingEvent =" + " :key", DateTime.class) - .setParameter("key", recurring.createVKey()) + .setParameter("key", billingRecurrence.createVKey()) .getResultList()); Set eventTimesToExpand = difference(eventTimes, existingEventTimes); @@ -301,7 +305,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { return; } - DateTime recurrenceLastExpansionTime = recurring.getRecurrenceLastExpansion(); + DateTime recurrenceLastExpansionTime = billingRecurrence.getRecurrenceLastExpansion(); // Create new OneTime and DomainHistory for EventTimes that needs to be expanded. for (DateTime eventTime : eventTimesToExpand) { @@ -333,7 +337,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { DomainHistory historyEntry = new DomainHistory.Builder() .setBySuperuser(false) - .setRegistrarId(recurring.getRegistrarId()) + .setRegistrarId(billingRecurrence.getRegistrarId()) .setModificationTime(tm().getTransactionTime()) .setDomain(domain) .setPeriod(Period.create(1, YEARS)) @@ -385,35 +389,43 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { // It is OK to always create a OneTime, even though the domain might be deleted or transferred // later during autorenew grace period, as a cancellation will always be written out in those // instances. - OneTime oneTime = null; + BillingEvent billingEvent = null; try { - oneTime = - new OneTime.Builder() + billingEvent = + new BillingEvent.Builder() .setBillingTime(billingTime) - .setRegistrarId(recurring.getRegistrarId()) + .setRegistrarId(billingRecurrence.getRegistrarId()) // Determine the cost for a one-year renewal. .setCost( domainPricingLogic .getRenewPrice( - tld, recurring.getTargetId(), eventTime, 1, recurring, Optional.empty()) + tld, + billingRecurrence.getTargetId(), + eventTime, + 1, + billingRecurrence, + Optional.empty()) .getRenewCost()) .setEventTime(eventTime) - .setFlags(union(recurring.getFlags(), Flag.SYNTHETIC)) + .setFlags(union(billingRecurrence.getFlags(), Flag.SYNTHETIC)) .setDomainHistory(historyEntry) .setPeriodYears(1) - .setReason(recurring.getReason()) + .setReason(billingRecurrence.getReason()) .setSyntheticCreationTime(endTime) - .setCancellationMatchingBillingEvent(recurring) - .setTargetId(recurring.getTargetId()) + .setCancellationMatchingBillingEvent(billingRecurrence) + .setTargetId(billingRecurrence.getTargetId()) .build(); } catch (AllocationTokenInvalidForPremiumNameException e) { // This should not be reached since we are not using an allocation token return; } - results.add(oneTime); + results.add(billingEvent); } results.add( - recurring.asBuilder().setRecurrenceLastExpansion(recurrenceLastExpansionTime).build()); + billingRecurrence + .asBuilder() + .setRecurrenceLastExpansion(recurrenceLastExpansionTime) + .build()); } private PDone advanceCursor(PCollection persisted) { @@ -456,11 +468,11 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { } public static void main(String[] args) { - PipelineOptionsFactory.register(ExpandRecurringBillingEventsPipelineOptions.class); - ExpandRecurringBillingEventsPipelineOptions options = + PipelineOptionsFactory.register(ExpandBillingRecurrencesPipelineOptions.class); + ExpandBillingRecurrencesPipelineOptions options = PipelineOptionsFactory.fromArgs(args) .withValidation() - .as(ExpandRecurringBillingEventsPipelineOptions.class); + .as(ExpandBillingRecurrencesPipelineOptions.class); // Hardcode the transaction level to be at serializable we do not want concurrent runs of the // pipeline for the same window to create duplicate OneTimes. This ensures that the set of // existing OneTimes do not change by the time new OneTimes are inserted within a transaction. @@ -468,7 +480,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { // Per PostgreSQL, serializable isolation level does not introduce any blocking beyond that // present in repeatable read other than some overhead related to monitoring possible // serializable anomalies. Therefore, in most cases, since each worker of the same job works on - // a different set of recurrings, it is not possible for their execution order to affect + // a different set of recurrences, it is not possible for their execution order to affect // serialization outcome, and the performance penalty should be minimum when using serializable // compared to using repeatable read. // @@ -480,7 +492,7 @@ public class ExpandRecurringBillingEventsPipeline implements Serializable { // See: https://www.postgresql.org/docs/current/transaction-iso.html options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_SERIALIZABLE); Pipeline pipeline = Pipeline.create(options); - new ExpandRecurringBillingEventsPipeline(options, new SystemClock()).run(pipeline); + new ExpandBillingRecurrencesPipeline(options, new SystemClock()).run(pipeline); } @Singleton diff --git a/core/src/main/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipelineOptions.java b/core/src/main/java/google/registry/beam/billing/ExpandBillingRecurrencesPipelineOptions.java similarity index 94% rename from core/src/main/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipelineOptions.java rename to core/src/main/java/google/registry/beam/billing/ExpandBillingRecurrencesPipelineOptions.java index 894313666..adcd20a3c 100644 --- a/core/src/main/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipelineOptions.java +++ b/core/src/main/java/google/registry/beam/billing/ExpandBillingRecurrencesPipelineOptions.java @@ -18,7 +18,7 @@ import google.registry.beam.common.RegistryPipelineOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; -public interface ExpandRecurringBillingEventsPipelineOptions extends RegistryPipelineOptions { +public interface ExpandBillingRecurrencesPipelineOptions extends RegistryPipelineOptions { @Description( "The inclusive lower bound of on the range of event times that will be expanded, in ISO 8601" + " format") diff --git a/core/src/main/java/google/registry/beam/billing/InvoicingPipeline.java b/core/src/main/java/google/registry/beam/billing/InvoicingPipeline.java index a7f9e3bd7..d3a4a7f0d 100644 --- a/core/src/main/java/google/registry/beam/billing/InvoicingPipeline.java +++ b/core/src/main/java/google/registry/beam/billing/InvoicingPipeline.java @@ -22,8 +22,8 @@ import google.registry.beam.billing.BillingEvent.InvoiceGroupingKey; import google.registry.beam.billing.BillingEvent.InvoiceGroupingKey.InvoiceGroupingKeyCoder; import google.registry.beam.common.RegistryJpaIO; import google.registry.beam.common.RegistryJpaIO.Read; -import google.registry.model.billing.BillingEvent.Flag; -import google.registry.model.billing.BillingEvent.OneTime; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingEvent; import google.registry.model.registrar.Registrar; import google.registry.persistence.PersistenceModule.TransactionIsolationLevel; import google.registry.reporting.billing.BillingModule; @@ -86,29 +86,30 @@ public class InvoicingPipeline implements Serializable { void setupPipeline(Pipeline pipeline) { options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_READ_COMMITTED); - PCollection billingEvents = readFromCloudSql(options, pipeline); + PCollection billingEvents = + readFromCloudSql(options, pipeline); saveInvoiceCsv(billingEvents, options); saveDetailedCsv(billingEvents, options); } - static PCollection readFromCloudSql( + static PCollection readFromCloudSql( InvoicingPipelineOptions options, Pipeline pipeline) { - Read read = - RegistryJpaIO.read( + Read read = + RegistryJpaIO.read( makeCloudSqlQuery(options.getYearMonth()), false, row -> parseRow(row).orElse(null)) - .withCoder(SerializableCoder.of(BillingEvent.class)); + .withCoder(SerializableCoder.of(google.registry.beam.billing.BillingEvent.class)); - PCollection billingEventsWithNulls = + PCollection billingEventsWithNulls = pipeline.apply("Read BillingEvents from Cloud SQL", read); // Remove null billing events return billingEventsWithNulls.apply(Filter.by(Objects::nonNull)); } - private static Optional parseRow(Object[] row) { - OneTime oneTime = (OneTime) row[0]; + private static Optional parseRow(Object[] row) { + BillingEvent billingEvent = (BillingEvent) row[0]; Registrar registrar = (Registrar) row[1]; - CurrencyUnit currency = oneTime.getCost().getCurrencyUnit(); + CurrencyUnit currency = billingEvent.getCost().getCurrencyUnit(); if (!registrar.getBillingAccountMap().containsKey(currency)) { logger.atSevere().log( "Registrar %s does not have a product account key for the currency unit: %s", @@ -117,37 +118,40 @@ public class InvoicingPipeline implements Serializable { } return Optional.of( - BillingEvent.create( - oneTime.getId(), - oneTime.getBillingTime(), - oneTime.getEventTime(), + google.registry.beam.billing.BillingEvent.create( + billingEvent.getId(), + billingEvent.getBillingTime(), + billingEvent.getEventTime(), registrar.getRegistrarId(), registrar.getBillingAccountMap().get(currency), registrar.getPoNumber().orElse(""), - DomainNameUtils.getTldFromDomainName(oneTime.getTargetId()), - oneTime.getReason().toString(), - oneTime.getTargetId(), - oneTime.getDomainRepoId(), - Optional.ofNullable(oneTime.getPeriodYears()).orElse(0), - oneTime.getCost().getCurrencyUnit().toString(), - oneTime.getCost().getAmount().doubleValue(), + DomainNameUtils.getTldFromDomainName(billingEvent.getTargetId()), + billingEvent.getReason().toString(), + billingEvent.getTargetId(), + billingEvent.getDomainRepoId(), + Optional.ofNullable(billingEvent.getPeriodYears()).orElse(0), + billingEvent.getCost().getCurrencyUnit().toString(), + billingEvent.getCost().getAmount().doubleValue(), String.join( - " ", oneTime.getFlags().stream().map(Flag::toString).collect(toImmutableSet())))); + " ", + billingEvent.getFlags().stream().map(Flag::toString).collect(toImmutableSet())))); } /** Transform that converts a {@code BillingEvent} into an invoice CSV row. */ private static class GenerateInvoiceRows - extends PTransform, PCollection> { + extends PTransform< + PCollection, PCollection> { private static final long serialVersionUID = -8090619008258393728L; @Override - public PCollection expand(PCollection input) { + public PCollection expand( + PCollection input) { return input .apply( "Map to invoicing key", MapElements.into(TypeDescriptor.of(InvoiceGroupingKey.class)) - .via(BillingEvent::getInvoiceGroupingKey)) + .via(google.registry.beam.billing.BillingEvent::getInvoiceGroupingKey)) .apply( "Filter out free events", Filter.by((InvoiceGroupingKey key) -> key.unitPrice() != 0)) .setCoder(new InvoiceGroupingKeyCoder()) @@ -161,7 +165,8 @@ public class InvoicingPipeline implements Serializable { /** Saves the billing events to a single overall invoice CSV file. */ static void saveInvoiceCsv( - PCollection billingEvents, InvoicingPipelineOptions options) { + PCollection billingEvents, + InvoicingPipelineOptions options) { billingEvents .apply("Generate overall invoice rows", new GenerateInvoiceRows()) .apply( @@ -182,16 +187,17 @@ public class InvoicingPipeline implements Serializable { /** Saves the billing events to detailed report CSV files keyed by registrar-tld pairs. */ static void saveDetailedCsv( - PCollection billingEvents, InvoicingPipelineOptions options) { + PCollection billingEvents, + InvoicingPipelineOptions options) { String yearMonth = options.getYearMonth(); billingEvents.apply( "Write detailed report for each registrar-tld pair", - FileIO.writeDynamic() + FileIO.writeDynamic() .to( String.format( "%s/%s/%s", options.getBillingBucketUrl(), BillingModule.INVOICES_DIRECTORY, yearMonth)) - .by(BillingEvent::getDetailedReportGroupingKey) + .by(google.registry.beam.billing.BillingEvent::getDetailedReportGroupingKey) .withNumShards(1) .withDestinationCoder(StringUtf8Coder.of()) .withNaming( @@ -200,8 +206,8 @@ public class InvoicingPipeline implements Serializable { String.format( "%s_%s_%s.csv", BillingModule.DETAIL_REPORT_PREFIX, yearMonth, key)) .via( - Contextful.fn(BillingEvent::toCsv), - TextIO.sink().withHeader(BillingEvent.getHeader()))); + Contextful.fn(google.registry.beam.billing.BillingEvent::toCsv), + TextIO.sink().withHeader(google.registry.beam.billing.BillingEvent.getHeader()))); } /** Create the Cloud SQL query for a given yearMonth at runtime. */ diff --git a/core/src/main/java/google/registry/env/alpha/default/WEB-INF/cloud-scheduler-tasks.xml b/core/src/main/java/google/registry/env/alpha/default/WEB-INF/cloud-scheduler-tasks.xml index dc12142ca..49be09491 100644 --- a/core/src/main/java/google/registry/env/alpha/default/WEB-INF/cloud-scheduler-tasks.xml +++ b/core/src/main/java/google/registry/env/alpha/default/WEB-INF/cloud-scheduler-tasks.xml @@ -80,12 +80,13 @@ - - expandRecurringBillingEvents + + expandBillingRecurrences - This job runs an action that creates synthetic OneTime billing events from Recurring billing - events. Events are created for all instances of Recurring billing events that should exist - between the RECURRING_BILLING cursor's time and the execution time of the action. + This job runs an action that creates synthetic one-time billing events + from billing recurrences. Events are created for all recurrences that + should exist between the RECURRING_BILLING cursor's time and the execution + time of the action. 0 3 * * * diff --git a/core/src/main/java/google/registry/env/common/backend/WEB-INF/web.xml b/core/src/main/java/google/registry/env/common/backend/WEB-INF/web.xml index 8da5a244b..0437d04e6 100644 --- a/core/src/main/java/google/registry/env/common/backend/WEB-INF/web.xml +++ b/core/src/main/java/google/registry/env/common/backend/WEB-INF/web.xml @@ -246,10 +246,10 @@ /_dr/task/refreshDnsOnHostRename - + backend-servlet - /_dr/task/expandRecurringBillingEvents + /_dr/task/expandBillingRecurrences diff --git a/core/src/main/java/google/registry/env/production/default/WEB-INF/cloud-scheduler-tasks.xml b/core/src/main/java/google/registry/env/production/default/WEB-INF/cloud-scheduler-tasks.xml index 9dff6af20..a5a583ad1 100644 --- a/core/src/main/java/google/registry/env/production/default/WEB-INF/cloud-scheduler-tasks.xml +++ b/core/src/main/java/google/registry/env/production/default/WEB-INF/cloud-scheduler-tasks.xml @@ -122,12 +122,13 @@ - - expandRecurringBillingEvents + + expandBillingRecurrences - This job runs an action that creates synthetic OneTime billing events from Recurring billing - events. Events are created for all instances of Recurring billing events that should exist - between the RECURRING_BILLING cursor's time and the execution time of the action. + This job runs an action that creates synthetic one-time billing events + from billing recurrences. Events are created for all recurrences that + should exist between the RECURRING_BILLING cursor's time and the execution + time of the action. 0 3 * * * @@ -253,7 +254,7 @@ reports to the associated registrars' drive folders. See GenerateInvoicesAction for more details. - - google.registry.model.billing.VKeyConverter_Cancellation - google.registry.model.billing.VKeyConverter_OneTime - google.registry.model.billing.VKeyConverter_Recurring + google.registry.model.billing.VKeyConverter_BillingCancellation + google.registry.model.billing.VKeyConverter_BillingEvent + google.registry.model.billing.VKeyConverter_BillingRecurrence google.registry.model.contact.VKeyConverter_Contact google.registry.model.domain.VKeyConverter_Domain google.registry.model.domain.token.VKeyConverter_AllocationToken diff --git a/core/src/main/resources/google/registry/beam/billing/sql/cloud_sql_billing_events.sql b/core/src/main/resources/google/registry/beam/billing/sql/cloud_sql_billing_events.sql index 2cf65334c..4825527c8 100644 --- a/core/src/main/resources/google/registry/beam/billing/sql/cloud_sql_billing_events.sql +++ b/core/src/main/resources/google/registry/beam/billing/sql/cloud_sql_billing_events.sql @@ -29,8 +29,8 @@ SELECT b, r FROM BillingEvent b JOIN Registrar r ON b.clientId = r.registrarId JOIN Domain d ON b.domainRepoId = d.repoId JOIN Tld t ON t.tldStr = d.tld -LEFT JOIN BillingCancellation c ON b.id = c.refOneTime -LEFT JOIN BillingCancellation cr ON b.cancellationMatchingBillingEvent = cr.refRecurring +LEFT JOIN BillingCancellation c ON b.id = c.billingEvent +LEFT JOIN BillingCancellation cr ON b.cancellationMatchingBillingEvent = cr.billingRecurrence WHERE r.billingAccountMap IS NOT NULL AND r.type = 'REAL' AND t.invoicingEnabled IS TRUE diff --git a/core/src/main/resources/google/registry/beam/expand_recurring_billing_events_pipeline_metadata.json b/core/src/main/resources/google/registry/beam/expand_billing_recurrences_pipeline_metadata.json similarity index 100% rename from core/src/main/resources/google/registry/beam/expand_recurring_billing_events_pipeline_metadata.json rename to core/src/main/resources/google/registry/beam/expand_billing_recurrences_pipeline_metadata.json diff --git a/core/src/test/java/google/registry/batch/CheckPackagesComplianceActionTest.java b/core/src/test/java/google/registry/batch/CheckPackagesComplianceActionTest.java index 29fde5e31..25469f048 100644 --- a/core/src/test/java/google/registry/batch/CheckPackagesComplianceActionTest.java +++ b/core/src/test/java/google/registry/batch/CheckPackagesComplianceActionTest.java @@ -28,7 +28,7 @@ import static org.mockito.Mockito.verifyNoInteractions; import com.google.common.collect.ImmutableList; import com.google.common.testing.TestLogHandler; -import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; import google.registry.model.contact.Contact; import google.registry.model.domain.token.AllocationToken; import google.registry.model.domain.token.AllocationToken.TokenType; diff --git a/core/src/test/java/google/registry/batch/DeleteExpiredDomainsActionTest.java b/core/src/test/java/google/registry/batch/DeleteExpiredDomainsActionTest.java index f9960abad..abb738807 100644 --- a/core/src/test/java/google/registry/batch/DeleteExpiredDomainsActionTest.java +++ b/core/src/test/java/google/registry/batch/DeleteExpiredDomainsActionTest.java @@ -29,9 +29,9 @@ import com.google.common.collect.ImmutableSet; import google.registry.flows.DaggerEppTestComponent; import google.registry.flows.EppController; import google.registry.flows.EppTestComponent.FakesAndMocksModule; -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.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.poll.PollMessage; @@ -173,7 +173,7 @@ class DeleteExpiredDomainsActionTest { .setModificationTime(clock.nowUtc().minusMonths(9)) .setRegistrarId(pendingExpirationDomain.getCreationRegistrarId()) .build()); - BillingEvent.Recurring autorenewBillingEvent = + BillingRecurrence autorenewBillingEvent = persistResource(createAutorenewBillingEvent(createHistoryEntry).build()); PollMessage.Autorenew autorenewPollMessage = persistResource(createAutorenewPollMessage(createHistoryEntry).build()); @@ -189,9 +189,8 @@ class DeleteExpiredDomainsActionTest { return pendingExpirationDomain; } - private BillingEvent.Recurring.Builder createAutorenewBillingEvent( - DomainHistory createHistoryEntry) { - return new BillingEvent.Recurring.Builder() + private BillingRecurrence.Builder createAutorenewBillingEvent(DomainHistory createHistoryEntry) { + return new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId("fizz.tld") diff --git a/core/src/test/java/google/registry/batch/DeleteProberDataActionTest.java b/core/src/test/java/google/registry/batch/DeleteProberDataActionTest.java index b4fbd7c97..d757fe952 100644 --- a/core/src/test/java/google/registry/batch/DeleteProberDataActionTest.java +++ b/core/src/test/java/google/registry/batch/DeleteProberDataActionTest.java @@ -36,8 +36,8 @@ import com.google.common.collect.ImmutableSet; import google.registry.config.RegistryEnvironment; import google.registry.dns.DnsUtils; import google.registry.model.ImmutableObject; +import google.registry.model.billing.BillingBase.Reason; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.Reason; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.poll.PollMessage; @@ -290,9 +290,9 @@ class DeleteProberDataActionTest { .setRegistrarId("TheRegistrar") .setModificationTime(DELETION_TIME.minusYears(3)) .build()); - BillingEvent.OneTime billingEvent = + BillingEvent billingEvent = persistSimpleResource( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setDomainHistory(historyEntry) .setBillingTime(DELETION_TIME.plusYears(1)) .setCost(Money.parse("USD 10")) diff --git a/core/src/test/java/google/registry/batch/ExpandRecurringBillingEventsActionTest.java b/core/src/test/java/google/registry/batch/ExpandBillingRecurrencesActionTest.java similarity index 91% rename from core/src/test/java/google/registry/batch/ExpandRecurringBillingEventsActionTest.java rename to core/src/test/java/google/registry/batch/ExpandBillingRecurrencesActionTest.java index 1b63afd21..7e8c07d5c 100644 --- a/core/src/test/java/google/registry/batch/ExpandRecurringBillingEventsActionTest.java +++ b/core/src/test/java/google/registry/batch/ExpandBillingRecurrencesActionTest.java @@ -39,15 +39,14 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.mockito.ArgumentCaptor; -/** Unit tests for {@link ExpandRecurringBillingEventsAction}. */ -public class ExpandRecurringBillingEventsActionTest extends BeamActionTestBase { +/** Unit tests for {@link ExpandBillingRecurrencesAction}. */ +public class ExpandBillingRecurrencesActionTest extends BeamActionTestBase { private final DateTime cursorTime = DateTime.parse("2020-02-01T00:00:00Z"); private final DateTime now = DateTime.parse("2020-02-02T00:00:00Z"); private final FakeClock clock = new FakeClock(now); - private final ExpandRecurringBillingEventsAction action = - new ExpandRecurringBillingEventsAction(); + private final ExpandBillingRecurrencesAction action = new ExpandBillingRecurrencesAction(); private final HashMap expectedParameters = new HashMap<>(); private final ArgumentCaptor launchRequest = @@ -82,7 +81,7 @@ public class ExpandRecurringBillingEventsActionTest extends BeamActionTestBase { action.run(); assertThat(response.getStatus()).isEqualTo(200); assertThat(response.getPayload()) - .isEqualTo("Launched recurring billing event expansion pipeline: jobid"); + .isEqualTo("Launched billing recurrence expansion pipeline: jobid"); verify(templates, times(1)).launch(eq("projectId"), eq("jobRegion"), launchRequest.capture()); assertThat(launchRequest.getValue().getLaunchParameter().getParameters()) .containsExactlyEntriesIn(expectedParameters); @@ -95,7 +94,7 @@ public class ExpandRecurringBillingEventsActionTest extends BeamActionTestBase { action.run(); assertThat(response.getStatus()).isEqualTo(200); assertThat(response.getPayload()) - .isEqualTo("Launched recurring billing event expansion pipeline: jobid"); + .isEqualTo("Launched billing recurrence expansion pipeline: jobid"); verify(templates, times(1)).launch(eq("projectId"), eq("jobRegion"), launchRequest.capture()); assertThat(launchRequest.getValue().getLaunchParameter().getParameters()) .containsExactlyEntriesIn(expectedParameters); @@ -108,7 +107,7 @@ public class ExpandRecurringBillingEventsActionTest extends BeamActionTestBase { action.run(); assertThat(response.getStatus()).isEqualTo(200); assertThat(response.getPayload()) - .isEqualTo("Launched recurring billing event expansion pipeline: jobid"); + .isEqualTo("Launched billing recurrence expansion pipeline: jobid"); verify(templates, times(1)).launch(eq("projectId"), eq("jobRegion"), launchRequest.capture()); assertThat(launchRequest.getValue().getLaunchParameter().getParameters()) .containsExactlyEntriesIn(expectedParameters); @@ -121,7 +120,7 @@ public class ExpandRecurringBillingEventsActionTest extends BeamActionTestBase { action.run(); assertThat(response.getStatus()).isEqualTo(200); assertThat(response.getPayload()) - .isEqualTo("Launched recurring billing event expansion pipeline: jobid"); + .isEqualTo("Launched billing recurrence expansion pipeline: jobid"); verify(templates, times(1)).launch(eq("projectId"), eq("jobRegion"), launchRequest.capture()); assertThat(launchRequest.getValue().getLaunchParameter().getParameters()) .containsExactlyEntriesIn(expectedParameters); @@ -136,7 +135,7 @@ public class ExpandRecurringBillingEventsActionTest extends BeamActionTestBase { action.run(); assertThat(response.getStatus()).isEqualTo(200); assertThat(response.getPayload()) - .isEqualTo("Launched recurring billing event expansion pipeline: jobid"); + .isEqualTo("Launched billing recurrence expansion pipeline: jobid"); verify(templates, times(1)).launch(eq("projectId"), eq("jobRegion"), launchRequest.capture()); assertThat(launchRequest.getValue().getLaunchParameter().getParameters()) .containsExactlyEntriesIn(expectedParameters); diff --git a/core/src/test/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipelineTest.java b/core/src/test/java/google/registry/beam/billing/ExpandBillingRecurrencesPipelineTest.java similarity index 87% rename from core/src/test/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipelineTest.java rename to core/src/test/java/google/registry/beam/billing/ExpandBillingRecurrencesPipelineTest.java index 558831800..70cb68e26 100644 --- a/core/src/test/java/google/registry/beam/billing/ExpandRecurringBillingEventsPipelineTest.java +++ b/core/src/test/java/google/registry/beam/billing/ExpandBillingRecurrencesPipelineTest.java @@ -37,11 +37,10 @@ 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.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; 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.billing.BillingRecurrence; import google.registry.model.common.Cursor; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; @@ -71,8 +70,8 @@ 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 { +/** Unit tests for {@link ExpandBillingRecurrencesPipeline}. */ +public class ExpandBillingRecurrencesPipelineTest { private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); @@ -87,7 +86,7 @@ public class ExpandRecurringBillingEventsPipelineTest { private Domain domain; - private Recurring recurring; + private BillingRecurrence billingRecurrence; private final TestOptions options = PipelineOptionsFactory.create().as(TestOptions.class); @@ -115,7 +114,7 @@ public class ExpandRecurringBillingEventsPipelineTest { // Set up the database. createTld("tld"); - recurring = createDomainAtTime("example.tld", startTime.minusYears(1).plusHours(12)); + billingRecurrence = createDomainAtTime("example.tld", startTime.minusYears(1).plusHours(12)); domain = loadByForeignKey(Domain.class, "example.tld", clock.nowUtc()).get(); } @@ -150,7 +149,7 @@ public class ExpandRecurringBillingEventsPipelineTest { assertBillingEventsForResource( domain, defaultOneTime(getOnlyAutoRenewHistory()), - recurring + billingRecurrence .asBuilder() .setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1)) .build()); @@ -162,8 +161,9 @@ public class ExpandRecurringBillingEventsPipelineTest { @Test void testSuccess_expandSingleEvent_deletedDuringGracePeriod() { domain = persistResource(domain.asBuilder().setDeletionTime(endTime.minusHours(2)).build()); - recurring = - persistResource(recurring.asBuilder().setRecurrenceEndTime(endTime.minusHours(2)).build()); + billingRecurrence = + persistResource( + billingRecurrence.asBuilder().setRecurrenceEndTime(endTime.minusHours(2)).build()); runPipeline(); // Assert about DomainHistory, no transaction record should have been written. @@ -174,7 +174,7 @@ public class ExpandRecurringBillingEventsPipelineTest { assertBillingEventsForResource( domain, defaultOneTime(getOnlyAutoRenewHistory()), - recurring + billingRecurrence .asBuilder() .setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1)) .build()); @@ -199,7 +199,7 @@ public class ExpandRecurringBillingEventsPipelineTest { assertBillingEventsForResource( domain, defaultOneTime(getOnlyAutoRenewHistory()), - recurring + billingRecurrence .asBuilder() .setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1)) .build()); @@ -210,11 +210,11 @@ public class ExpandRecurringBillingEventsPipelineTest { @Test void testSuccess_noExpansion_recurrenceClosedBeforeEventTime() { - recurring = + billingRecurrence = persistResource( - recurring + billingRecurrence .asBuilder() - .setRecurrenceEndTime(recurring.getEventTime().minusDays(1)) + .setRecurrenceEndTime(billingRecurrence.getEventTime().minusDays(1)) .build()); runPipeline(); assertNoExpansionsHappened(); @@ -222,19 +222,20 @@ public class ExpandRecurringBillingEventsPipelineTest { @Test void testSuccess_noExpansion_recurrenceClosedBeforeStartTime() { - recurring = - persistResource(recurring.asBuilder().setRecurrenceEndTime(startTime.minusDays(1)).build()); + billingRecurrence = + persistResource( + billingRecurrence.asBuilder().setRecurrenceEndTime(startTime.minusDays(1)).build()); runPipeline(); assertNoExpansionsHappened(); } @Test void testSuccess_noExpansion_recurrenceClosedBeforeNextExpansion() { - recurring = + billingRecurrence = persistResource( - recurring + billingRecurrence .asBuilder() - .setEventTime(recurring.getEventTime().minusYears(1)) + .setEventTime(billingRecurrence.getEventTime().minusYears(1)) .setRecurrenceEndTime(startTime.plusHours(6)) .build()); runPipeline(); @@ -243,16 +244,17 @@ public class ExpandRecurringBillingEventsPipelineTest { @Test void testSuccess_noExpansion_eventTimeAfterEndTime() { - recurring = persistResource(recurring.asBuilder().setEventTime(endTime.plusDays(1)).build()); + billingRecurrence = + persistResource(billingRecurrence.asBuilder().setEventTime(endTime.plusDays(1)).build()); runPipeline(); assertNoExpansionsHappened(); } @Test void testSuccess_noExpansion_LastExpansionLessThanAYearAgo() { - recurring = + billingRecurrence = persistResource( - recurring + billingRecurrence .asBuilder() .setRecurrenceLastExpansion(startTime.minusYears(1).plusDays(1)) .build()); @@ -263,7 +265,7 @@ public class ExpandRecurringBillingEventsPipelineTest { @Test void testSuccess_noExpansion_oneTimeAlreadyExists() { DomainHistory history = persistResource(defaultDomainHistory()); - OneTime oneTime = persistResource(defaultOneTime(history)); + BillingEvent billingEvent = persistResource(defaultOneTime(history)); runPipeline(); // Assert about DomainHistory. @@ -271,7 +273,7 @@ public class ExpandRecurringBillingEventsPipelineTest { // Assert about BillingEvents. No expansion happened, so last recurrence expansion time is // unchanged. - assertBillingEventsForResource(domain, oneTime, recurring); + assertBillingEventsForResource(domain, billingEvent, billingRecurrence); // Assert about Cursor. assertCursorAt(endTime); @@ -296,7 +298,7 @@ public class ExpandRecurringBillingEventsPipelineTest { assertBillingEventsForResource( domain, defaultOneTime(getOnlyAutoRenewHistory()), - recurring + billingRecurrence .asBuilder() .setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1)) .build()); @@ -306,7 +308,7 @@ public class ExpandRecurringBillingEventsPipelineTest { } // 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 + // properly. When two threads are used, the two recurrences are processed in different workers and // should be processed in parallel. @ParameterizedTest @ValueSource(ints = {1, 2}) @@ -318,7 +320,7 @@ public class ExpandRecurringBillingEventsPipelineTest { .setPremiumList(persistPremiumList("premium", USD, "other,USD 100")) .build()); DateTime otherCreateTime = startTime.minusYears(1).plusHours(5); - Recurring otherRecurring = createDomainAtTime("other.test", otherCreateTime); + BillingRecurrence otherBillingRecurrence = createDomainAtTime("other.test", otherCreateTime); Domain otherDomain = loadByForeignKey(Domain.class, "other.test", clock.nowUtc()).get(); options.setTargetParallelism(numOfThreads); @@ -334,14 +336,15 @@ public class ExpandRecurringBillingEventsPipelineTest { assertBillingEventsForResource( domain, defaultOneTime(getOnlyAutoRenewHistory()), - recurring + billingRecurrence .asBuilder() .setRecurrenceLastExpansion(domain.getCreationTime().plusYears(1)) .build()); assertBillingEventsForResource( otherDomain, - defaultOneTime(otherDomain, getOnlyAutoRenewHistory(otherDomain), otherRecurring, 100), - otherRecurring + defaultOneTime( + otherDomain, getOnlyAutoRenewHistory(otherDomain), otherBillingRecurrence, 100), + otherBillingRecurrence .asBuilder() .setRecurrenceLastExpansion(otherDomain.getCreationTime().plusYears(1)) .build()); @@ -396,7 +399,7 @@ public class ExpandRecurringBillingEventsPipelineTest { .setBillingTime( domain.getCreationTime().plusYears(2).plus(Tld.DEFAULT_AUTO_RENEW_GRACE_PERIOD)) .build(), - recurring + billingRecurrence .asBuilder() .setRecurrenceLastExpansion(domain.getCreationTime().plusYears(2)) .build()); @@ -406,9 +409,9 @@ public class ExpandRecurringBillingEventsPipelineTest { } private void runPipeline() { - ExpandRecurringBillingEventsPipeline expandRecurringBillingEventsPipeline = - new ExpandRecurringBillingEventsPipeline(options, clock); - expandRecurringBillingEventsPipeline.setupPipeline(pipeline); + ExpandBillingRecurrencesPipeline expandBillingRecurrencesPipeline = + new ExpandBillingRecurrencesPipeline(options, clock); + expandBillingRecurrencesPipeline.setupPipeline(pipeline); pipeline.run(options).waitUntilFinish(); } @@ -424,7 +427,7 @@ public class ExpandRecurringBillingEventsPipelineTest { assertThat(persistedHistory.get(0).getType()).isEqualTo(DOMAIN_CREATE); // Only the original recurrence is present. - assertBillingEventsForResource(domain, recurring); + assertBillingEventsForResource(domain, billingRecurrence); // 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 @@ -458,13 +461,13 @@ public class ExpandRecurringBillingEventsPipelineTest { .build(); } - private OneTime defaultOneTime(DomainHistory history) { - return defaultOneTime(domain, history, recurring, 11); + private BillingEvent defaultOneTime(DomainHistory history) { + return defaultOneTime(domain, history, billingRecurrence, 11); } - private OneTime defaultOneTime( - Domain domain, DomainHistory history, Recurring recurring, int cost) { - return new BillingEvent.OneTime.Builder() + private BillingEvent defaultOneTime( + Domain domain, DomainHistory history, BillingRecurrence billingRecurrence, int cost) { + return new BillingEvent.Builder() .setBillingTime( domain.getCreationTime().plusYears(1).plus(Tld.DEFAULT_AUTO_RENEW_GRACE_PERIOD)) .setRegistrarId("TheRegistrar") @@ -474,7 +477,7 @@ public class ExpandRecurringBillingEventsPipelineTest { .setPeriodYears(1) .setReason(Reason.RENEW) .setSyntheticCreationTime(endTime) - .setCancellationMatchingBillingEvent(recurring) + .setCancellationMatchingBillingEvent(billingRecurrence) .setTargetId(domain.getDomainName()) .setDomainHistory(history) .build(); @@ -517,7 +520,7 @@ public class ExpandRecurringBillingEventsPipelineTest { assertThat(cursor.getCursorTime()).isEqualTo(expectedCursorTime); } - private static Recurring createDomainAtTime(String domainName, DateTime createTime) { + private static BillingRecurrence createDomainAtTime(String domainName, DateTime createTime) { Domain domain = persistActiveDomain(domainName, createTime); DomainHistory domainHistory = persistResource( @@ -528,7 +531,7 @@ public class ExpandRecurringBillingEventsPipelineTest { .setDomain(domain) .build()); return persistResource( - new Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setRegistrarId(domain.getCreationRegistrarId()) .setEventTime(createTime.plusYears(1)) @@ -539,5 +542,5 @@ public class ExpandRecurringBillingEventsPipelineTest { .build()); } - public interface TestOptions extends ExpandRecurringBillingEventsPipelineOptions, DirectOptions {} + public interface TestOptions extends ExpandBillingRecurrencesPipelineOptions, DirectOptions {} } diff --git a/core/src/test/java/google/registry/beam/billing/InvoicingPipelineTest.java b/core/src/test/java/google/registry/beam/billing/InvoicingPipelineTest.java index 0bfdc21ac..56b0c6570 100644 --- a/core/src/test/java/google/registry/beam/billing/InvoicingPipelineTest.java +++ b/core/src/test/java/google/registry/beam/billing/InvoicingPipelineTest.java @@ -36,11 +36,11 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedMap; import com.google.common.testing.TestLogHandler; import google.registry.beam.TestPipelineExtension; -import google.registry.model.billing.BillingEvent.Cancellation; -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.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingCancellation; +import google.registry.model.billing.BillingEvent; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.registrar.Registrar; @@ -89,9 +89,9 @@ class InvoicingPipelineTest { private static final String YEAR_MONTH = "2017-10"; private static final String INVOICE_FILE_PREFIX = "REG-INV"; - private static final ImmutableList INPUT_EVENTS = + private static final ImmutableList INPUT_EVENTS = ImmutableList.of( - BillingEvent.create( + google.registry.beam.billing.BillingEvent.create( 1, DateTime.parse("2017-10-04T00:00:00Z"), DateTime.parse("2017-10-04T00:00:00Z"), @@ -106,7 +106,7 @@ class InvoicingPipelineTest { "USD", 20.5, ""), - BillingEvent.create( + google.registry.beam.billing.BillingEvent.create( 2, DateTime.parse("2017-10-04T00:00:00Z"), DateTime.parse("2017-10-04T00:00:00Z"), @@ -121,7 +121,7 @@ class InvoicingPipelineTest { "USD", 20.5, ""), - BillingEvent.create( + google.registry.beam.billing.BillingEvent.create( 3, DateTime.parse("2017-10-02T00:00:00Z"), DateTime.parse("2017-09-29T00:00:00Z"), @@ -136,7 +136,7 @@ class InvoicingPipelineTest { "JPY", 70.0, ""), - BillingEvent.create( + google.registry.beam.billing.BillingEvent.create( 4, DateTime.parse("2017-10-04T00:00:00Z"), DateTime.parse("2017-10-04T00:00:00Z"), @@ -151,7 +151,7 @@ class InvoicingPipelineTest { "USD", 20.5, ""), - BillingEvent.create( + google.registry.beam.billing.BillingEvent.create( 5, DateTime.parse("2017-10-04T00:00:00Z"), DateTime.parse("2017-10-04T00:00:00Z"), @@ -166,7 +166,7 @@ class InvoicingPipelineTest { "USD", 0.0, "SUNRISE ANCHOR_TENANT"), - BillingEvent.create( + google.registry.beam.billing.BillingEvent.create( 6, DateTime.parse("2017-10-04T00:00:00Z"), DateTime.parse("2017-10-04T00:00:00Z"), @@ -181,7 +181,7 @@ class InvoicingPipelineTest { "USD", 0.0, ""), - BillingEvent.create( + google.registry.beam.billing.BillingEvent.create( 7, DateTime.parse("2017-10-04T00:00:00Z"), DateTime.parse("2017-10-04T00:00:00Z"), @@ -237,7 +237,7 @@ class InvoicingPipelineTest { PipelineOptionsFactory.create().as(InvoicingPipelineOptions.class); private File billingBucketUrl; - private PCollection billingEvents; + private PCollection billingEvents; private final TestLogHandler logHandler = new TestLogHandler(); private final Logger loggerToIntercept = @@ -251,7 +251,9 @@ class InvoicingPipelineTest { options.setYearMonth(YEAR_MONTH); options.setInvoiceFilePrefix(INVOICE_FILE_PREFIX); billingEvents = - pipeline.apply(Create.of(INPUT_EVENTS).withCoder(SerializableCoder.of(BillingEvent.class))); + pipeline.apply( + Create.of(INPUT_EVENTS) + .withCoder(SerializableCoder.of(google.registry.beam.billing.BillingEvent.class))); } @Test @@ -274,7 +276,8 @@ class InvoicingPipelineTest { @Test void testSuccess_readFromCloudSql() throws Exception { setupCloudSql(); - PCollection billingEvents = InvoicingPipeline.readFromCloudSql(options, pipeline); + PCollection billingEvents = + InvoicingPipeline.readFromCloudSql(options, pipeline); billingEvents = billingEvents.apply(new ChangeDomainRepo()); PAssert.that(billingEvents).containsInAnyOrder(INPUT_EVENTS); pipeline.run().waitUntilFinish(); @@ -299,8 +302,9 @@ class InvoicingPipelineTest { persistResource(test); Domain domain = persistActiveDomain("mycanadiandomain.test"); - persistOneTimeBillingEvent(25, domain, registrar, Reason.RENEW, 3, Money.of(CAD, 20.5)); - PCollection billingEvents = InvoicingPipeline.readFromCloudSql(options, pipeline); + persistBillingEvent(25, domain, registrar, Reason.RENEW, 3, Money.of(CAD, 20.5)); + PCollection billingEvents = + InvoicingPipeline.readFromCloudSql(options, pipeline); billingEvents = billingEvents.apply(new ChangeDomainRepo()); PAssert.that(billingEvents).containsInAnyOrder(INPUT_EVENTS); pipeline.run().waitUntilFinish(); @@ -352,9 +356,9 @@ class InvoicingPipelineTest { + "JOIN Registrar r ON b.clientId = r.registrarId\n" + "JOIN Domain d ON b.domainRepoId = d.repoId\n" + "JOIN Tld t ON t.tldStr = d.tld\n" - + "LEFT JOIN BillingCancellation c ON b.id = c.refOneTime\n" + + "LEFT JOIN BillingCancellation c ON b.id = c.billingEvent\n" + "LEFT JOIN BillingCancellation cr ON b.cancellationMatchingBillingEvent =" - + " cr.refRecurring\n" + + " cr.billingRecurrence\n" + "WHERE r.billingAccountMap IS NOT NULL\n" + "AND r.type = 'REAL'\n" + "AND t.invoicingEnabled IS TRUE\n" @@ -416,9 +420,9 @@ class InvoicingPipelineTest { Domain domain6 = persistActiveDomain("locked.test"); Domain domain7 = persistActiveDomain("update-prohibited.test"); - persistOneTimeBillingEvent(1, domain1, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); - persistOneTimeBillingEvent(2, domain2, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); - persistOneTimeBillingEvent( + persistBillingEvent(1, domain1, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); + persistBillingEvent(2, domain2, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); + persistBillingEvent( 3, domain3, registrar1, @@ -427,8 +431,8 @@ class InvoicingPipelineTest { Money.ofMajor(JPY, 70), DateTime.parse("2017-09-29T00:00:00.0Z"), DateTime.parse("2017-10-02T00:00:00.0Z")); - persistOneTimeBillingEvent(4, domain4, registrar2, Reason.RENEW, 1, Money.of(USD, 20.5)); - persistOneTimeBillingEvent( + persistBillingEvent(4, domain4, registrar2, Reason.RENEW, 1, Money.of(USD, 20.5)); + persistBillingEvent( 5, domain5, registrar3, @@ -439,15 +443,15 @@ class InvoicingPipelineTest { DateTime.parse("2017-10-04T00:00:00.0Z"), Flag.SUNRISE, Flag.ANCHOR_TENANT); - persistOneTimeBillingEvent(6, domain6, registrar1, Reason.SERVER_STATUS, 0, Money.of(USD, 0)); - persistOneTimeBillingEvent(7, domain7, registrar1, Reason.SERVER_STATUS, 0, Money.of(USD, 20)); + persistBillingEvent(6, domain6, registrar1, Reason.SERVER_STATUS, 0, Money.of(USD, 0)); + persistBillingEvent(7, domain7, registrar1, Reason.SERVER_STATUS, 0, Money.of(USD, 20)); // Add billing event for a non-billable registrar Registrar registrar4 = persistNewRegistrar("noBillRegistrar"); registrar4 = registrar4.asBuilder().setBillingAccountMap(null).build(); persistResource(registrar4); Domain domain8 = persistActiveDomain("non-billable.test"); - persistOneTimeBillingEvent(8, domain8, registrar4, Reason.RENEW, 3, Money.of(USD, 20.5)); + persistBillingEvent(8, domain8, registrar4, Reason.RENEW, 3, Money.of(USD, 20.5)); // Add billing event for a non-real registrar Registrar registrar5 = persistNewRegistrar("notRealRegistrar"); @@ -460,16 +464,16 @@ class InvoicingPipelineTest { .build(); persistResource(registrar5); Domain domain9 = persistActiveDomain("not-real.test"); - persistOneTimeBillingEvent(9, domain9, registrar5, Reason.RENEW, 3, Money.of(USD, 20.5)); + persistBillingEvent(9, domain9, registrar5, Reason.RENEW, 3, Money.of(USD, 20.5)); // Add billing event for a non-invoicing TLD createTld("nobill"); Domain domain10 = persistActiveDomain("test.nobill"); - persistOneTimeBillingEvent(10, domain10, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); + persistBillingEvent(10, domain10, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); // Add billing event before October 2017 Domain domain11 = persistActiveDomain("july.test"); - persistOneTimeBillingEvent( + persistBillingEvent( 11, domain11, registrar1, @@ -481,64 +485,64 @@ class InvoicingPipelineTest { // Add a billing event with a corresponding cancellation Domain domain12 = persistActiveDomain("cancel.test"); - OneTime oneTime = - persistOneTimeBillingEvent(12, domain12, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); + BillingEvent billingEvent = + persistBillingEvent(12, domain12, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); DomainHistory domainHistory = persistDomainHistory(domain12, registrar1); - Cancellation cancellation = - new Cancellation() + BillingCancellation cancellation = + new BillingCancellation() .asBuilder() .setId(1) .setRegistrarId(registrar1.getRegistrarId()) .setEventTime(DateTime.parse("2017-10-05T00:00:00.0Z")) .setBillingTime(DateTime.parse("2017-10-04T00:00:00.0Z")) - .setOneTimeEventKey(oneTime.createVKey()) + .setBillingEvent(billingEvent.createVKey()) .setTargetId(domain12.getDomainName()) .setReason(Reason.RENEW) .setDomainHistory(domainHistory) .build(); persistResource(cancellation); - // Add billing event with a corresponding recurring billing event and cancellation - Domain domain13 = persistActiveDomain("cancel-recurring.test"); - DomainHistory domainHistoryRecurring = persistDomainHistory(domain13, registrar1); + // Add billing event with a corresponding recurrence billing event and cancellation + Domain domain13 = persistActiveDomain("cancel-recurrence.test"); + DomainHistory domainHistoryRecurrence = persistDomainHistory(domain13, registrar1); - Recurring recurring = - new Recurring() + BillingRecurrence billingRecurrence = + new BillingRecurrence() .asBuilder() .setRegistrarId(registrar1.getRegistrarId()) .setRecurrenceEndTime(END_OF_TIME) .setId(1) - .setDomainHistory(domainHistoryRecurring) + .setDomainHistory(domainHistoryRecurrence) .setTargetId(domain13.getDomainName()) .setEventTime(DateTime.parse("2017-10-04T00:00:00.0Z")) .setReason(Reason.RENEW) .build(); - persistResource(recurring); - OneTime oneTimeRecurring = - persistOneTimeBillingEvent(13, domain13, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); - oneTimeRecurring = - oneTimeRecurring + persistResource(billingRecurrence); + BillingEvent billingEventRecurrence = + persistBillingEvent(13, domain13, registrar1, Reason.RENEW, 3, Money.of(USD, 20.5)); + billingEventRecurrence = + billingEventRecurrence .asBuilder() - .setCancellationMatchingBillingEvent(recurring) + .setCancellationMatchingBillingEvent(billingRecurrence) .setFlags(ImmutableSet.of(Flag.SYNTHETIC)) .setSyntheticCreationTime(DateTime.parse("2017-10-03T00:00:00.0Z")) .build(); - persistResource(oneTimeRecurring); + persistResource(billingEventRecurrence); - Cancellation cancellationRecurring = - new Cancellation() + BillingCancellation cancellationRecurrence = + new BillingCancellation() .asBuilder() .setId(2) .setRegistrarId(registrar1.getRegistrarId()) .setEventTime(DateTime.parse("2017-10-05T00:00:00.0Z")) .setBillingTime(DateTime.parse("2017-10-04T00:00:00.0Z")) - .setRecurringEventKey(recurring.createVKey()) + .setBillingRecurrence(billingRecurrence.createVKey()) .setTargetId(domain13.getDomainName()) .setReason(Reason.RENEW) - .setDomainHistory(domainHistoryRecurring) + .setDomainHistory(domainHistoryRecurrence) .build(); - persistResource(cancellationRecurring); + persistResource(cancellationRecurrence); } private static DomainHistory persistDomainHistory(Domain domain, Registrar registrar) { @@ -552,9 +556,9 @@ class InvoicingPipelineTest { return persistResource(domainHistory); } - private static OneTime persistOneTimeBillingEvent( + private static BillingEvent persistBillingEvent( int id, Domain domain, Registrar registrar, Reason reason, int years, Money money) { - return persistOneTimeBillingEvent( + return persistBillingEvent( id, domain, registrar, @@ -565,7 +569,7 @@ class InvoicingPipelineTest { DateTime.parse("2017-10-04T00:00:00.0Z")); } - private static OneTime persistOneTimeBillingEvent( + private static BillingEvent persistBillingEvent( int id, Domain domain, Registrar registrar, @@ -575,8 +579,8 @@ class InvoicingPipelineTest { DateTime eventTime, DateTime billingTime, Flag... flags) { - OneTime.Builder billingEventBuilder = - new OneTime() + BillingEvent.Builder billingEventBuilder = + new BillingEvent() .asBuilder() .setId(id) .setBillingTime(billingTime) @@ -596,18 +600,21 @@ class InvoicingPipelineTest { } private static class ChangeDomainRepo - extends PTransform, PCollection> { + extends PTransform< + PCollection, + PCollection> { private static final long serialVersionUID = 2695033474967615250L; @Override - public PCollection expand(PCollection input) { + public PCollection expand( + PCollection input) { return input.apply( "Map to invoicing key", - MapElements.into(TypeDescriptor.of(BillingEvent.class)) + MapElements.into(TypeDescriptor.of(google.registry.beam.billing.BillingEvent.class)) .via( billingEvent -> - BillingEvent.create( + google.registry.beam.billing.BillingEvent.create( billingEvent.id(), billingEvent.billingTime(), billingEvent.eventTime(), diff --git a/core/src/test/java/google/registry/flows/EppLifecycleDomainTest.java b/core/src/test/java/google/registry/flows/EppLifecycleDomainTest.java index 8fc5a567a..523526e60 100644 --- a/core/src/test/java/google/registry/flows/EppLifecycleDomainTest.java +++ b/core/src/test/java/google/registry/flows/EppLifecycleDomainTest.java @@ -38,9 +38,8 @@ import com.google.common.collect.Ordering; import com.google.common.truth.Truth; import com.google.re2j.Matcher; import com.google.re2j.Pattern; +import google.registry.model.billing.BillingBase.Reason; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.OneTime; -import google.registry.model.billing.BillingEvent.Reason; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.reporting.HistoryEntry.Type; @@ -361,24 +360,22 @@ class EppLifecycleDomainTest extends EppTestCase { "CODE", "2303", "MSG", "The domain with given ID (example.tld) doesn't exist.")); // The expected one-time billing event, that should have an associated Cancellation. - OneTime oneTimeCreateBillingEvent = makeOneTimeCreateBillingEvent(domain, createTime); - OneTime oneTimeRenewBillingEvent = makeOneTimeRenewBillingEvent(domain, renewTime); + BillingEvent createBillingEvent = makeOneTimeCreateBillingEvent(domain, createTime); + BillingEvent renewBillingEvent = makeOneTimeRenewBillingEvent(domain, renewTime); // Verify that the OneTime billing event associated with the domain creation is canceled. assertBillingEventsForResource( domain, // There should be one-time billing events for the create and the renew. - oneTimeCreateBillingEvent, - oneTimeRenewBillingEvent, + createBillingEvent, + renewBillingEvent, // There should be two ended recurring billing events, one each from the create and renew. // (The former was ended by the renew and the latter was ended by the delete.) - makeRecurringCreateBillingEvent(domain, createTime.plusYears(2), renewTime), - makeRecurringRenewBillingEvent(domain, createTime.plusYears(5), deleteTime), + makeCreateRecurrence(domain, createTime.plusYears(2), renewTime), + makeRenewRecurrence(domain, createTime.plusYears(5), deleteTime), // There should be Cancellations offsetting both of the one-times. - makeCancellationBillingEventForCreate( - domain, oneTimeCreateBillingEvent, createTime, deleteTime), - makeCancellationBillingEventForRenew( - domain, oneTimeRenewBillingEvent, renewTime, deleteTime)); + makeCancellationBillingEventForCreate(domain, createBillingEvent, createTime, deleteTime), + makeCancellationBillingEventForRenew(domain, renewBillingEvent, renewTime, deleteTime)); // Verify that the registration expiration time was set back to the creation time, because the // entire cost of registration was refunded. We have to do this through the DB instead of EPP @@ -424,16 +421,15 @@ class EppLifecycleDomainTest extends EppTestCase { "CODE", "2303", "MSG", "The domain with given ID (example.tld) doesn't exist.")); // The expected one-time billing event, that should have an associated Cancellation. - OneTime oneTimeCreateBillingEvent = makeOneTimeCreateBillingEvent(domain, createTime); + BillingEvent createBillingEvent = makeOneTimeCreateBillingEvent(domain, createTime); // Verify that the OneTime billing event associated with the domain creation is canceled. assertBillingEventsForResource( domain, // Check the existence of the expected create one-time billing event. - oneTimeCreateBillingEvent, - makeRecurringCreateBillingEvent(domain, createTime.plusYears(2), deleteTime), + createBillingEvent, + makeCreateRecurrence(domain, createTime.plusYears(2), deleteTime), // Check for the existence of a cancellation for the given one-time billing event. - makeCancellationBillingEventForCreate( - domain, oneTimeCreateBillingEvent, createTime, deleteTime)); + makeCancellationBillingEventForCreate(domain, createBillingEvent, createTime, deleteTime)); // Verify that the registration expiration time was set back to the creation time, because the // entire cost of registration was refunded. We have to do this through the DB instead of EPP @@ -495,7 +491,7 @@ class EppLifecycleDomainTest extends EppTestCase { assertBillingEventsForResource( domain, makeOneTimeCreateBillingEvent(domain, createTime), - makeRecurringCreateBillingEvent(domain, createTime.plusYears(2), deleteTime)); + makeCreateRecurrence(domain, createTime.plusYears(2), deleteTime)); assertThatLogoutSucceeds(); @@ -541,8 +537,8 @@ class EppLifecycleDomainTest extends EppTestCase { // Verify that the OneTime billing event associated with the base fee of domain registration and // is canceled and the autorenew is ended, but that the EAP fee is not canceled. - OneTime expectedCreateEapBillingEvent = - new BillingEvent.OneTime.Builder() + BillingEvent expectedCreateEapBillingEvent = + new BillingEvent.Builder() .setReason(Reason.FEE_EARLY_ACCESS) .setTargetId("example.tld") .setRegistrarId("NewRegistrar") @@ -555,17 +551,17 @@ class EppLifecycleDomainTest extends EppTestCase { .build(); // The expected one-time billing event, that should have an associated Cancellation. - OneTime expectedOneTimeCreateBillingEvent = makeOneTimeCreateBillingEvent(domain, createTime); + BillingEvent expectedCreateBillingEvent = makeOneTimeCreateBillingEvent(domain, createTime); assertBillingEventsForResource( domain, // Check for the expected create one-time billing event ... - expectedOneTimeCreateBillingEvent, + expectedCreateBillingEvent, // ... and the expected one-time EAP fee billing event ... expectedCreateEapBillingEvent, - makeRecurringCreateBillingEvent(domain, createTime.plusYears(2), deleteTime), + makeCreateRecurrence(domain, createTime.plusYears(2), deleteTime), // ... and verify that the create one-time billing event was canceled ... makeCancellationBillingEventForCreate( - domain, expectedOneTimeCreateBillingEvent, createTime, deleteTime)); + domain, expectedCreateBillingEvent, createTime, deleteTime)); // ... but there was NOT a Cancellation for the EAP fee, as this would fail if additional // billing events were present. diff --git a/core/src/test/java/google/registry/flows/EppTestCase.java b/core/src/test/java/google/registry/flows/EppTestCase.java index b9aa451d7..44b90dffc 100644 --- a/core/src/test/java/google/registry/flows/EppTestCase.java +++ b/core/src/test/java/google/registry/flows/EppTestCase.java @@ -28,10 +28,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.net.MediaType; import google.registry.flows.EppTestComponent.FakesAndMocksModule; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingCancellation; 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.BillingRecurrence; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.eppcommon.EppXmlTransformer; @@ -301,9 +302,8 @@ public class EppTestCase { } /** Makes a one-time billing event corresponding to the given domain's creation. */ - protected static BillingEvent.OneTime makeOneTimeCreateBillingEvent( - Domain domain, DateTime createTime) { - return new BillingEvent.OneTime.Builder() + protected static BillingEvent makeOneTimeCreateBillingEvent(Domain domain, DateTime createTime) { + return new BillingEvent.Builder() .setReason(Reason.CREATE) .setTargetId(domain.getDomainName()) .setRegistrarId(domain.getCurrentSponsorRegistrarId()) @@ -317,8 +317,8 @@ public class EppTestCase { } /** Makes a one-time billing event corresponding to the given domain's renewal. */ - static BillingEvent.OneTime makeOneTimeRenewBillingEvent(Domain domain, DateTime renewTime) { - return new BillingEvent.OneTime.Builder() + static BillingEvent makeOneTimeRenewBillingEvent(Domain domain, DateTime renewTime) { + return new BillingEvent.Builder() .setReason(Reason.RENEW) .setTargetId(domain.getDomainName()) .setRegistrarId(domain.getCurrentSponsorRegistrarId()) @@ -330,30 +330,30 @@ public class EppTestCase { .build(); } - /** Makes a recurring billing event corresponding to the given domain's creation. */ - static BillingEvent.Recurring makeRecurringCreateBillingEvent( + /** Makes a recurrence billing event corresponding to the given domain's creation. */ + static BillingRecurrence makeCreateRecurrence( Domain domain, DateTime eventTime, DateTime endTime) { - return makeRecurringBillingEvent( + return makeRecurrence( domain, getOnlyHistoryEntryOfType(domain, Type.DOMAIN_CREATE, DomainHistory.class), eventTime, endTime); } - /** Makes a recurring billing event corresponding to the given domain's renewal. */ - static BillingEvent.Recurring makeRecurringRenewBillingEvent( + /** Makes a recurrence billing event corresponding to the given domain's renewal. */ + static BillingRecurrence makeRenewRecurrence( Domain domain, DateTime eventTime, DateTime endTime) { - return makeRecurringBillingEvent( + return makeRecurrence( domain, getOnlyHistoryEntryOfType(domain, Type.DOMAIN_RENEW, DomainHistory.class), eventTime, endTime); } - /** Makes a recurring billing event corresponding to the given history entry. */ - protected static BillingEvent.Recurring makeRecurringBillingEvent( + /** Makes a recurrence corresponding to the given history entry. */ + protected static BillingRecurrence makeRecurrence( Domain domain, DomainHistory historyEntry, DateTime eventTime, DateTime endTime) { - return new BillingEvent.Recurring.Builder() + return new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId(domain.getDomainName()) @@ -365,13 +365,13 @@ public class EppTestCase { } /** Makes a cancellation billing event cancelling out the given domain create billing event. */ - static BillingEvent.Cancellation makeCancellationBillingEventForCreate( - Domain domain, OneTime billingEventToCancel, DateTime createTime, DateTime deleteTime) { - return new BillingEvent.Cancellation.Builder() + static BillingCancellation makeCancellationBillingEventForCreate( + Domain domain, BillingEvent billingEventToCancel, DateTime createTime, DateTime deleteTime) { + return new BillingCancellation.Builder() .setTargetId(domain.getDomainName()) .setRegistrarId(domain.getCurrentSponsorRegistrarId()) .setEventTime(deleteTime) - .setOneTimeEventKey(findKeyToActualOneTimeBillingEvent(billingEventToCancel)) + .setBillingEvent(findKeyToActualOneTimeBillingEvent(billingEventToCancel)) .setBillingTime(createTime.plus(Tld.get(domain.getTld()).getAddGracePeriodLength())) .setReason(Reason.CREATE) .setDomainHistory( @@ -380,13 +380,13 @@ public class EppTestCase { } /** Makes a cancellation billing event cancelling out the given domain renew billing event. */ - static BillingEvent.Cancellation makeCancellationBillingEventForRenew( - Domain domain, OneTime billingEventToCancel, DateTime renewTime, DateTime deleteTime) { - return new BillingEvent.Cancellation.Builder() + static BillingCancellation makeCancellationBillingEventForRenew( + Domain domain, BillingEvent billingEventToCancel, DateTime renewTime, DateTime deleteTime) { + return new BillingCancellation.Builder() .setTargetId(domain.getDomainName()) .setRegistrarId(domain.getCurrentSponsorRegistrarId()) .setEventTime(deleteTime) - .setOneTimeEventKey(findKeyToActualOneTimeBillingEvent(billingEventToCancel)) + .setBillingEvent(findKeyToActualOneTimeBillingEvent(billingEventToCancel)) .setBillingTime(renewTime.plus(Tld.get(domain.getTld()).getRenewGracePeriodLength())) .setReason(Reason.RENEW) .setDomainHistory( @@ -403,9 +403,10 @@ public class EppTestCase { * This is necessary because the ID will be different even though all the rest of the fields are * the same. */ - private static VKey findKeyToActualOneTimeBillingEvent(OneTime expectedBillingEvent) { - Optional actualCreateBillingEvent = - loadAllOf(BillingEvent.OneTime.class).stream() + private static VKey findKeyToActualOneTimeBillingEvent( + BillingEvent expectedBillingEvent) { + Optional actualCreateBillingEvent = + loadAllOf(BillingEvent.class).stream() .filter( b -> Objects.equals( diff --git a/core/src/test/java/google/registry/flows/FlowTestCase.java b/core/src/test/java/google/registry/flows/FlowTestCase.java index 70ef50889..3854bf962 100644 --- a/core/src/test/java/google/registry/flows/FlowTestCase.java +++ b/core/src/test/java/google/registry/flows/FlowTestCase.java @@ -34,7 +34,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.ObjectArrays; import google.registry.flows.EppTestComponent.FakesAndMocksModule; import google.registry.flows.picker.FlowPicker; -import google.registry.model.billing.BillingEvent; +import google.registry.model.billing.BillingBase; import google.registry.model.domain.GracePeriod; import google.registry.model.eppcommon.ProtocolDefinition; import google.registry.model.eppinput.EppInput; @@ -169,10 +169,10 @@ public abstract class FlowTestCase { * the grace periods and the IDs on the billing events (by setting them all to the same dummy * values), since they will vary between instantiations even when the other data is the same. */ - private static ImmutableMap canonicalizeGracePeriods( - ImmutableMap gracePeriods) { - ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); - for (Map.Entry entry : gracePeriods.entrySet()) { + private static ImmutableMap canonicalizeGracePeriods( + ImmutableMap gracePeriods) { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + for (Map.Entry entry : gracePeriods.entrySet()) { builder.put( GracePeriod.create( entry.getKey().getType(), @@ -186,7 +186,7 @@ public abstract class FlowTestCase { return builder.build(); } - private static BillingEvent expandGracePeriod(GracePeriod gracePeriod) { + private static BillingBase expandGracePeriod(GracePeriod gracePeriod) { assertWithMessage("Billing event is present for grace period: " + gracePeriod) .that(gracePeriod.hasBillingEvent()) .isTrue(); @@ -194,8 +194,7 @@ public abstract class FlowTestCase { () -> tm().loadByKey( firstNonNull( - gracePeriod.getOneTimeBillingEvent(), - gracePeriod.getRecurringBillingEvent()))); + gracePeriod.getBillingEvent(), gracePeriod.getBillingRecurrence()))); } /** @@ -204,7 +203,7 @@ public abstract class FlowTestCase { * on the grace periods and IDs on the billing events are ignored. */ protected static void assertGracePeriods( - Iterable actual, ImmutableMap expected) { + Iterable actual, ImmutableMap expected) { assertThat(canonicalizeGracePeriods(Maps.toMap(actual, FlowTestCase::expandGracePeriod))) .isEqualTo(canonicalizeGracePeriods(expected)); } diff --git a/core/src/test/java/google/registry/flows/domain/DomainCheckFlowTest.java b/core/src/test/java/google/registry/flows/domain/DomainCheckFlowTest.java index 705879d01..04c7fea95 100644 --- a/core/src/test/java/google/registry/flows/domain/DomainCheckFlowTest.java +++ b/core/src/test/java/google/registry/flows/domain/DomainCheckFlowTest.java @@ -14,9 +14,9 @@ package google.registry.flows.domain; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.DEFAULT; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.NONPREMIUM; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.SPECIFIED; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.DEFAULT; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.NONPREMIUM; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.SPECIFIED; import static google.registry.model.domain.token.AllocationToken.TokenType.DEFAULT_PROMO; import static google.registry.model.domain.token.AllocationToken.TokenType.SINGLE_USE; import static google.registry.model.domain.token.AllocationToken.TokenType.UNLIMITED_USE; @@ -70,9 +70,9 @@ import google.registry.flows.domain.DomainFlowUtils.TrailingDashException; import google.registry.flows.domain.DomainFlowUtils.TransfersAreAlwaysForOneYearException; import google.registry.flows.domain.DomainFlowUtils.UnknownFeeCommandException; import google.registry.flows.exceptions.TooManyResourceChecksException; -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.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.domain.fee.FeeQueryCommandExtensionItem.CommandName; @@ -1612,9 +1612,9 @@ class DomainCheckFlowTest extends ResourceCheckFlowTestCase expectedBillingFlags) throws Exception { + String domainTld, ImmutableSet expectedBillingFlags) throws Exception { assertSuccessfulCreate(domainTld, expectedBillingFlags, null); } private void assertSuccessfulCreate( String domainTld, - ImmutableSet expectedBillingFlags, + ImmutableSet expectedBillingFlags, @Nullable AllocationToken allocationToken) throws Exception { Domain domain = reloadResourceByForeignKey(); @@ -311,9 +313,9 @@ class DomainCreateFlowTest extends ResourceFlowTestCase expectedBillingEvents = - new ImmutableSet.Builder().add(createBillingEvent).add(renewBillingEvent); + ImmutableSet.Builder expectedBillingEvents = + new ImmutableSet.Builder().add(createBillingEvent).add(renewBillingEvent); // If EAP is applied, a billing event for EAP should be present. // EAP fees are bypassed for anchor tenant domains. if (!isAnchorTenant && !eapFee.isZero()) { - BillingEvent.OneTime eapBillingEvent = - new BillingEvent.OneTime.Builder() + BillingEvent eapBillingEvent = + new BillingEvent.Builder() .setReason(Reason.FEE_EARLY_ACCESS) .setTargetId(getUniqueIdFromCommand()) .setRegistrarId("TheRegistrar") @@ -1570,8 +1572,8 @@ class DomainCreateFlowTest extends ResourceFlowTestCase substitutions) throws Exception { // Persist the billing event so it can be retrieved for cancellation generation and checking. setUpSuccessfulTest(); - BillingEvent.OneTime renewBillingEvent = + BillingEvent renewBillingEvent = persistResource(createBillingEvent(Reason.RENEW, Money.of(USD, 456))); setUpGracePeriods( GracePeriod.forBillingEvent(GracePeriodStatus.RENEW, domain.getRepoId(), renewBillingEvent), @@ -720,7 +720,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase { .setModificationTime(clock.nowUtc()) .setRegistrarId(domain.getCreationRegistrarId()) .build()); - Recurring renewEvent = + BillingRecurrence renewEvent = persistResource( - new Recurring.Builder() + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId(getUniqueIdFromCommand()) @@ -395,18 +395,18 @@ class DomainInfoFlowTest extends ResourceFlowTestCase { .setRecurrenceEndTime(END_OF_TIME) .setDomainHistory(historyEntry) .build()); - VKey recurringVKey = renewEvent.createVKey(); + VKey recurrenceVKey = renewEvent.createVKey(); // Add an AUTO_RENEW grace period to the saved resource. persistResource( domain .asBuilder() .addGracePeriod( - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), clock.nowUtc().plusDays(1), "TheRegistrar", - recurringVKey)) + recurrenceVKey)) .build()); doSuccessfulTest("domain_info_response_autorenewperiod.xml", false, ImmutableMap.of(), true); } diff --git a/core/src/test/java/google/registry/flows/domain/DomainPricingLogicTest.java b/core/src/test/java/google/registry/flows/domain/DomainPricingLogicTest.java index 43b6f9497..0b972ac75 100644 --- a/core/src/test/java/google/registry/flows/domain/DomainPricingLogicTest.java +++ b/core/src/test/java/google/registry/flows/domain/DomainPricingLogicTest.java @@ -15,10 +15,10 @@ package google.registry.flows.domain; import static com.google.common.truth.Truth.assertThat; -import static google.registry.model.billing.BillingEvent.Flag.AUTO_RENEW; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.DEFAULT; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.NONPREMIUM; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.SPECIFIED; +import static google.registry.model.billing.BillingBase.Flag.AUTO_RENEW; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.DEFAULT; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.NONPREMIUM; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.SPECIFIED; import static google.registry.model.domain.fee.BaseFee.FeeType.RENEW; import static google.registry.model.domain.token.AllocationToken.TokenType.SINGLE_USE; import static google.registry.model.reporting.HistoryEntry.Type.DOMAIN_CREATE; @@ -40,10 +40,9 @@ import google.registry.flows.HttpSessionMetadata; import google.registry.flows.SessionMetadata; import google.registry.flows.custom.DomainPricingCustomLogic; import google.registry.flows.domain.DomainPricingLogic.AllocationTokenInvalidForPremiumNameException; -import google.registry.model.billing.BillingEvent; -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.billing.BillingBase.Reason; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.domain.fee.Fee; @@ -98,8 +97,8 @@ public class DomainPricingLogicTest { .build()); } - /** helps to set up the domain info and returns a recurring billing event for testing */ - private Recurring persistDomainAndSetRecurringBillingEvent( + /** helps to set up the domain info and returns a recurrence billing event for testing */ + private BillingRecurrence persistDomainAndSetRecurrence( String domainName, RenewalPriceBehavior renewalPriceBehavior, Optional renewalPrice) { domain = persistResource( @@ -115,9 +114,9 @@ public class DomainPricingLogicTest { .setModificationTime(DateTime.parse("1999-01-05T00:00:00Z")) .setDomain(domain) .build()); - Recurring recurring = + BillingRecurrence billingRecurrence = persistResource( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(historyEntry) .setRegistrarId(domain.getCreationRegistrarId()) .setEventTime(DateTime.parse("1999-01-05T00:00:00Z")) @@ -129,8 +128,9 @@ public class DomainPricingLogicTest { .setRecurrenceEndTime(END_OF_TIME) .setTargetId(domain.getDomainName()) .build()); - persistResource(domain.asBuilder().setAutorenewBillingEvent(recurring.createVKey()).build()); - return recurring; + persistResource( + domain.asBuilder().setAutorenewBillingEvent(billingRecurrence.createVKey()).build()); + return billingRecurrence; } @Test @@ -193,8 +193,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( - "premium.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", DEFAULT, Optional.empty()), Optional.empty())) .isEqualTo( new FeesAndCredits.Builder() @@ -220,8 +219,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( - "premium.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", DEFAULT, Optional.empty()), Optional.of(allocationToken))) .isEqualTo( new FeesAndCredits.Builder() @@ -250,8 +248,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( - "premium.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", DEFAULT, Optional.empty()), Optional.of(allocationToken))); } @@ -263,8 +260,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( - "premium.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", DEFAULT, Optional.empty()), Optional.empty())) .isEqualTo( new FeesAndCredits.Builder() @@ -291,8 +287,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( - "premium.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", DEFAULT, Optional.empty()), Optional.of(allocationToken))) .isEqualTo( new FeesAndCredits.Builder() @@ -322,8 +317,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( - "premium.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", DEFAULT, Optional.empty()), Optional.of(allocationToken))); } @@ -336,8 +330,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( - "standard.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("standard.example", DEFAULT, Optional.empty()), Optional.empty())) .isEqualTo( new FeesAndCredits.Builder() @@ -363,8 +356,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( - "standard.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("standard.example", DEFAULT, Optional.empty()), Optional.of(allocationToken))) .isEqualTo( new FeesAndCredits.Builder() @@ -382,8 +374,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( - "standard.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("standard.example", DEFAULT, Optional.empty()), Optional.empty())) .isEqualTo( new FeesAndCredits.Builder() @@ -410,8 +401,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( - "standard.example", DEFAULT, Optional.empty()), + persistDomainAndSetRecurrence("standard.example", DEFAULT, Optional.empty()), Optional.of(allocationToken))) .isEqualTo( new FeesAndCredits.Builder() @@ -429,8 +419,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( - "premium.example", NONPREMIUM, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", NONPREMIUM, Optional.empty()), Optional.empty())) .isEqualTo( new FeesAndCredits.Builder() @@ -457,8 +446,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( - "premium.example", NONPREMIUM, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", NONPREMIUM, Optional.empty()), Optional.of(allocationToken))) .isEqualTo( new FeesAndCredits.Builder() @@ -476,8 +464,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( - "premium.example", NONPREMIUM, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", NONPREMIUM, Optional.empty()), Optional.empty())) .isEqualTo( new FeesAndCredits.Builder() @@ -505,8 +492,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( - "premium.example", NONPREMIUM, Optional.empty()), + persistDomainAndSetRecurrence("premium.example", NONPREMIUM, Optional.empty()), Optional.of(allocationToken))) .isEqualTo( new FeesAndCredits.Builder() @@ -524,8 +510,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( - "standard.example", NONPREMIUM, Optional.empty()), + persistDomainAndSetRecurrence("standard.example", NONPREMIUM, Optional.empty()), Optional.empty())) .isEqualTo( new FeesAndCredits.Builder() @@ -543,8 +528,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( - "standard.example", NONPREMIUM, Optional.empty()), + persistDomainAndSetRecurrence("standard.example", NONPREMIUM, Optional.empty()), Optional.empty())) .isEqualTo( new FeesAndCredits.Builder() @@ -562,7 +546,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "standard.example", SPECIFIED, Optional.of(Money.of(USD, 1))), Optional.empty())) .isEqualTo( @@ -590,7 +574,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "standard.example", SPECIFIED, Optional.of(Money.of(USD, 1))), Optional.of(allocationToken))) @@ -621,7 +605,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "standard.example", SPECIFIED, Optional.of(Money.of(USD, 1))), Optional.of(allocationToken))) @@ -632,7 +616,7 @@ public class DomainPricingLogicTest { .addFeeOrCredit(Fee.create(Money.of(USD, 1).getAmount(), RENEW, false)) .build()); assertThat( - Iterables.getLast(DatabaseHelper.loadAllOf(BillingEvent.Recurring.class)) + Iterables.getLast(DatabaseHelper.loadAllOf(BillingRecurrence.class)) .getRenewalPriceBehavior()) .isEqualTo(SPECIFIED); } @@ -646,7 +630,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "standard.example", SPECIFIED, Optional.of(Money.of(USD, 1))), Optional.empty())) .isEqualTo( @@ -674,7 +658,7 @@ public class DomainPricingLogicTest { "standard.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "standard.example", SPECIFIED, Optional.of(Money.of(USD, 1))), Optional.of(allocationToken))) .isEqualTo( @@ -693,7 +677,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 1, - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "premium.example", SPECIFIED, Optional.of(Money.of(USD, 17))), Optional.empty())) .isEqualTo( @@ -712,7 +696,7 @@ public class DomainPricingLogicTest { "premium.example", clock.nowUtc(), 5, - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "premium.example", SPECIFIED, Optional.of(Money.of(USD, 17))), Optional.empty())) .isEqualTo( @@ -764,8 +748,7 @@ public class DomainPricingLogicTest { registry, "standard.example", clock.nowUtc(), - persistDomainAndSetRecurringBillingEvent( - "standard.example", DEFAULT, Optional.empty()))) + persistDomainAndSetRecurrence("standard.example", DEFAULT, Optional.empty()))) .isEqualTo( new FeesAndCredits.Builder() .setCurrency(USD) @@ -780,8 +763,7 @@ public class DomainPricingLogicTest { registry, "premium.example", clock.nowUtc(), - persistDomainAndSetRecurringBillingEvent( - "premium.example", DEFAULT, Optional.empty()))) + persistDomainAndSetRecurrence("premium.example", DEFAULT, Optional.empty()))) .isEqualTo( new FeesAndCredits.Builder() .setCurrency(USD) @@ -797,8 +779,7 @@ public class DomainPricingLogicTest { registry, "standard.example", clock.nowUtc(), - persistDomainAndSetRecurringBillingEvent( - "standard.example", NONPREMIUM, Optional.empty()))) + persistDomainAndSetRecurrence("standard.example", NONPREMIUM, Optional.empty()))) .isEqualTo( new FeesAndCredits.Builder() .setCurrency(USD) @@ -814,8 +795,7 @@ public class DomainPricingLogicTest { registry, "premium.example", clock.nowUtc(), - persistDomainAndSetRecurringBillingEvent( - "premium.example", NONPREMIUM, Optional.empty()))) + persistDomainAndSetRecurrence("premium.example", NONPREMIUM, Optional.empty()))) .isEqualTo( new FeesAndCredits.Builder() .setCurrency(USD) @@ -831,7 +811,7 @@ public class DomainPricingLogicTest { registry, "standard.example", clock.nowUtc(), - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "standard.example", SPECIFIED, Optional.of(Money.of(USD, 1.23))))) .isEqualTo( new FeesAndCredits.Builder() @@ -848,7 +828,7 @@ public class DomainPricingLogicTest { registry, "premium.example", clock.nowUtc(), - persistDomainAndSetRecurringBillingEvent( + persistDomainAndSetRecurrence( "premium.example", SPECIFIED, Optional.of(Money.of(USD, 1.23))))) .isEqualTo( new FeesAndCredits.Builder() diff --git a/core/src/test/java/google/registry/flows/domain/DomainRenewFlowTest.java b/core/src/test/java/google/registry/flows/domain/DomainRenewFlowTest.java index fb4d5a432..56a40581b 100644 --- a/core/src/test/java/google/registry/flows/domain/DomainRenewFlowTest.java +++ b/core/src/test/java/google/registry/flows/domain/DomainRenewFlowTest.java @@ -17,9 +17,9 @@ package google.registry.flows.domain; import static com.google.common.truth.Truth.assertThat; import static com.google.common.truth.Truth8.assertThat; import static google.registry.flows.domain.DomainTransferFlowTestCase.persistWithPendingTransfer; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.DEFAULT; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.NONPREMIUM; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.SPECIFIED; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.DEFAULT; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.NONPREMIUM; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.SPECIFIED; import static google.registry.model.domain.token.AllocationToken.TokenType.DEFAULT_PROMO; import static google.registry.model.domain.token.AllocationToken.TokenType.PACKAGE; import static google.registry.model.domain.token.AllocationToken.TokenType.SINGLE_USE; @@ -80,10 +80,11 @@ import google.registry.flows.domain.token.AllocationTokenFlowUtils.InvalidAlloca import google.registry.flows.domain.token.AllocationTokenFlowUtils.MissingRemovePackageTokenOnPackageDomainException; import google.registry.flows.domain.token.AllocationTokenFlowUtils.RemovePackageTokenOnNonPackageDomainException; import google.registry.flows.exceptions.ResourceStatusProhibitsOperationException; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; 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.billing.BillingRecurrence; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.domain.GracePeriod; @@ -162,8 +163,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase .setModificationTime(clock.nowUtc()) .setRegistrarId(domain.getCreationRegistrarId()) .build(); - BillingEvent.Recurring autorenewEvent = - new BillingEvent.Recurring.Builder() + BillingRecurrence autorenewEvent = + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId(getUniqueIdFromCommand()) @@ -273,8 +274,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase .and() .hasLastEppUpdateRegistrarId(renewalClientId); assertAboutHistoryEntries().that(historyEntryDomainRenew).hasPeriodYears(renewalYears); - BillingEvent.OneTime renewBillingEvent = - new BillingEvent.OneTime.Builder() + BillingEvent renewBillingEvent = + new BillingEvent.Builder() .setReason(Reason.RENEW) .setTargetId(getUniqueIdFromCommand()) .setRegistrarId(renewalClientId) @@ -286,7 +287,7 @@ class DomainRenewFlowTest extends ResourceFlowTestCase .build(); assertBillingEvents( renewBillingEvent, - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setRenewalPriceBehavior(renewalPriceBehavior) .setRenewalPrice(renewalPrice) @@ -299,7 +300,7 @@ class DomainRenewFlowTest extends ResourceFlowTestCase getOnlyHistoryEntryOfType( domain, HistoryEntry.Type.DOMAIN_CREATE, DomainHistory.class)) .build(), - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setRenewalPriceBehavior(renewalPriceBehavior) .setRenewalPrice(renewalPrice) @@ -616,8 +617,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); assertThat(DatabaseHelper.loadByEntity(allocationToken).getRedemptionHistoryId()).isPresent(); - BillingEvent.OneTime billingEvent = - Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + BillingEvent billingEvent = + Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("example.tld"); assertThat(billingEvent.getAllocationToken().get().getKey()) .isEqualTo(allocationToken.getToken()); @@ -641,8 +642,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase loadFile( "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); - BillingEvent.OneTime billingEvent = - Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + BillingEvent billingEvent = + Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("example.tld"); assertThat(billingEvent.getAllocationToken().get().getKey()).isEqualTo("abc123"); assertThat(billingEvent.getCost()).isEqualTo(Money.of(USD, 16.5)); @@ -655,7 +656,7 @@ class DomainRenewFlowTest extends ResourceFlowTestCase loadFile( "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "other-example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); - billingEvent = Iterables.getLast(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + billingEvent = Iterables.getLast(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("other-example.tld"); assertThat(billingEvent.getAllocationToken().get().getKey()).isEqualTo("abc123"); assertThat(billingEvent.getCost()).isEqualTo(Money.of(USD, 16.5)); @@ -681,8 +682,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); assertThat(DatabaseHelper.loadByEntity(allocationToken).getRedemptionHistoryId()).isPresent(); - BillingEvent.OneTime billingEvent = - Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + BillingEvent billingEvent = + Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("example.tld"); assertThat(billingEvent.getAllocationToken().get().getKey()) .isEqualTo(allocationToken.getToken()); @@ -1404,8 +1405,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase loadFile( "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); - BillingEvent.OneTime billingEvent = - Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + BillingEvent billingEvent = + Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("example.tld"); assertThat(billingEvent.getAllocationToken().get().getKey()) .isEqualTo(defaultToken1.getToken()); @@ -1455,8 +1456,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase loadFile( "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); - BillingEvent.OneTime billingEvent = - Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + BillingEvent billingEvent = + Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("example.tld"); assertThat(billingEvent.getAllocationToken().get().getKey()) .isEqualTo(allocationToken.getToken()); @@ -1498,8 +1499,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase loadFile( "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); - BillingEvent.OneTime billingEvent = - Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + BillingEvent billingEvent = + Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("example.tld"); assertThat(billingEvent.getAllocationToken()).isEmpty(); assertThat(billingEvent.getCost()).isEqualTo(Money.of(USD, 22)); @@ -1550,8 +1551,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase loadFile( "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); - BillingEvent.OneTime billingEvent = - Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + BillingEvent billingEvent = + Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("example.tld"); assertThat(billingEvent.getAllocationToken().get().getKey()) .isEqualTo(defaultToken2.getToken()); @@ -1592,8 +1593,8 @@ class DomainRenewFlowTest extends ResourceFlowTestCase loadFile( "domain_renew_response.xml", ImmutableMap.of("DOMAIN", "example.tld", "EXDATE", "2002-04-03T22:00:00.0Z"))); - BillingEvent.OneTime billingEvent = - Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class)); + BillingEvent billingEvent = + Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class)); assertThat(billingEvent.getTargetId()).isEqualTo("example.tld"); assertThat(billingEvent.getAllocationToken().get().getKey()) .isEqualTo(defaultToken2.getToken()); diff --git a/core/src/test/java/google/registry/flows/domain/DomainRestoreRequestFlowTest.java b/core/src/test/java/google/registry/flows/domain/DomainRestoreRequestFlowTest.java index 84248373d..ed4b68359 100644 --- a/core/src/test/java/google/registry/flows/domain/DomainRestoreRequestFlowTest.java +++ b/core/src/test/java/google/registry/flows/domain/DomainRestoreRequestFlowTest.java @@ -58,9 +58,10 @@ import google.registry.flows.domain.DomainFlowUtils.RegistrarMustBeActiveForThis import google.registry.flows.domain.DomainFlowUtils.UnsupportedFeeAttributeException; import google.registry.flows.domain.DomainRestoreRequestFlow.DomainNotEligibleForRestoreException; import google.registry.flows.domain.DomainRestoreRequestFlow.RestoreCommandIncludesChangesException; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; 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.BillingRecurrence; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.domain.GracePeriod; @@ -200,10 +201,9 @@ class DomainRestoreRequestFlowTest extends ResourceFlowTestCase existingAutorenewEvent = domain.getAutorenewBillingEvent(); + VKey existingAutorenewEvent = domain.getAutorenewBillingEvent(); // Set domain to have auto-renewed just before the transfer request, so that it will have an // active autorenew grace period spanning the entire transfer window. DateTime autorenewTime = clock.nowUtc().minusDays(1); @@ -859,7 +860,7 @@ class DomainTransferApproveFlowTest transferDataBuilder.setTransferPeriod(Period.create(0, Unit.YEARS)).build()) .setRegistrationExpirationTime(expirationTime) .addGracePeriod( - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), autorenewTime.plus(Tld.get("tld").getAutoRenewGracePeriodLength()), diff --git a/core/src/test/java/google/registry/flows/domain/DomainTransferFlowTestCase.java b/core/src/test/java/google/registry/flows/domain/DomainTransferFlowTestCase.java index 1dbd4ac65..d337fb78d 100644 --- a/core/src/test/java/google/registry/flows/domain/DomainTransferFlowTestCase.java +++ b/core/src/test/java/google/registry/flows/domain/DomainTransferFlowTestCase.java @@ -32,9 +32,10 @@ import com.google.common.collect.ImmutableSet; import google.registry.flows.Flow; import google.registry.flows.ResourceFlowTestCase; import google.registry.model.EppResource; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; 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.BillingRecurrence; import google.registry.model.contact.Contact; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; @@ -129,7 +130,7 @@ abstract class DomainTransferFlowTestCase getOnlyHistoryEntryOfType(domain, DOMAIN_CREATE, DomainHistory.class); } - BillingEvent.OneTime getBillingEventForImplicitTransfer() { + BillingEvent getBillingEventForImplicitTransfer() { DomainHistory historyEntry = getOnlyHistoryEntryOfType( domain, HistoryEntry.Type.DOMAIN_TRANSFER_REQUEST, DomainHistory.class); @@ -138,8 +139,8 @@ abstract class DomainTransferFlowTestCase } /** Get the autorenew event that the losing client will have after a SERVER_APPROVED transfer. */ - BillingEvent.Recurring getLosingClientAutorenewEvent() { - return new BillingEvent.Recurring.Builder() + BillingRecurrence getLosingClientAutorenewEvent() { + return new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId(domain.getDomainName()) @@ -151,8 +152,8 @@ abstract class DomainTransferFlowTestCase } /** Get the autorenew event that the gaining client will have after a SERVER_APPROVED transfer. */ - BillingEvent.Recurring getGainingClientAutorenewEvent() { - return new BillingEvent.Recurring.Builder() + BillingRecurrence getGainingClientAutorenewEvent() { + return new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId(domain.getDomainName()) diff --git a/core/src/test/java/google/registry/flows/domain/DomainTransferRequestFlowTest.java b/core/src/test/java/google/registry/flows/domain/DomainTransferRequestFlowTest.java index 44599a5eb..bf536ccc3 100644 --- a/core/src/test/java/google/registry/flows/domain/DomainTransferRequestFlowTest.java +++ b/core/src/test/java/google/registry/flows/domain/DomainTransferRequestFlowTest.java @@ -92,9 +92,12 @@ import google.registry.flows.exceptions.ObjectAlreadySponsoredException; import google.registry.flows.exceptions.ResourceStatusProhibitsOperationException; import google.registry.flows.exceptions.TransferPeriodMustBeOneYearException; import google.registry.flows.exceptions.TransferPeriodZeroAndFeeTransferExtensionException; +import google.registry.model.billing.BillingBase; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; +import google.registry.model.billing.BillingCancellation; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.Reason; -import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.contact.ContactAuthInfo; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainAuthInfo; @@ -258,20 +261,20 @@ class DomainTransferRequestFlowTest Optional transferCost, ImmutableSet originalGracePeriods, boolean expectTransferBillingEvent, - BillingEvent.Cancellation.Builder... extraExpectedBillingEvents) { + BillingCancellation.Builder... extraExpectedBillingEvents) { Tld registry = Tld.get(domain.getTld()); final DomainHistory historyEntryTransferRequest = getOnlyHistoryEntryOfType(domain, DOMAIN_TRANSFER_REQUEST, DomainHistory.class); // Construct the billing events we expect to exist, starting with the (optional) billing // event for the transfer itself. - Optional optionalTransferBillingEvent; + Optional optionalTransferBillingEvent; if (expectTransferBillingEvent) { // For normal transfers, a BillingEvent should be created AUTOMATIC_TRANSFER_DAYS in the // future, for the case when the transfer is implicitly acked. optionalTransferBillingEvent = Optional.of( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.TRANSFER) .setTargetId(domain.getDomainName()) .setEventTime(implicitTransferTime) @@ -290,29 +293,29 @@ class DomainTransferRequestFlowTest // all of the other transfer flow tests happen on day 3 of the transfer, but the initial // request by definition takes place on day 1, so we need to edit the times in the // autorenew events from the base test case. - BillingEvent.Recurring losingClientAutorenew = + BillingRecurrence losingClientAutorenew = getLosingClientAutorenewEvent() .asBuilder() .setRecurrenceEndTime(implicitTransferTime) .build(); - BillingEvent.Recurring gainingClientAutorenew = + BillingRecurrence gainingClientAutorenew = getGainingClientAutorenewEvent() .asBuilder() .setEventTime(expectedExpirationTime) .setRecurrenceLastExpansion(expectedExpirationTime.minusYears(1)) .build(); // Construct extra billing events expected by the specific test. - ImmutableSet extraBillingEvents = + ImmutableSet extraBillingBases = Stream.of(extraExpectedBillingEvents) .map(builder -> builder.setDomainHistory(historyEntryTransferRequest).build()) .collect(toImmutableSet()); // Assert that the billing events we constructed above actually exist in the database. - ImmutableSet expectedBillingEvents = + ImmutableSet expectedBillingBases = Streams.concat( Stream.of(losingClientAutorenew, gainingClientAutorenew), optionalToStream(optionalTransferBillingEvent)) .collect(toImmutableSet()); - assertBillingEvents(Sets.union(expectedBillingEvents, extraBillingEvents)); + assertBillingEvents(Sets.union(expectedBillingBases, extraBillingBases)); // Assert that the domain's TransferData server-approve billing events match the above. if (expectTransferBillingEvent) { assertBillingEventsEqual( @@ -326,16 +329,16 @@ class DomainTransferRequestFlowTest gainingClientAutorenew); // Assert that the full set of server-approve billing events is exactly the extra ones plus // the transfer billing event (if present) and the gaining client autorenew. - ImmutableSet expectedServeApproveBillingEvents = + ImmutableSet expectedServeApproveBillingBases = Streams.concat( Stream.of(gainingClientAutorenew), optionalToStream(optionalTransferBillingEvent)) .collect(toImmutableSet()); assertBillingEventsEqual( Iterables.filter( - loadByKeys(domain.getTransferData().getServerApproveEntities()), BillingEvent.class), - Sets.union(expectedServeApproveBillingEvents, extraBillingEvents)); + loadByKeys(domain.getTransferData().getServerApproveEntities()), BillingBase.class), + Sets.union(expectedServeApproveBillingBases, extraBillingBases)); // The domain's autorenew billing event should still point to the losing client's event. - BillingEvent.Recurring domainAutorenewEvent = loadByKey(domain.getAutorenewBillingEvent()); + BillingRecurrence domainAutorenewEvent = loadByKey(domain.getAutorenewBillingEvent()); assertThat(domainAutorenewEvent.getRegistrarId()).isEqualTo("TheRegistrar"); assertThat(domainAutorenewEvent.getRecurrenceEndTime()).isEqualTo(implicitTransferTime); // The original grace periods should remain untouched. @@ -468,7 +471,7 @@ class DomainTransferRequestFlowTest DateTime expectedExpirationTime, Map substitutions, Optional transferCost, - BillingEvent.Cancellation.Builder... extraExpectedBillingEvents) + BillingCancellation.Builder... extraExpectedBillingEvents) throws Exception { setEppInput(commandFilename, substitutions); ImmutableSet originalGracePeriods = domain.getGracePeriods(); @@ -530,7 +533,7 @@ class DomainTransferRequestFlowTest String commandFilename, String expectedXmlFilename, DateTime expectedExpirationTime, - BillingEvent.Cancellation.Builder... extraExpectedBillingEvents) + BillingCancellation.Builder... extraExpectedBillingEvents) throws Exception { doSuccessfulTest( commandFilename, @@ -568,7 +571,7 @@ class DomainTransferRequestFlowTest Optional transferCost, Period expectedPeriod, Duration expectedAutomaticTransferLength, - BillingEvent.Cancellation.Builder... extraExpectedBillingEvents) + BillingCancellation.Builder... extraExpectedBillingEvents) throws Exception { eppRequestSource = EppRequestSource.TOOL; setEppInput(commandFilename, substitutions); @@ -952,7 +955,7 @@ class DomainTransferRequestFlowTest @Test void testSuccess_superuserExtension_zeroPeriod_autorenewGraceActive() throws Exception { setupDomain("example", "tld"); - VKey existingAutorenewEvent = domain.getAutorenewBillingEvent(); + VKey existingAutorenewEvent = domain.getAutorenewBillingEvent(); // Set domain to have auto-renewed just before the transfer request, so that it will have an // active autorenew grace period spanning the entire transfer window. DateTime autorenewTime = clock.nowUtc().minusDays(1); @@ -963,7 +966,7 @@ class DomainTransferRequestFlowTest .asBuilder() .setRegistrationExpirationTime(expirationTime) .addGracePeriod( - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), autorenewTime.plus(Tld.get("tld").getAutoRenewGracePeriodLength()), @@ -1114,7 +1117,7 @@ class DomainTransferRequestFlowTest .asBuilder() .setRegistrationExpirationTime(expirationTime) .addGracePeriod( - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), autorenewTime.plus(Tld.get("tld").getAutoRenewGracePeriodLength()), @@ -1133,7 +1136,7 @@ class DomainTransferRequestFlowTest @Test void testSuccess_autorenewGraceActive_throughoutTransferWindow() throws Exception { setupDomain("example", "tld"); - VKey existingAutorenewEvent = domain.getAutorenewBillingEvent(); + VKey existingAutorenewEvent = domain.getAutorenewBillingEvent(); // Set domain to have auto-renewed just before the transfer request, so that it will have an // active autorenew grace period spanning the entire transfer window. DateTime autorenewTime = clock.nowUtc().minusDays(1); @@ -1144,7 +1147,7 @@ class DomainTransferRequestFlowTest .asBuilder() .setRegistrationExpirationTime(expirationTime) .addGracePeriod( - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), autorenewTime.plus(Tld.get("tld").getAutoRenewGracePeriodLength()), @@ -1158,7 +1161,7 @@ class DomainTransferRequestFlowTest "domain_transfer_request.xml", "domain_transfer_request_response_autorenew_grace_throughout_transfer_window.xml", expirationTime, - new BillingEvent.Cancellation.Builder() + new BillingCancellation.Builder() .setReason(Reason.RENEW) .setTargetId("example.tld") .setRegistrarId("TheRegistrar") @@ -1166,13 +1169,13 @@ class DomainTransferRequestFlowTest .setEventTime(clock.nowUtc().plus(Tld.get("tld").getAutomaticTransferLength())) .setBillingTime(autorenewTime.plus(Tld.get("tld").getAutoRenewGracePeriodLength())) // The cancellation should refer to the old autorenew billing event. - .setRecurringEventKey(existingAutorenewEvent)); + .setBillingRecurrence(existingAutorenewEvent)); } @Test void testSuccess_autorenewGraceActive_onlyAtAutomaticTransferTime() throws Exception { setupDomain("example", "tld"); - VKey existingAutorenewEvent = domain.getAutorenewBillingEvent(); + VKey existingAutorenewEvent = domain.getAutorenewBillingEvent(); // Set domain to expire in 1 day, so that it will be in the autorenew grace period by the // automatic transfer time, even though it isn't yet. DateTime expirationTime = clock.nowUtc().plusDays(1); @@ -1185,7 +1188,7 @@ class DomainTransferRequestFlowTest "domain_transfer_request.xml", "domain_transfer_request_response_autorenew_grace_at_transfer_only.xml", expirationTime.plusYears(1), - new BillingEvent.Cancellation.Builder() + new BillingCancellation.Builder() .setReason(Reason.RENEW) .setTargetId("example.tld") .setRegistrarId("TheRegistrar") @@ -1193,7 +1196,7 @@ class DomainTransferRequestFlowTest .setEventTime(clock.nowUtc().plus(Tld.get("tld").getAutomaticTransferLength())) .setBillingTime(expirationTime.plus(Tld.get("tld").getAutoRenewGracePeriodLength())) // The cancellation should refer to the old autorenew billing event. - .setRecurringEventKey(existingAutorenewEvent)); + .setBillingRecurrence(existingAutorenewEvent)); } @Test @@ -1247,7 +1250,7 @@ class DomainTransferRequestFlowTest // transfer is not explicitly acked) maintains the non-premium behavior. assertBillingEventsForResource( domain, - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setBillingTime(now.plusDays(10)) // 5 day pending transfer + 5 day billing grace period .setEventTime(now.plusDays(5)) .setRegistrarId("NewRegistrar") @@ -1301,7 +1304,7 @@ class DomainTransferRequestFlowTest // transfer is not explicitly acked) maintains the non-premium behavior. assertBillingEventsForResource( domain, - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setBillingTime(now.plusDays(10)) // 5 day pending transfer + 5 day billing grace period .setEventTime(now.plusDays(5)) .setRegistrarId("NewRegistrar") @@ -1360,7 +1363,7 @@ class DomainTransferRequestFlowTest // transfer is not explicitly acked) maintains the non-premium behavior. assertBillingEventsForResource( domain, - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setBillingTime(now.plusDays(10)) // 5 day pending transfer + 5 day billing grace period .setEventTime(now.plusDays(5)) .setRegistrarId("NewRegistrar") @@ -1418,7 +1421,7 @@ class DomainTransferRequestFlowTest // transfer is not explicitly acked) maintains the non-premium behavior. assertBillingEventsForResource( domain, - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setBillingTime(now.plusDays(10)) // 5 day pending transfer + 5 day billing grace period .setEventTime(now.plusDays(5)) .setRegistrarId("NewRegistrar") diff --git a/core/src/test/java/google/registry/flows/domain/DomainUpdateFlowTest.java b/core/src/test/java/google/registry/flows/domain/DomainUpdateFlowTest.java index e64ba8278..859b87305 100644 --- a/core/src/test/java/google/registry/flows/domain/DomainUpdateFlowTest.java +++ b/core/src/test/java/google/registry/flows/domain/DomainUpdateFlowTest.java @@ -88,8 +88,8 @@ import google.registry.flows.exceptions.OnlyToolCanPassMetadataException; import google.registry.flows.exceptions.ResourceHasClientUpdateProhibitedException; import google.registry.flows.exceptions.ResourceStatusProhibitsOperationException; import google.registry.model.ImmutableObject; +import google.registry.model.billing.BillingBase.Reason; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.Reason; import google.registry.model.contact.Contact; import google.registry.model.domain.DesignatedContact; import google.registry.model.domain.DesignatedContact.Type; @@ -827,7 +827,7 @@ class DomainUpdateFlowTest extends ResourceFlowTestCase> E commonInit( + private static > E commonInit( B builder) { return builder.setRegistrarId("TheRegistrar").setTargetId("foo.tld").build(); } @Test void testPersistence() { - assertThat(loadByEntity(oneTime)).isEqualTo(oneTime); - assertThat(loadByEntity(oneTimeSynthetic)).isEqualTo(oneTimeSynthetic); - assertThat(loadByEntity(recurring)).isEqualTo(recurring); + assertThat(loadByEntity(billingEvent)).isEqualTo(billingEvent); + assertThat(loadByEntity(billingEventSynthetic)).isEqualTo(billingEventSynthetic); + assertThat(loadByEntity(billingRecurrence)).isEqualTo(billingRecurrence); assertThat(loadByEntity(cancellationOneTime)).isEqualTo(cancellationOneTime); - assertThat(loadByEntity(cancellationRecurring)).isEqualTo(cancellationRecurring); + assertThat(loadByEntity(cancellationRecurrence)).isEqualTo(cancellationRecurrence); } @Test void testSerializable() { - BillingEvent persisted = loadByEntity(oneTime); + BillingBase persisted = loadByEntity(billingEvent); assertThat(serializeDeserialize(persisted)).isEqualTo(persisted); - persisted = loadByEntity(oneTimeSynthetic); + persisted = loadByEntity(billingEventSynthetic); assertThat(serializeDeserialize(persisted)).isEqualTo(persisted); - persisted = loadByEntity(recurring); + persisted = loadByEntity(billingRecurrence); assertThat(serializeDeserialize(persisted)).isEqualTo(persisted); persisted = loadByEntity(cancellationOneTime); assertThat(serializeDeserialize(persisted)).isEqualTo(persisted); - persisted = loadByEntity(cancellationRecurring); + persisted = loadByEntity(cancellationRecurrence); assertThat(serializeDeserialize(persisted)).isEqualTo(persisted); } @Test void testCancellationMatching() { - VKey recurringKey = loadByEntity(oneTimeSynthetic).getCancellationMatchingBillingEvent(); - assertThat(loadByKey(recurringKey)).isEqualTo(recurring); + VKey recurrenceKey = + loadByEntity(billingEventSynthetic).getCancellationMatchingBillingEvent(); + assertThat(loadByKey(recurrenceKey)).isEqualTo(billingRecurrence); } @Test @@ -204,10 +204,10 @@ public class BillingEventTest extends EntityTestCase { assertThrows( IllegalStateException.class, () -> - oneTime + billingEvent .asBuilder() - .setFlags(ImmutableSet.of(BillingEvent.Flag.SYNTHETIC)) - .setCancellationMatchingBillingEvent(recurring) + .setFlags(ImmutableSet.of(BillingBase.Flag.SYNTHETIC)) + .setCancellationMatchingBillingEvent(billingRecurrence) .build()); assertThat(thrown) .hasMessageThat() @@ -219,7 +219,7 @@ public class BillingEventTest extends EntityTestCase { IllegalStateException thrown = assertThrows( IllegalStateException.class, - () -> oneTime.asBuilder().setSyntheticCreationTime(now.plusDays(10)).build()); + () -> billingEvent.asBuilder().setSyntheticCreationTime(now.plusDays(10)).build()); assertThat(thrown) .hasMessageThat() .contains("Synthetic creation time must be set if and only if the SYNTHETIC flag is set"); @@ -231,9 +231,9 @@ public class BillingEventTest extends EntityTestCase { assertThrows( IllegalStateException.class, () -> - oneTime + billingEvent .asBuilder() - .setFlags(ImmutableSet.of(BillingEvent.Flag.SYNTHETIC)) + .setFlags(ImmutableSet.of(BillingBase.Flag.SYNTHETIC)) .setSyntheticCreationTime(END_OF_TIME) .build()); assertThat(thrown) @@ -248,7 +248,11 @@ public class BillingEventTest extends EntityTestCase { IllegalStateException thrown = assertThrows( IllegalStateException.class, - () -> oneTime.asBuilder().setCancellationMatchingBillingEvent(recurring).build()); + () -> + billingEvent + .asBuilder() + .setCancellationMatchingBillingEvent(billingRecurrence) + .build()); assertThat(thrown) .hasMessageThat() .contains( @@ -258,9 +262,9 @@ public class BillingEventTest extends EntityTestCase { @Test void testSuccess_cancellation_forGracePeriod_withOneTime() { - BillingEvent.Cancellation newCancellation = - BillingEvent.Cancellation.forGracePeriod( - GracePeriod.forBillingEvent(GracePeriodStatus.ADD, domain.getRepoId(), oneTime), + BillingCancellation newCancellation = + BillingCancellation.forGracePeriod( + GracePeriod.forBillingEvent(GracePeriodStatus.ADD, domain.getRepoId(), billingEvent), domainHistory2.getModificationTime(), domainHistory2.getHistoryEntryId(), "foo.tld"); @@ -270,21 +274,21 @@ public class BillingEventTest extends EntityTestCase { } @Test - void testSuccess_cancellation_forGracePeriod_withRecurring() { - BillingEvent.Cancellation newCancellation = - BillingEvent.Cancellation.forGracePeriod( - GracePeriod.createForRecurring( + void testSuccess_cancellation_forGracePeriod_withRecurrence() { + BillingCancellation newCancellation = + BillingCancellation.forGracePeriod( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), now.plusYears(1).plusDays(45), "TheRegistrar", - recurring.createVKey()), + billingRecurrence.createVKey()), domainHistory2.getModificationTime(), domainHistory2.getHistoryEntryId(), "foo.tld"); // Set ID to be the same to ignore for the purposes of comparison. - assertThat(newCancellation.asBuilder().setId(cancellationRecurring.getId()).build()) - .isEqualTo(cancellationRecurring); + assertThat(newCancellation.asBuilder().setId(cancellationRecurrence.getId()).build()) + .isEqualTo(cancellationRecurrence); } @Test @@ -293,7 +297,7 @@ public class BillingEventTest extends EntityTestCase { assertThrows( IllegalArgumentException.class, () -> - BillingEvent.Cancellation.forGracePeriod( + BillingCancellation.forGracePeriod( GracePeriod.createWithoutBillingEvent( GracePeriodStatus.REDEMPTION, domain.getRepoId(), @@ -313,8 +317,8 @@ public class BillingEventTest extends EntityTestCase { () -> cancellationOneTime .asBuilder() - .setOneTimeEventKey(null) - .setRecurringEventKey(null) + .setBillingEvent(null) + .setBillingRecurrence(null) .build()); assertThat(thrown).hasMessageThat().contains("exactly one billing event"); } @@ -327,8 +331,8 @@ public class BillingEventTest extends EntityTestCase { () -> cancellationOneTime .asBuilder() - .setOneTimeEventKey(oneTime.createVKey()) - .setRecurringEventKey(recurring.createVKey()) + .setBillingEvent(billingEvent.createVKey()) + .setBillingRecurrence(billingRecurrence.createVKey()) .build()); assertThat(thrown).hasMessageThat().contains("exactly one billing event"); } @@ -339,7 +343,7 @@ public class BillingEventTest extends EntityTestCase { assertThrows( IllegalStateException.class, () -> - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setBillingTime(DateTime.parse("2020-02-05T15:33:11Z")) .setEventTime(DateTime.parse("2020-01-05T15:33:11Z")) .setCost(Money.of(USD, 10)) @@ -360,7 +364,7 @@ public class BillingEventTest extends EntityTestCase { assertThrows( IllegalStateException.class, () -> - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setBillingTime(DateTime.parse("2020-02-05T15:33:11Z")) .setEventTime(DateTime.parse("2020-01-05T15:33:11Z")) .setCost(Money.of(USD, 10)) @@ -380,7 +384,7 @@ public class BillingEventTest extends EntityTestCase { void testReasonRequiringPeriodYears_missingPeriodYears_isAllowedOnOldData() { // This won't throw even though periodYears is missing on a RESTORE because the event time // is before 2019. - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setBillingTime(DateTime.parse("2018-02-05T15:33:11Z")) .setEventTime(DateTime.parse("2018-01-05T15:33:11Z")) .setReason(Reason.RESTORE) @@ -393,132 +397,135 @@ public class BillingEventTest extends EntityTestCase { @Test void testSuccess_defaultRenewalPriceBehavior_assertsIsDefault() { - assertThat(recurring.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); - assertThat(recurring.getRenewalPrice()).isEmpty(); + assertThat(billingRecurrence.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); } @Test void testSuccess_getRenewalPriceBehavior_returnsRightBehavior() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.NONPREMIUM) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.NONPREMIUM); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.NONPREMIUM); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); } @Test void testSuccess_setRenewalPriceBehaviorThenBuild_defaultToSpecified() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.DEFAULT) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); persistResource( loadedEntity .asBuilder() .setRenewalPrice(Money.of(USD, 100)) .setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED) .build()); - assertThat(loadByEntity(recurringEvent).getRenewalPriceBehavior()) + assertThat(loadByEntity(billingRecurrence).getRenewalPriceBehavior()) .isEqualTo(RenewalPriceBehavior.SPECIFIED); - assertThat(loadByEntity(recurringEvent).getRenewalPrice()).hasValue(Money.of(USD, 100)); + assertThat(loadByEntity(billingRecurrence).getRenewalPrice()).hasValue(Money.of(USD, 100)); } @Test void testSuccess_setRenewalPriceBehaviorThenBuild_defaultToNonPremium() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.DEFAULT) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); persistResource( loadedEntity.asBuilder().setRenewalPriceBehavior(RenewalPriceBehavior.NONPREMIUM).build()); - assertThat(loadByEntity(recurringEvent).getRenewalPriceBehavior()) + assertThat(loadByEntity(billingRecurrence).getRenewalPriceBehavior()) .isEqualTo(RenewalPriceBehavior.NONPREMIUM); - assertThat(loadByEntity(recurringEvent).getRenewalPrice()).isEmpty(); + assertThat(loadByEntity(billingRecurrence).getRenewalPrice()).isEmpty(); } @Test void testSuccess_setRenewalPriceBehaviorThenBuild_nonPremiumToSpecified() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.NONPREMIUM) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.NONPREMIUM); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.NONPREMIUM); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); persistResource( loadedEntity .asBuilder() .setRenewalPrice(Money.of(USD, 100)) .setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED) .build()); - assertThat(loadByEntity(recurringEvent).getRenewalPriceBehavior()) + assertThat(loadByEntity(billingRecurrence).getRenewalPriceBehavior()) .isEqualTo(RenewalPriceBehavior.SPECIFIED); - assertThat(loadByEntity(recurringEvent).getRenewalPrice()).hasValue(Money.of(USD, 100)); + assertThat(loadByEntity(billingRecurrence).getRenewalPrice()).hasValue(Money.of(USD, 100)); } @Test void testSuccess_setRenewalPriceBehaviorThenBuild_nonPremiumToDefault() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.NONPREMIUM) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.NONPREMIUM); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.NONPREMIUM); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); persistResource( loadedEntity.asBuilder().setRenewalPriceBehavior(RenewalPriceBehavior.DEFAULT).build()); - assertThat(loadByEntity(recurringEvent).getRenewalPriceBehavior()) + assertThat(loadByEntity(billingRecurrence).getRenewalPriceBehavior()) .isEqualTo(RenewalPriceBehavior.DEFAULT); - assertThat(loadByEntity(recurringEvent).getRenewalPrice()).isEmpty(); + assertThat(loadByEntity(billingRecurrence).getRenewalPrice()).isEmpty(); } @Test void testSuccess_setRenewalPriceBehaviorThenBuild_specifiedToDefault() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) @@ -526,27 +533,28 @@ public class BillingEventTest extends EntityTestCase { .setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED) .setRenewalPrice(Money.of(USD, 100)) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.SPECIFIED); - assertThat(recurringEvent.getRenewalPrice()).hasValue(Money.of(USD, 100)); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.SPECIFIED); + assertThat(billingRecurrence.getRenewalPrice()).hasValue(Money.of(USD, 100)); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); persistResource( loadedEntity .asBuilder() .setRenewalPrice(null) .setRenewalPriceBehavior(RenewalPriceBehavior.DEFAULT) .build()); - assertThat(loadByEntity(recurringEvent).getRenewalPriceBehavior()) + assertThat(loadByEntity(billingRecurrence).getRenewalPriceBehavior()) .isEqualTo(RenewalPriceBehavior.DEFAULT); - assertThat(loadByEntity(recurringEvent).getRenewalPrice()).isEmpty(); + assertThat(loadByEntity(billingRecurrence).getRenewalPrice()).isEmpty(); } @Test void testSuccess_setRenewalPriceBehaviorThenBuild_specifiedToNonPremium() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) @@ -554,37 +562,38 @@ public class BillingEventTest extends EntityTestCase { .setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED) .setRenewalPrice(Money.of(USD, 100)) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.SPECIFIED); - assertThat(recurringEvent.getRenewalPrice()).hasValue(Money.of(USD, 100)); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.SPECIFIED); + assertThat(billingRecurrence.getRenewalPrice()).hasValue(Money.of(USD, 100)); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); persistResource( loadedEntity .asBuilder() .setRenewalPrice(null) .setRenewalPriceBehavior(RenewalPriceBehavior.NONPREMIUM) .build()); - assertThat(loadByEntity(recurringEvent).getRenewalPriceBehavior()) + assertThat(loadByEntity(billingRecurrence).getRenewalPriceBehavior()) .isEqualTo(RenewalPriceBehavior.NONPREMIUM); - assertThat(loadByEntity(recurringEvent).getRenewalPrice()).isEmpty(); + assertThat(loadByEntity(billingRecurrence).getRenewalPrice()).isEmpty(); } @Test void testFailure_setRenewalPriceBehaviorThenBuild_defaultToSpecified_needRenewalPrice() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.DEFAULT) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); IllegalArgumentException thrown = assertThrows( IllegalArgumentException.class, @@ -602,20 +611,20 @@ public class BillingEventTest extends EntityTestCase { @Test void testFailure_setRenewalPriceBehaviorThenBuild_defaultToPremium_noNeedToAddRenewalPrice() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.DEFAULT) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); IllegalArgumentException thrown = assertThrows( IllegalArgumentException.class, @@ -634,20 +643,21 @@ public class BillingEventTest extends EntityTestCase { @Test void testFailure_setRenewalPriceBehaviorThenBuild_nonPremiumToDefault_noNeedToAddRenewalPrice() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.NONPREMIUM) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.NONPREMIUM); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.NONPREMIUM); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); IllegalArgumentException thrown = assertThrows( IllegalArgumentException.class, @@ -666,20 +676,21 @@ public class BillingEventTest extends EntityTestCase { @Test void testFailure_setRenewalPriceBehaviorThenBuild_nonPremiumToSpecified_needRenewalPrice() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.NONPREMIUM) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.NONPREMIUM); - assertThat(recurringEvent.getRenewalPrice()).isEmpty(); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.NONPREMIUM); + assertThat(billingRecurrence.getRenewalPrice()).isEmpty(); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); IllegalArgumentException thrown = assertThrows( IllegalArgumentException.class, @@ -697,10 +708,10 @@ public class BillingEventTest extends EntityTestCase { @Test void testFailure_setRenewalPriceBehaviorThenBuild_specifiedToNonPremium_removeRenewalPrice() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) @@ -708,10 +719,11 @@ public class BillingEventTest extends EntityTestCase { .setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED) .setRenewalPrice(Money.of(USD, 100)) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.SPECIFIED); - assertThat(recurringEvent.getRenewalPrice()).hasValue(Money.of(USD, 100)); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.SPECIFIED); + assertThat(billingRecurrence.getRenewalPrice()).hasValue(Money.of(USD, 100)); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); IllegalArgumentException thrown = assertThrows( IllegalArgumentException.class, @@ -729,10 +741,10 @@ public class BillingEventTest extends EntityTestCase { @Test void testFailure_setRenewalPriceBehaviorThenBuild_specifiedToDefault_removeRenewalPrice() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) @@ -740,10 +752,11 @@ public class BillingEventTest extends EntityTestCase { .setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED) .setRenewalPrice(Money.of(USD, 100)) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.SPECIFIED); - assertThat(recurringEvent.getRenewalPrice()).hasValue(Money.of(USD, 100)); - BillingEvent.Recurring loadedEntity = loadByEntity(recurringEvent); - assertThat(loadedEntity).isEqualTo(recurringEvent); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.SPECIFIED); + assertThat(billingRecurrence.getRenewalPrice()).hasValue(Money.of(USD, 100)); + BillingRecurrence loadedEntity = loadByEntity(billingRecurrence); + assertThat(loadedEntity).isEqualTo(billingRecurrence); IllegalArgumentException thrown = assertThrows( IllegalArgumentException.class, @@ -761,10 +774,10 @@ public class BillingEventTest extends EntityTestCase { @Test void testSuccess_buildWithDefaultRenewalBehavior() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) @@ -772,33 +785,35 @@ public class BillingEventTest extends EntityTestCase { .setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED) .setRenewalPrice(Money.of(USD, BigDecimal.valueOf(100))) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.SPECIFIED); - assertThat(recurringEvent.getRenewalPrice()).hasValue(Money.of(USD, 100)); - assertThat(recurringEvent.getRecurrenceLastExpansion()).isEqualTo(now); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.SPECIFIED); + assertThat(billingRecurrence.getRenewalPrice()).hasValue(Money.of(USD, 100)); + assertThat(billingRecurrence.getRecurrenceLastExpansion()).isEqualTo(now); } @Test void testSuccess_buildWithNonPremiumRenewalBehavior() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setEventTime(now.plusYears(1)) .setRenewalPriceBehavior(RenewalPriceBehavior.NONPREMIUM) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.NONPREMIUM); - assertThat(loadByEntity(recurringEvent).getRenewalPrice()).isEmpty(); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.NONPREMIUM); + assertThat(loadByEntity(billingRecurrence).getRenewalPrice()).isEmpty(); } @Test void testSuccess_buildWithSpecifiedRenewalBehavior() { - BillingEvent.Recurring recurringEvent = + BillingRecurrence billingRecurrence = persistResource( commonInit( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) @@ -806,8 +821,9 @@ public class BillingEventTest extends EntityTestCase { .setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED) .setRenewalPrice(Money.of(USD, BigDecimal.valueOf(100))) .setRecurrenceEndTime(END_OF_TIME))); - assertThat(recurringEvent.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.SPECIFIED); - assertThat(recurringEvent.getRenewalPrice()).hasValue(Money.of(USD, 100)); + assertThat(billingRecurrence.getRenewalPriceBehavior()) + .isEqualTo(RenewalPriceBehavior.SPECIFIED); + assertThat(billingRecurrence.getRenewalPrice()).hasValue(Money.of(USD, 100)); } @Test @@ -816,7 +832,7 @@ public class BillingEventTest extends EntityTestCase { assertThrows( IllegalArgumentException.class, () -> - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) @@ -837,7 +853,7 @@ public class BillingEventTest extends EntityTestCase { assertThrows( IllegalArgumentException.class, () -> - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) @@ -859,7 +875,7 @@ public class BillingEventTest extends EntityTestCase { assertThrows( IllegalArgumentException.class, () -> - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(domainHistory) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) diff --git a/core/src/test/java/google/registry/model/domain/DomainSqlTest.java b/core/src/test/java/google/registry/model/domain/DomainSqlTest.java index b80fe856b..46b9f8f08 100644 --- a/core/src/test/java/google/registry/model/domain/DomainSqlTest.java +++ b/core/src/test/java/google/registry/model/domain/DomainSqlTest.java @@ -34,7 +34,7 @@ import static org.joda.time.DateTimeZone.UTC; import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.Sets; -import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; import google.registry.model.contact.Contact; import google.registry.model.domain.DesignatedContact.Type; import google.registry.model.domain.launch.LaunchNotice; diff --git a/core/src/test/java/google/registry/model/domain/DomainTest.java b/core/src/test/java/google/registry/model/domain/DomainTest.java index 4a96b45c1..3a0f35303 100644 --- a/core/src/test/java/google/registry/model/domain/DomainTest.java +++ b/core/src/test/java/google/registry/model/domain/DomainTest.java @@ -19,7 +19,7 @@ import static com.google.common.collect.Iterables.getOnlyElement; import static com.google.common.truth.Truth.assertThat; import static com.google.common.truth.Truth8.assertThat; import static google.registry.model.EppResourceUtils.loadByForeignKey; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.SPECIFIED; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.SPECIFIED; import static google.registry.model.domain.token.AllocationToken.TokenType.PACKAGE; import static google.registry.model.domain.token.AllocationToken.TokenType.SINGLE_USE; import static google.registry.testing.DatabaseHelper.cloneAndSetAutoTimestamps; @@ -46,9 +46,10 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Streams; import google.registry.model.ImmutableObject; import google.registry.model.ImmutableObjectSubject; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; 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.BillingRecurrence; import google.registry.model.contact.Contact; import google.registry.model.domain.DesignatedContact.Type; import google.registry.model.domain.launch.LaunchNotice; @@ -87,8 +88,8 @@ public class DomainTest { new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationWithCoverageExtension(); private Domain domain; - private VKey oneTimeBillKey; - private VKey recurringBillKey; + private VKey oneTimeBillKey; + private VKey recurrenceBillKey; private DomainHistory domainHistory; private VKey contact1Key, contact2Key; @@ -113,7 +114,7 @@ public class DomainTest { .build()); oneTimeBillKey = persistResource( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() // Use SERVER_STATUS, so we don't have to add a period. .setReason(Reason.SERVER_STATUS) .setTargetId(domain.getDomainName()) @@ -137,8 +138,8 @@ public class DomainTest { .setRequestedByRegistrar(false) .setXmlBytes(new byte[0]) .build(); - BillingEvent.OneTime oneTimeBill = - new BillingEvent.OneTime.Builder() + BillingEvent billingEventBill = + new BillingEvent.Builder() .setId(500L) // Use SERVER_STATUS, so we don't have to add a period. .setReason(Reason.SERVER_STATUS) @@ -149,9 +150,9 @@ public class DomainTest { .setEventTime(DateTime.now(UTC).plusYears(1)) .setDomainHistory(historyEntry) .build(); - oneTimeBillKey = oneTimeBill.createVKey(); - BillingEvent.Recurring recurringBill = - new BillingEvent.Recurring.Builder() + oneTimeBillKey = billingEventBill.createVKey(); + BillingRecurrence billingRecurrence = + new BillingRecurrence.Builder() .setId(200L) .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) @@ -161,8 +162,8 @@ public class DomainTest { .setRecurrenceEndTime(END_OF_TIME) .setDomainHistory(historyEntry) .build(); - insertInDb(historyEntry, oneTimeBill, recurringBill); - recurringBillKey = recurringBill.createVKey(); + insertInDb(historyEntry, billingEventBill, billingRecurrence); + recurrenceBillKey = billingRecurrence.createVKey(); VKey autorenewPollKey = VKey.create(PollMessage.Autorenew.class, 3L); VKey onetimePollKey = VKey.create(PollMessage.OneTime.class, 1L); // Set up a new persisted domain entity. @@ -201,16 +202,17 @@ public class DomainTest { .setServerApproveEntities( historyEntry.getRepoId(), historyEntry.getRevisionId(), - ImmutableSet.of(oneTimeBillKey, recurringBillKey, autorenewPollKey)) + ImmutableSet.of( + oneTimeBillKey, recurrenceBillKey, autorenewPollKey)) .setServerApproveBillingEvent(oneTimeBillKey) - .setServerApproveAutorenewEvent(recurringBillKey) + .setServerApproveAutorenewEvent(recurrenceBillKey) .setServerApproveAutorenewPollMessage(autorenewPollKey) .setTransferRequestTime(fakeClock.nowUtc().plusDays(1)) .setTransferStatus(TransferStatus.SERVER_APPROVED) .setTransferRequestTrid(Trid.create("client-trid", "server-trid")) .build()) .setDeletePollMessage(onetimePollKey) - .setAutorenewBillingEvent(recurringBillKey) + .setAutorenewBillingEvent(recurrenceBillKey) .setAutorenewPollMessage(autorenewPollKey) .setSmdId("smdid") .addGracePeriod( @@ -385,7 +387,7 @@ public class DomainTest { } private void assertTransferred( - Domain domain, DateTime newExpirationTime, VKey newAutorenewEvent) { + Domain domain, DateTime newExpirationTime, VKey newAutorenewEvent) { assertThat(domain.getTransferData().getTransferStatus()) .isEqualTo(TransferStatus.SERVER_APPROVED); assertThat(domain.getCurrentSponsorRegistrarId()).isEqualTo("TheRegistrar"); @@ -403,9 +405,9 @@ public class DomainTest { .setRegistrarId(domain.getCurrentSponsorRegistrarId()) .setType(HistoryEntry.Type.DOMAIN_TRANSFER_REQUEST) .build()); - BillingEvent.OneTime transferBillingEvent = + BillingEvent transferBillingEvent = persistResource( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.TRANSFER) .setRegistrarId("TheRegistrar") .setTargetId(domain.getDomainName()) @@ -449,7 +451,7 @@ public class DomainTest { .build(); Domain afterTransfer = domain.cloneProjectedAtTime(fakeClock.nowUtc().plusDays(1)); DateTime newExpirationTime = oldExpirationTime.plusYears(1); - VKey serverApproveAutorenewEvent = + VKey serverApproveAutorenewEvent = domain.getTransferData().getServerApproveAutorenewEvent(); assertTransferred(afterTransfer, newExpirationTime, serverApproveAutorenewEvent); assertThat(afterTransfer.getGracePeriods()) @@ -689,7 +691,7 @@ public class DomainTest { assertThat(renewedThreeTimes.getLastEppUpdateTime()).isEqualTo(oldExpirationTime.plusYears(2)); assertThat(renewedThreeTimes.getGracePeriods()) .containsExactly( - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), oldExpirationTime.plusYears(2).plus(Tld.get("com").getAutoRenewGracePeriodLength()), @@ -921,7 +923,7 @@ public class DomainTest { .setPendingTransferExpirationTime(transferExpirationTime) .setTransferStatus(TransferStatus.PENDING) .setGainingRegistrarId("TheRegistrar") - .setServerApproveAutorenewEvent(recurringBillKey) + .setServerApproveAutorenewEvent(recurrenceBillKey) .setServerApproveBillingEvent(oneTimeBillKey) .build(); domain = @@ -931,14 +933,14 @@ public class DomainTest { .setRegistrationExpirationTime(previousExpiration) .setGracePeriods( ImmutableSet.of( - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), now.plusDays(1), "NewRegistrar", - recurringBillKey))) + recurrenceBillKey))) .setTransferData(transferData) - .setAutorenewBillingEvent(recurringBillKey) + .setAutorenewBillingEvent(recurrenceBillKey) .build()); Domain clone = domain.cloneProjectedAtTime(now); assertThat(clone.getRegistrationExpirationTime()) @@ -959,12 +961,12 @@ public class DomainTest { .setRegistrationExpirationTime(now.plusYears(1)) .setGracePeriods( ImmutableSet.of( - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.AUTO_RENEW, domain.getRepoId(), now.plusDays(1), "NewRegistrar", - recurringBillKey), + recurrenceBillKey), GracePeriod.create( GracePeriodStatus.RENEW, domain.getRepoId(), @@ -974,26 +976,23 @@ public class DomainTest { .build()); ImmutableSet historyIds = domain.getGracePeriods().stream() - .map( - gp -> new BillEventInfo(gp.getRecurringBillingEvent(), gp.getOneTimeBillingEvent())) + .map(gp -> new BillEventInfo(gp.getBillingRecurrence(), gp.getBillingEvent())) .collect(toImmutableSet()); assertThat(historyIds) .isEqualTo( ImmutableSet.of( new BillEventInfo(null, oneTimeBillKey), - new BillEventInfo(recurringBillKey, null))); + new BillEventInfo(recurrenceBillKey, null))); } static class BillEventInfo extends ImmutableObject { - VKey billingEventRecurring; - Long billingEventRecurringHistoryId; - VKey billingEventOneTime; + VKey recurrence; + Long recurrenceHistoryId; + VKey billingEventOneTime; Long billingEventOneTimeHistoryId; - BillEventInfo( - VKey billingEventRecurring, - VKey billingEventOneTime) { - this.billingEventRecurring = billingEventRecurring; + BillEventInfo(VKey recurrence, VKey billingEventOneTime) { + this.recurrence = recurrence; this.billingEventOneTime = billingEventOneTime; } } diff --git a/core/src/test/java/google/registry/model/domain/GracePeriodTest.java b/core/src/test/java/google/registry/model/domain/GracePeriodTest.java index 8cae3768d..bc79c043a 100644 --- a/core/src/test/java/google/registry/model/domain/GracePeriodTest.java +++ b/core/src/test/java/google/registry/model/domain/GracePeriodTest.java @@ -18,9 +18,9 @@ import static com.google.common.truth.Truth.assertThat; import static org.joda.time.DateTimeZone.UTC; import static org.junit.jupiter.api.Assertions.assertThrows; +import google.registry.model.billing.BillingBase.Reason; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.Reason; -import google.registry.model.billing.BillingEvent.Recurring; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.domain.rgp.GracePeriodStatus; import google.registry.model.reporting.HistoryEntry.HistoryEntryId; import google.registry.persistence.VKey; @@ -41,13 +41,13 @@ public class GracePeriodTest { new JpaTestExtensions.Builder().buildIntegrationTestExtension(); private final DateTime now = DateTime.now(UTC); - private BillingEvent.OneTime onetime; - private VKey recurringKey; + private BillingEvent onetime; + private VKey recurrenceKey; @BeforeEach void before() { onetime = - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setEventTime(now) .setBillingTime(now.plusDays(1)) .setRegistrarId("TheRegistrar") @@ -57,7 +57,7 @@ public class GracePeriodTest { .setPeriodYears(1) .setTargetId("foo.google") .build(); - recurringKey = Recurring.createVKey(12345L); + recurrenceKey = BillingRecurrence.createVKey(12345L); } @Test @@ -65,22 +65,22 @@ public class GracePeriodTest { GracePeriod gracePeriod = GracePeriod.forBillingEvent(GracePeriodStatus.ADD, "1-TEST", onetime); assertThat(gracePeriod.getType()).isEqualTo(GracePeriodStatus.ADD); assertThat(gracePeriod.getDomainRepoId()).isEqualTo("1-TEST"); - assertThat(gracePeriod.getOneTimeBillingEvent()).isEqualTo(onetime.createVKey()); - assertThat(gracePeriod.getRecurringBillingEvent()).isNull(); + assertThat(gracePeriod.getBillingEvent()).isEqualTo(onetime.createVKey()); + assertThat(gracePeriod.getBillingRecurrence()).isNull(); assertThat(gracePeriod.getRegistrarId()).isEqualTo("TheRegistrar"); assertThat(gracePeriod.getExpirationTime()).isEqualTo(now.plusDays(1)); assertThat(gracePeriod.hasBillingEvent()).isTrue(); } @Test - void testSuccess_forRecurringEvent() { + void testSuccess_forRecurrence() { GracePeriod gracePeriod = - GracePeriod.createForRecurring( - GracePeriodStatus.AUTO_RENEW, "1-TEST", now.plusDays(1), "TheRegistrar", recurringKey); + GracePeriod.createForRecurrence( + GracePeriodStatus.AUTO_RENEW, "1-TEST", now.plusDays(1), "TheRegistrar", recurrenceKey); assertThat(gracePeriod.getType()).isEqualTo(GracePeriodStatus.AUTO_RENEW); assertThat(gracePeriod.getDomainRepoId()).isEqualTo("1-TEST"); - assertThat(gracePeriod.getOneTimeBillingEvent()).isNull(); - assertThat(gracePeriod.getRecurringBillingEvent()).isEqualTo(recurringKey); + assertThat(gracePeriod.getBillingEvent()).isNull(); + assertThat(gracePeriod.getBillingRecurrence()).isEqualTo(recurrenceKey); assertThat(gracePeriod.getRegistrarId()).isEqualTo("TheRegistrar"); assertThat(gracePeriod.getExpirationTime()).isEqualTo(now.plusDays(1)); assertThat(gracePeriod.hasBillingEvent()).isTrue(); @@ -93,8 +93,8 @@ public class GracePeriodTest { GracePeriodStatus.REDEMPTION, "1-TEST", now, "TheRegistrar"); assertThat(gracePeriod.getType()).isEqualTo(GracePeriodStatus.REDEMPTION); assertThat(gracePeriod.getDomainRepoId()).isEqualTo("1-TEST"); - assertThat(gracePeriod.getOneTimeBillingEvent()).isNull(); - assertThat(gracePeriod.getRecurringBillingEvent()).isNull(); + assertThat(gracePeriod.getBillingEvent()).isNull(); + assertThat(gracePeriod.getBillingRecurrence()).isNull(); assertThat(gracePeriod.getRegistrarId()).isEqualTo("TheRegistrar"); assertThat(gracePeriod.getExpirationTime()).isEqualTo(now); assertThat(gracePeriod.hasBillingEvent()).isFalse(); @@ -110,17 +110,17 @@ public class GracePeriodTest { } @Test - void testFailure_createForRecurring_notAutoRenew() { + void testFailure_createForRecurrence_notAutoRenew() { IllegalArgumentException thrown = assertThrows( IllegalArgumentException.class, () -> - GracePeriod.createForRecurring( + GracePeriod.createForRecurrence( GracePeriodStatus.RENEW, "1-TEST", now.plusDays(1), "TheRegistrar", - recurringKey)); + recurrenceKey)); assertThat(thrown).hasMessageThat().contains("autorenew"); } } diff --git a/core/src/test/java/google/registry/model/domain/token/AllocationTokenTest.java b/core/src/test/java/google/registry/model/domain/token/AllocationTokenTest.java index fa3489045..721aa4cd7 100644 --- a/core/src/test/java/google/registry/model/domain/token/AllocationTokenTest.java +++ b/core/src/test/java/google/registry/model/domain/token/AllocationTokenTest.java @@ -35,7 +35,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedMap; import google.registry.model.Buildable; import google.registry.model.EntityTestCase; -import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; import google.registry.model.domain.Domain; import google.registry.model.domain.fee.FeeQueryCommandExtensionItem.CommandName; import google.registry.model.domain.token.AllocationToken.RegistrationBehavior; diff --git a/core/src/test/java/google/registry/model/domain/token/PackagePromotionTest.java b/core/src/test/java/google/registry/model/domain/token/PackagePromotionTest.java index 27188315f..3771a5378 100644 --- a/core/src/test/java/google/registry/model/domain/token/PackagePromotionTest.java +++ b/core/src/test/java/google/registry/model/domain/token/PackagePromotionTest.java @@ -22,7 +22,7 @@ import static google.registry.testing.DatabaseHelper.persistResource; import static org.junit.jupiter.api.Assertions.assertThrows; import google.registry.model.EntityTestCase; -import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; import google.registry.model.domain.token.AllocationToken.TokenType; import org.joda.money.CurrencyUnit; import org.joda.money.Money; diff --git a/core/src/test/java/google/registry/model/transfer/TransferDataTest.java b/core/src/test/java/google/registry/model/transfer/TransferDataTest.java index 9a0857aa5..e95e2283b 100644 --- a/core/src/test/java/google/registry/model/transfer/TransferDataTest.java +++ b/core/src/test/java/google/registry/model/transfer/TransferDataTest.java @@ -18,10 +18,9 @@ import static com.google.common.truth.Truth.assertThat; import static org.joda.time.DateTimeZone.UTC; import com.google.common.collect.ImmutableSet; +import google.registry.model.billing.BillingCancellation; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.Cancellation; -import google.registry.model.billing.BillingEvent.OneTime; -import google.registry.model.billing.BillingEvent.Recurring; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.domain.Period; import google.registry.model.eppcommon.Trid; import google.registry.model.poll.PollMessage; @@ -35,17 +34,17 @@ public class TransferDataTest { private final DateTime now = DateTime.now(UTC); - private VKey transferBillingEventKey; - private VKey otherServerApproveBillingEventKey; - private VKey recurringBillingEventKey; + private VKey transferBillingEventKey; + private VKey otherServerApproveBillingEventKey; + private VKey recurrenceKey; private VKey autorenewPollMessageKey; private VKey otherServerApprovePollMessageKey; @BeforeEach void beforeEach() { - transferBillingEventKey = OneTime.createVKey(12345L); - otherServerApproveBillingEventKey = Cancellation.createVKey(2468L); - recurringBillingEventKey = Recurring.createVKey(13579L); + transferBillingEventKey = BillingEvent.createVKey(12345L); + otherServerApproveBillingEventKey = BillingCancellation.createVKey(2468L); + recurrenceKey = BillingRecurrence.createVKey(13579L); autorenewPollMessageKey = VKey.create(PollMessage.Autorenew.class, 67890L); otherServerApprovePollMessageKey = VKey.create(PollMessage.OneTime.class, 314159L); } @@ -72,11 +71,11 @@ public class TransferDataTest { ImmutableSet.of( transferBillingEventKey, otherServerApproveBillingEventKey, - recurringBillingEventKey, + recurrenceKey, autorenewPollMessageKey, otherServerApprovePollMessageKey)) .setServerApproveBillingEvent(transferBillingEventKey) - .setServerApproveAutorenewEvent(recurringBillingEventKey) + .setServerApproveAutorenewEvent(recurrenceKey) .setServerApproveAutorenewPollMessage(autorenewPollMessageKey) .build(); // asBuilder() copies over all fields diff --git a/core/src/test/java/google/registry/rde/DomainToXjcConverterTest.java b/core/src/test/java/google/registry/rde/DomainToXjcConverterTest.java index 07137c4c7..5d607dda0 100644 --- a/core/src/test/java/google/registry/rde/DomainToXjcConverterTest.java +++ b/core/src/test/java/google/registry/rde/DomainToXjcConverterTest.java @@ -30,9 +30,10 @@ import static org.joda.money.CurrencyUnit.USD; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.net.InetAddresses; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; 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.BillingRecurrence; import google.registry.model.contact.Contact; import google.registry.model.contact.ContactAddress; import google.registry.model.contact.ContactPhoneNumber; @@ -229,9 +230,9 @@ public class DomainToXjcConverterTest { .setDomain(domain) .setRegistrarId(domain.getCreationRegistrarId()) .build()); - BillingEvent.OneTime billingEvent = + BillingEvent billingEvent = persistResource( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.CREATE) .setTargetId("example.xn--q9jyb4c") .setRegistrarId("TheRegistrar") @@ -290,7 +291,7 @@ public class DomainToXjcConverterTest { GracePeriodStatus.RENEW, domain.getRepoId(), persistResource( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.RENEW) .setTargetId("love.xn--q9jyb4c") .setRegistrarId("TheRegistrar") @@ -315,7 +316,7 @@ public class DomainToXjcConverterTest { StatusValue.SERVER_UPDATE_PROHIBITED)) .setAutorenewBillingEvent( persistResource( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId("lol") @@ -344,7 +345,7 @@ public class DomainToXjcConverterTest { .setServerApproveBillingEvent(billingEvent.createVKey()) .setServerApproveAutorenewEvent( persistResource( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId("example.xn--q9jyb4c") diff --git a/core/src/test/java/google/registry/rde/RdeFixtures.java b/core/src/test/java/google/registry/rde/RdeFixtures.java index f1b60bf4e..5b2c62ff6 100644 --- a/core/src/test/java/google/registry/rde/RdeFixtures.java +++ b/core/src/test/java/google/registry/rde/RdeFixtures.java @@ -25,9 +25,10 @@ import static org.joda.money.CurrencyUnit.USD; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.net.InetAddresses; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; 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.BillingRecurrence; import google.registry.model.contact.Contact; import google.registry.model.contact.ContactAddress; import google.registry.model.contact.ContactPhoneNumber; @@ -74,9 +75,9 @@ final class RdeFixtures { .setRegistrarId("TheRegistrar") .build()); clock.advanceOneMilli(); - BillingEvent.OneTime billingEvent = + BillingEvent billingEvent = persistResource( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.CREATE) .setTargetId("example." + tld) .setRegistrarId("TheRegistrar") @@ -129,7 +130,7 @@ final class RdeFixtures { GracePeriodStatus.RENEW, domain.getRepoId(), persistResource( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.RENEW) .setTargetId("love." + tld) .setRegistrarId("TheRegistrar") @@ -154,7 +155,7 @@ final class RdeFixtures { StatusValue.SERVER_UPDATE_PROHIBITED)) .setAutorenewBillingEvent( persistResource( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId(tld) @@ -183,7 +184,7 @@ final class RdeFixtures { .setServerApproveBillingEvent(billingEvent.createVKey()) .setServerApproveAutorenewEvent( persistResource( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId("example." + tld) 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 3b75cd11d..12f6b924e 100644 --- a/core/src/test/java/google/registry/schema/integration/SqlIntegrationTestSuite.java +++ b/core/src/test/java/google/registry/schema/integration/SqlIntegrationTestSuite.java @@ -16,7 +16,7 @@ package google.registry.schema.integration; import static com.google.common.truth.Truth.assert_; -import google.registry.model.billing.BillingEventTest; +import google.registry.model.billing.BillingBaseTest; import google.registry.model.common.CursorTest; import google.registry.model.common.DnsRefreshRequestTest; import google.registry.model.console.UserTest; @@ -81,7 +81,7 @@ import org.junit.runner.RunWith; // BeforeSuiteTest must be the first entry. See class javadoc for details. BeforeSuiteTest.class, AllocationTokenTest.class, - BillingEventTest.class, + BillingBaseTest.class, ClaimsListDaoTest.class, ContactHistoryTest.class, ContactTest.class, diff --git a/core/src/test/java/google/registry/testing/DatabaseHelper.java b/core/src/test/java/google/registry/testing/DatabaseHelper.java index d5e5ab1ae..4fa789bfd 100644 --- a/core/src/test/java/google/registry/testing/DatabaseHelper.java +++ b/core/src/test/java/google/registry/testing/DatabaseHelper.java @@ -61,11 +61,13 @@ import google.registry.model.Buildable; import google.registry.model.EppResource; import google.registry.model.EppResourceUtils; import google.registry.model.ImmutableObject; +import google.registry.model.billing.BillingBase; +import google.registry.model.billing.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; +import google.registry.model.billing.BillingCancellation; 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.Recurring; -import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.common.DatabaseMigrationStateSchedule; import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState; import google.registry.model.contact.Contact; @@ -311,7 +313,9 @@ public final class DatabaseHelper { return persistResource(domain.asBuilder().setDeletionTime(deletionTime).build()); } - /** Persists a {@link Recurring} and {@link HistoryEntry} for a domain that already exists. */ + /** + * Persists a {@link BillingRecurrence} and {@link HistoryEntry} for a domain that already exists. + */ public static Domain persistBillingRecurrenceForDomain( Domain domain, RenewalPriceBehavior renewalPriceBehavior, @Nullable Money renewalPrice) { DomainHistory historyEntry = @@ -322,9 +326,9 @@ public final class DatabaseHelper { .setModificationTime(domain.getCreationTime()) .setDomain(domain) .build()); - Recurring recurring = + BillingRecurrence billingRecurrence = persistResource( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setDomainHistory(historyEntry) .setRenewalPrice(renewalPrice) .setRenewalPriceBehavior(renewalPriceBehavior) @@ -337,7 +341,7 @@ public final class DatabaseHelper { .setTargetId(domain.getDomainName()) .build()); return persistResource( - domain.asBuilder().setAutorenewBillingEvent(recurring.createVKey()).build()); + domain.asBuilder().setAutorenewBillingEvent(billingRecurrence.createVKey()).build()); } public static ReservedList persistReservedList(String listName, String... lines) { @@ -442,14 +446,14 @@ public final class DatabaseHelper { * Deletes "domain" and all history records, billing events, poll messages and subordinate hosts. */ public static void deleteTestDomain(Domain domain, DateTime now) { - Iterable billingEvents = getBillingEvents(domain); + Iterable billingEvents = getBillingEvents(domain); Iterable historyEntries = HistoryEntryDao.loadHistoryObjectsForResource(domain.createVKey()); Iterable pollMessages = loadAllOf(PollMessage.class); tm().transact( () -> { deleteResource(domain); - for (BillingEvent event : billingEvents) { + for (BillingBase event : billingEvents) { deleteResource(event); } for (PollMessage pollMessage : pollMessages) { @@ -521,9 +525,9 @@ public final class DatabaseHelper { .build(); } - public static BillingEvent.OneTime createBillingEventForTransfer( + public static BillingEvent createBillingEventForTransfer( Domain domain, DomainHistory historyEntry, DateTime costLookupTime, DateTime eventTime) { - return new BillingEvent.OneTime.Builder() + return new BillingEvent.Builder() .setReason(Reason.TRANSFER) .setTargetId(domain.getDomainName()) .setEventTime(eventTime) @@ -618,9 +622,9 @@ public final class DatabaseHelper { .setDomain(domain) .setRegistrarId(domain.getCreationRegistrarId()) .build()); - BillingEvent.Recurring autorenewEvent = + BillingRecurrence autorenewEvent = persistResource( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setReason(Reason.RENEW) .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setTargetId(domainName) @@ -660,13 +664,13 @@ public final class DatabaseHelper { .setDomain(domain) .setRegistrarId("TheRegistrar") .build()); - BillingEvent.OneTime transferBillingEvent = + BillingEvent transferBillingEvent = persistResource( createBillingEventForTransfer( domain, historyEntryDomainTransfer, requestTime, expirationTime)); - BillingEvent.Recurring gainingClientAutorenewEvent = + BillingRecurrence gainingClientAutorenewEvent = persistResource( - new BillingEvent.Recurring.Builder() + new BillingRecurrence.Builder() .setFlags(ImmutableSet.of(Flag.AUTO_RENEW)) .setReason(Reason.RENEW) .setTargetId(domain.getDomainName()) @@ -785,27 +789,27 @@ public final class DatabaseHelper { return newRegistrars.build(); } - public static Iterable getBillingEvents() { + public static Iterable getBillingEvents() { return tm().transact( () -> Iterables.concat( - tm().loadAllOf(BillingEvent.OneTime.class), - tm().loadAllOf(BillingEvent.Recurring.class), - tm().loadAllOf(BillingEvent.Cancellation.class))); + tm().loadAllOf(BillingEvent.class), + tm().loadAllOf(BillingRecurrence.class), + tm().loadAllOf(BillingCancellation.class))); } - private static Iterable getBillingEvents(EppResource resource) { + private static Iterable getBillingEvents(EppResource resource) { return tm().transact( () -> Iterables.concat( - tm().loadAllOfStream(BillingEvent.OneTime.class) + tm().loadAllOfStream(BillingEvent.class) .filter(oneTime -> oneTime.getDomainRepoId().equals(resource.getRepoId())) .collect(toImmutableList()), - tm().loadAllOfStream(BillingEvent.Recurring.class) + tm().loadAllOfStream(BillingRecurrence.class) .filter( - recurring -> recurring.getDomainRepoId().equals(resource.getRepoId())) + recurrence -> recurrence.getDomainRepoId().equals(resource.getRepoId())) .collect(toImmutableList()), - tm().loadAllOfStream(BillingEvent.Cancellation.class) + tm().loadAllOfStream(BillingCancellation.class) .filter( cancellation -> cancellation.getDomainRepoId().equals(resource.getRepoId())) @@ -813,33 +817,32 @@ public final class DatabaseHelper { } /** Assert that the actual billing event matches the expected one, ignoring IDs. */ - public static void assertBillingEventsEqual(BillingEvent actual, BillingEvent expected) { + public static void assertBillingEventsEqual(BillingBase actual, BillingBase expected) { assertAboutImmutableObjects().that(actual).isEqualExceptFields(expected, "id"); } /** Assert that the actual billing events match the expected ones, ignoring IDs and order. */ public static void assertBillingEventsEqual( - Iterable actual, Iterable expected) { + Iterable actual, Iterable expected) { assertThat(actual) .comparingElementsUsing(immutableObjectCorrespondence("id")) .containsExactlyElementsIn(expected); } /** Assert that the expected billing events are exactly the ones found in test database. */ - public static void assertBillingEvents(BillingEvent... expected) { + public static void assertBillingEvents(BillingBase... expected) { assertBillingEventsEqual(getBillingEvents(), asList(expected)); } /** Assert that the expected billing events set is exactly the one found in test database. */ - public static void assertBillingEvents(Set expected) { + public static void assertBillingEvents(Set expected) { assertBillingEventsEqual(getBillingEvents(), expected); } /** * Assert that the expected billing events are exactly the ones found for the given EppResource. */ - public static void assertBillingEventsForResource( - EppResource resource, BillingEvent... expected) { + public static void assertBillingEventsForResource(EppResource resource, BillingBase... expected) { assertBillingEventsEqual(getBillingEvents(resource), asList(expected)); } @@ -853,8 +856,8 @@ public final class DatabaseHelper { * *

Note: Prefer {@link #assertPollMessagesEqual} when that is suitable. */ - public static BillingEvent stripBillingEventId(BillingEvent billingEvent) { - return billingEvent.asBuilder().setId(1L).build(); + public static BillingBase stripBillingEventId(BillingBase billingBase) { + return billingBase.asBuilder().setId(1L).build(); } /** Assert that the actual poll message matches the expected one, ignoring IDs. */ diff --git a/core/src/test/java/google/registry/tools/CreatePackagePromotionCommandTest.java b/core/src/test/java/google/registry/tools/CreatePackagePromotionCommandTest.java index 4a46ffd61..04eb3e070 100644 --- a/core/src/test/java/google/registry/tools/CreatePackagePromotionCommandTest.java +++ b/core/src/test/java/google/registry/tools/CreatePackagePromotionCommandTest.java @@ -21,7 +21,7 @@ import static google.registry.testing.DatabaseHelper.persistResource; import static google.registry.util.DateTimeUtils.END_OF_TIME; import static org.junit.jupiter.api.Assertions.assertThrows; -import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; import google.registry.model.domain.token.AllocationToken; import google.registry.model.domain.token.AllocationToken.TokenType; import google.registry.model.domain.token.PackagePromotion; diff --git a/core/src/test/java/google/registry/tools/DomainLockUtilsTest.java b/core/src/test/java/google/registry/tools/DomainLockUtilsTest.java index a71a252e9..079119c91 100644 --- a/core/src/test/java/google/registry/tools/DomainLockUtilsTest.java +++ b/core/src/test/java/google/registry/tools/DomainLockUtilsTest.java @@ -35,8 +35,9 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import com.google.cloud.tasks.v2.HttpMethod; import com.google.common.collect.ImmutableList; import google.registry.batch.RelockDomainAction; +import google.registry.model.billing.BillingBase; +import google.registry.model.billing.BillingBase.Reason; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.Reason; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.domain.RegistryLock; @@ -558,11 +559,11 @@ public final class DomainLockUtilsTest { } private void assertBillingEvents(ImmutableList historyEntries) { - Set expectedEvents = + Set expectedEvents = historyEntries.stream() .map( entry -> - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.SERVER_STATUS) .setTargetId(domain.getForeignKey()) .setRegistrarId(domain.getCurrentSponsorRegistrarId()) diff --git a/core/src/test/java/google/registry/tools/EppLifecycleToolsTest.java b/core/src/test/java/google/registry/tools/EppLifecycleToolsTest.java index 79c4cbf39..021dd2b2f 100644 --- a/core/src/test/java/google/registry/tools/EppLifecycleToolsTest.java +++ b/core/src/test/java/google/registry/tools/EppLifecycleToolsTest.java @@ -23,8 +23,8 @@ import static google.registry.util.DateTimeUtils.END_OF_TIME; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import google.registry.flows.EppTestCase; +import google.registry.model.billing.BillingBase.Reason; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.Reason; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.reporting.HistoryEntry.Type; @@ -146,8 +146,8 @@ class EppLifecycleToolsTest extends EppTestCase { DateTime createTime = DateTime.parse("2000-06-01T00:02:00Z"); Domain domain = loadByForeignKey(Domain.class, "example.tld", DateTime.parse("2003-06-02T00:02:00Z")).get(); - BillingEvent.OneTime renewBillingEvent = - new BillingEvent.OneTime.Builder() + BillingEvent renewBillingEvent = + new BillingEvent.Builder() .setReason(Reason.RENEW) .setTargetId(domain.getDomainName()) .setRegistrarId(domain.getCurrentSponsorRegistrarId()) @@ -164,19 +164,19 @@ class EppLifecycleToolsTest extends EppTestCase { makeOneTimeCreateBillingEvent(domain, createTime), renewBillingEvent, // The initial autorenew billing event, which was closed at the time of the explicit renew. - makeRecurringBillingEvent( + makeRecurrence( domain, getOnlyHistoryEntryOfType(domain, Type.DOMAIN_CREATE, DomainHistory.class), createTime.plusYears(2), DateTime.parse("2000-06-07T00:00:00.000Z")), // The renew's autorenew billing event, which was closed at the time of the unrenew. - makeRecurringBillingEvent( + makeRecurrence( domain, getOnlyHistoryEntryOfType(domain, Type.DOMAIN_RENEW, DomainHistory.class), DateTime.parse("2006-06-01T00:02:00.000Z"), DateTime.parse("2001-06-07T00:00:00.000Z")), // The remaining active autorenew billing event which was created by the unrenew. - makeRecurringBillingEvent( + makeRecurrence( domain, getOnlyHistoryEntryOfType(domain, Type.SYNTHETIC, DomainHistory.class), DateTime.parse("2003-06-01T00:02:00.000Z"), diff --git a/core/src/test/java/google/registry/tools/GenerateAllocationTokensCommandTest.java b/core/src/test/java/google/registry/tools/GenerateAllocationTokensCommandTest.java index 1a95de87c..ed2b08d36 100644 --- a/core/src/test/java/google/registry/tools/GenerateAllocationTokensCommandTest.java +++ b/core/src/test/java/google/registry/tools/GenerateAllocationTokensCommandTest.java @@ -16,8 +16,8 @@ package google.registry.tools; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.truth.Truth.assertThat; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.NONPREMIUM; -import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.SPECIFIED; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.NONPREMIUM; +import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.SPECIFIED; import static google.registry.model.domain.token.AllocationToken.TokenType.SINGLE_USE; import static google.registry.model.domain.token.AllocationToken.TokenType.UNLIMITED_USE; import static google.registry.testing.DatabaseHelper.assertAllocationTokens; diff --git a/core/src/test/java/google/registry/tools/GetPackagePromotionCommandTest.java b/core/src/test/java/google/registry/tools/GetPackagePromotionCommandTest.java index b26ad3b86..4ae4520d3 100644 --- a/core/src/test/java/google/registry/tools/GetPackagePromotionCommandTest.java +++ b/core/src/test/java/google/registry/tools/GetPackagePromotionCommandTest.java @@ -21,7 +21,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import com.beust.jcommander.ParameterException; import com.google.common.collect.ImmutableSet; -import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; +import google.registry.model.billing.BillingBase.RenewalPriceBehavior; import google.registry.model.domain.token.AllocationToken; import google.registry.model.domain.token.AllocationToken.TokenType; import google.registry.model.domain.token.PackagePromotion; diff --git a/core/src/test/java/google/registry/tools/UnrenewDomainCommandTest.java b/core/src/test/java/google/registry/tools/UnrenewDomainCommandTest.java index a35cafb69..b83034819 100644 --- a/core/src/test/java/google/registry/tools/UnrenewDomainCommandTest.java +++ b/core/src/test/java/google/registry/tools/UnrenewDomainCommandTest.java @@ -34,9 +34,9 @@ import static google.registry.testing.HistoryEntrySubject.assertAboutHistoryEntr import static org.junit.jupiter.api.Assertions.assertThrows; import com.google.common.collect.ImmutableSet; -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.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.contact.Contact; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; @@ -127,7 +127,7 @@ public class UnrenewDomainCommandTest extends CommandTestCase { +public class UpdateBillingRecurrenceCommandTest extends CommandTestCase { @BeforeEach void beforeEach() { @@ -52,19 +52,21 @@ public class UpdateRecurrenceCommandTest extends CommandTestCase r.getId() != previousId).findFirst().get(); - assertThat(newRecurring.getRecurrenceEndTime()).isEqualTo(END_OF_TIME); - assertThat(newRecurring.getRenewalPriceBehavior()).isEqualTo(expectedBehavior); - assertThat(newRecurring.getRenewalPrice()).isEqualTo(Optional.ofNullable(expectedPrice)); - assertThat(newRecurring.getReason()).isEqualTo(Reason.RENEW); + BillingRecurrence newBillingRecurrence = + loadAllOf(BillingRecurrence.class).stream() + .filter(r -> r.getId() != previousId) + .findFirst() + .get(); + assertThat(newBillingRecurrence.getRecurrenceEndTime()).isEqualTo(END_OF_TIME); + assertThat(newBillingRecurrence.getRenewalPriceBehavior()).isEqualTo(expectedBehavior); + assertThat(newBillingRecurrence.getRenewalPrice()) + .isEqualTo(Optional.ofNullable(expectedPrice)); + assertThat(newBillingRecurrence.getReason()).isEqualTo(Reason.RENEW); DomainHistory newHistory = loadAllOf(DomainHistory.class).stream() diff --git a/core/src/test/java/google/registry/tools/UpdateDomainCommandTest.java b/core/src/test/java/google/registry/tools/UpdateDomainCommandTest.java index ee47d5762..dde245fdf 100644 --- a/core/src/test/java/google/registry/tools/UpdateDomainCommandTest.java +++ b/core/src/test/java/google/registry/tools/UpdateDomainCommandTest.java @@ -32,9 +32,9 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import com.beust.jcommander.ParameterException; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -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.BillingBase.Flag; +import google.registry.model.billing.BillingBase.Reason; +import google.registry.model.billing.BillingRecurrence; import google.registry.model.contact.Contact; import google.registry.model.domain.DesignatedContact; import google.registry.model.domain.Domain; @@ -324,9 +324,9 @@ class UpdateDomainCommandTest extends EppToolCommandTestCase { +/** Tests for {@link CreateCancellationsForBillingEventsCommand}. */ +public class CreateCancellationsForBillingEventsCommandTest + extends CommandTestCase { private Domain domain; - private OneTime oneTimeToCancel; + private BillingEvent billingEventToCancel; @BeforeEach void beforeEach() { @@ -58,61 +58,61 @@ public class CreateCancellationsForOneTimesCommandTest fakeClock.nowUtc(), fakeClock.nowUtc(), fakeClock.nowUtc().plusYears(2)); - oneTimeToCancel = createOneTime(); + billingEventToCancel = createBillingEvent(); } @Test void testSimpleDelete() throws Exception { - assertThat(DatabaseHelper.loadAllOf(Cancellation.class)).isEmpty(); - runCommandForced(String.valueOf(oneTimeToCancel.getId())); + assertThat(DatabaseHelper.loadAllOf(BillingCancellation.class)).isEmpty(); + runCommandForced(String.valueOf(billingEventToCancel.getId())); assertBillingEventCancelled(); - assertInStdout("Added Cancellation for OneTime with ID 9"); - assertInStdout("Created 1 Cancellation event(s)"); + assertInStdout("Added BillingCancellation for BillingEvent with ID 9"); + assertInStdout("Created 1 BillingCancellation event(s)"); } @Test void testSuccess_oneExistsOneDoesnt() throws Exception { - runCommandForced(String.valueOf(oneTimeToCancel.getId()), "9001"); + runCommandForced(String.valueOf(billingEventToCancel.getId()), "9001"); assertBillingEventCancelled(); - assertInStdout("Found 1 OneTime(s) to cancel"); - assertInStdout("Missing OneTime(s) for IDs [9001]"); - assertInStdout("Added Cancellation for OneTime with ID 9"); - assertInStdout("Created 1 Cancellation event(s)"); + assertInStdout("Found 1 BillingEvent(s) to cancel"); + assertInStdout("Missing BillingEvent(s) for IDs [9001]"); + assertInStdout("Added BillingCancellation for BillingEvent with ID 9"); + assertInStdout("Created 1 BillingCancellation event(s)"); } @Test void testSuccess_multipleCancellations() throws Exception { - OneTime secondToCancel = createOneTime(); - assertThat(DatabaseHelper.loadAllOf(Cancellation.class)).isEmpty(); + BillingEvent secondToCancel = createBillingEvent(); + assertThat(DatabaseHelper.loadAllOf(BillingCancellation.class)).isEmpty(); runCommandForced( - String.valueOf(oneTimeToCancel.getId()), String.valueOf(secondToCancel.getId())); - assertBillingEventCancelled(oneTimeToCancel.getId()); + String.valueOf(billingEventToCancel.getId()), String.valueOf(secondToCancel.getId())); + assertBillingEventCancelled(billingEventToCancel.getId()); assertBillingEventCancelled(secondToCancel.getId()); - assertInStdout("Create cancellations for 2 OneTime(s)?"); - assertInStdout("Added Cancellation for OneTime with ID 9"); - assertInStdout("Added Cancellation for OneTime with ID 10"); - assertInStdout("Created 2 Cancellation event(s)"); + assertInStdout("Create cancellations for 2 BillingEvent(s)?"); + assertInStdout("Added BillingCancellation for BillingEvent with ID 9"); + assertInStdout("Added BillingCancellation for BillingEvent with ID 10"); + assertInStdout("Created 2 BillingCancellation event(s)"); } @Test void testAlreadyCancelled() throws Exception { // multiple runs / cancellations should be a no-op - runCommandForced(String.valueOf(oneTimeToCancel.getId())); + runCommandForced(String.valueOf(billingEventToCancel.getId())); assertBillingEventCancelled(); - runCommandForced(String.valueOf(oneTimeToCancel.getId())); + runCommandForced(String.valueOf(billingEventToCancel.getId())); assertBillingEventCancelled(); - assertThat(DatabaseHelper.loadAllOf(Cancellation.class)).hasSize(1); - assertInStdout("Found 0 OneTime(s) to cancel"); - assertInStdout("The following OneTime IDs were already cancelled: [9]"); + assertThat(DatabaseHelper.loadAllOf(BillingCancellation.class)).hasSize(1); + assertInStdout("Found 0 BillingEvent(s) to cancel"); + assertInStdout("The following BillingEvent IDs were already cancelled: [9]"); } @Test void testFailure_doesntExist() throws Exception { runCommandForced("9001"); - assertThat(DatabaseHelper.loadAllOf(Cancellation.class)).isEmpty(); - assertInStdout("Found 0 OneTime(s) to cancel"); - assertInStdout("Missing OneTime(s) for IDs [9001]"); - assertInStdout("Created 0 Cancellation event(s)"); + assertThat(DatabaseHelper.loadAllOf(BillingCancellation.class)).isEmpty(); + assertInStdout("Found 0 BillingEvent(s) to cancel"); + assertInStdout("Missing BillingEvent(s) for IDs [9001]"); + assertInStdout("Created 0 BillingCancellation event(s)"); } @Test @@ -120,9 +120,9 @@ public class CreateCancellationsForOneTimesCommandTest assertThrows(ParameterException.class, this::runCommandForced); } - private OneTime createOneTime() { + private BillingEvent createBillingEvent() { return persistResource( - new OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.CREATE) .setTargetId(domain.getDomainName()) .setRegistrarId("TheRegistrar") @@ -139,13 +139,14 @@ public class CreateCancellationsForOneTimesCommandTest } private void assertBillingEventCancelled() { - assertBillingEventCancelled(oneTimeToCancel.getId()); + assertBillingEventCancelled(billingEventToCancel.getId()); } - private void assertBillingEventCancelled(long oneTimeId) { + private void assertBillingEventCancelled(long billingEventId) { assertThat( - DatabaseHelper.loadAllOf(Cancellation.class).stream() - .anyMatch(c -> c.getEventKey().equals(VKey.create(OneTime.class, oneTimeId)))) + DatabaseHelper.loadAllOf(BillingCancellation.class).stream() + .anyMatch( + c -> c.getEventKey().equals(VKey.create(BillingEvent.class, billingEventId)))) .isTrue(); } } diff --git a/core/src/test/java/google/registry/ui/server/registrar/RegistryLockVerifyActionTest.java b/core/src/test/java/google/registry/ui/server/registrar/RegistryLockVerifyActionTest.java index 4f37c7bce..c24e4166b 100644 --- a/core/src/test/java/google/registry/ui/server/registrar/RegistryLockVerifyActionTest.java +++ b/core/src/test/java/google/registry/ui/server/registrar/RegistryLockVerifyActionTest.java @@ -32,8 +32,8 @@ import com.google.appengine.api.users.User; import com.google.appengine.api.users.UserService; import com.google.appengine.api.users.UserServiceFactory; import com.google.common.collect.ImmutableMap; +import google.registry.model.billing.BillingBase.Reason; import google.registry.model.billing.BillingEvent; -import google.registry.model.billing.BillingEvent.Reason; import google.registry.model.domain.Domain; import google.registry.model.domain.DomainHistory; import google.registry.model.domain.RegistryLock; @@ -313,7 +313,7 @@ final class RegistryLockVerifyActionTest { private void assertBillingEvent(DomainHistory historyEntry) { DatabaseHelper.assertBillingEvents( - new BillingEvent.OneTime.Builder() + new BillingEvent.Builder() .setReason(Reason.SERVER_STATUS) .setTargetId(domain.getForeignKey()) .setRegistrarId(domain.getCurrentSponsorRegistrarId()) diff --git a/core/src/test/resources/google/registry/module/backend/backend_routing.txt b/core/src/test/resources/google/registry/module/backend/backend_routing.txt index 52edb259b..62b3e241b 100644 --- a/core/src/test/resources/google/registry/module/backend/backend_routing.txt +++ b/core/src/test/resources/google/registry/module/backend/backend_routing.txt @@ -8,7 +8,7 @@ PATH CLASS /_dr/task/deleteLoadTestData DeleteLoadTestDataAction POST n INTERNAL,API APP ADMIN /_dr/task/deleteProberData DeleteProberDataAction POST n INTERNAL,API APP ADMIN /_dr/task/executeCannedScript CannedScriptExecutionAction POST y INTERNAL,API APP ADMIN -/_dr/task/expandRecurringBillingEvents ExpandRecurringBillingEventsAction GET n INTERNAL,API APP ADMIN +/_dr/task/expandBillingRecurrences ExpandBillingRecurrencesAction GET n INTERNAL,API APP ADMIN /_dr/task/exportDomainLists ExportDomainListsAction POST n INTERNAL,API APP ADMIN /_dr/task/exportPremiumTerms ExportPremiumTermsAction POST n INTERNAL,API APP ADMIN /_dr/task/exportReservedTerms ExportReservedTermsAction POST n INTERNAL,API APP ADMIN diff --git a/db/src/main/resources/sql/er_diagram/brief_er_diagram.html b/db/src/main/resources/sql/er_diagram/brief_er_diagram.html index 908009350..c8857f099 100644 --- a/db/src/main/resources/sql/er_diagram/brief_er_diagram.html +++ b/db/src/main/resources/sql/er_diagram/brief_er_diagram.html @@ -261,7 +261,7 @@ td.section { generated on - 2023-03-28 19:28:38.164363 + 2023-04-21 18:52:34.656577 last flyway file @@ -284,7 +284,7 @@ td.section { generated on - 2023-03-28 19:28:38.164363 + 2023-04-21 18:52:34.656577 diff --git a/db/src/main/resources/sql/er_diagram/full_er_diagram.html b/db/src/main/resources/sql/er_diagram/full_er_diagram.html index e6bdbaa4b..2678fab9f 100644 --- a/db/src/main/resources/sql/er_diagram/full_er_diagram.html +++ b/db/src/main/resources/sql/er_diagram/full_er_diagram.html @@ -261,7 +261,7 @@ td.section { generated on - 2023-03-28 19:28:35.770605 + 2023-04-21 18:52:32.303901 last flyway file @@ -284,7 +284,7 @@ td.section { generated on - 2023-03-28 19:28:35.770605 + 2023-04-21 18:52:32.303901 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 d8577dfc9..8d641e199 100644 --- a/db/src/main/resources/sql/schema/db-schema.sql.generated +++ b/db/src/main/resources/sql/schema/db-schema.sql.generated @@ -41,9 +41,9 @@ flags text[], reason text not null, domain_name text not null, - billing_time timestamptz, billing_event_id int8, billing_recurrence_id int8, + billing_time timestamptz, primary key (billing_cancellation_id) ); diff --git a/docs/code-structure.md b/docs/code-structure.md index 4e604005e..004f3bfff 100644 --- a/docs/code-structure.md +++ b/docs/code-structure.md @@ -72,8 +72,8 @@ The following cursor types are defined: * **`RDE_UPLOAD`** - RDE (thick) escrow deposit upload * **`RDE_UPLOAD_SFTP`** - Cursor that tracks the last time we talked to the escrow provider's SFTP server for a given TLD. -* **`RECURRING_BILLING`** - Expansion of `Recurring` (renew) billing events - into `OneTime` events. +* **`RECURRING_BILLING`** - Expansion of `BillingRecurrence` (renew) billing events + into one-time `BillingEvent`s. * **`SYNC_REGISTRAR_SHEET`** - Tracks the last time the registrar spreadsheet was successfully synced. @@ -254,21 +254,19 @@ full list of which is represented by `BillingEvent.Reason`): * Server status changes * Domain transfers -A `BillingEvent` can also contain one or more `BillingEvent.Flag` flags that +A `BillingBase` can also contain one or more `BillingBase.Flag` flags that provide additional metadata about the billing event (e.g. the application phase during which the domain was applied for). -All `BillingEvent` entities contain a parent `Key` to identify the -mutation that spawned the `BillingEvent`. +All `BillingBase` entities contain a parent `VKey` to identify the +mutation that spawned the `BillingBase`. There are 4 types of billing events, all of which extend the abstract -`BillingEvent` base class: +`BillingBase` base class: -* **`OneTime`**, a one-time billing event. -* **`Recurring`**, a recurring billing event (used for events such as domain +* **`BillingEvent`**, a one-time billing event. +* **`BillingRecurrence`**, a recurring billing event (used for events such as domain renewals). -* **`Cancellation`**, which represents the cancellation of either a `OneTime` - or `Recurring` billing event. This is implemented as a distinct event to +* **`BillingCancellation`**, which represents the cancellation of either a `OneTime` + or `BillingRecurrence` billing event. This is implemented as a distinct event to preserve the immutability of billing events. -* **`Modification`**, a change to an existing `OneTime` billing event (for - instance, to represent a discount or refund). diff --git a/release/cloudbuild-nomulus.yaml b/release/cloudbuild-nomulus.yaml index 4d8adac40..44554ecea 100644 --- a/release/cloudbuild-nomulus.yaml +++ b/release/cloudbuild-nomulus.yaml @@ -92,8 +92,8 @@ steps: google/registry/beam/spec11_pipeline_metadata.json \ google.registry.beam.billing.InvoicingPipeline \ google/registry/beam/invoicing_pipeline_metadata.json \ - google.registry.beam.billing.ExpandRecurringBillingEventsPipeline \ - google/registry/beam/expand_recurring_billing_events_pipeline_metadata.json \ + google.registry.beam.billing.ExpandBillingRecurrencesPipeline \ + google/registry/beam/expand_billing_recurrences_metadata.json \ google.registry.beam.rde.RdePipeline \ google/registry/beam/rde_pipeline_metadata.json \ google.registry.beam.resave.ResaveAllEppResourcesPipeline \