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
This commit is contained in:
gbrodman 2023-04-28 14:27:37 -04:00 committed by GitHub
parent 75acd574cc
commit 668a7a47a2
97 changed files with 2086 additions and 1946 deletions

View file

@ -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 :
[

View file

@ -105,16 +105,15 @@ public class BatchModule {
}
@Provides
@Parameter(ExpandRecurringBillingEventsAction.PARAM_START_TIME)
@Parameter(ExpandBillingRecurrencesAction.PARAM_START_TIME)
static Optional<DateTime> 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<DateTime> 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

View file

@ -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");

View file

@ -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. */

View file

@ -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.
*
* <p>This pipeline works in three stages:
*
* <ul>
* <li>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)}.
* <li>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.
* <li>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)}.
* <li>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.
* <li>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}.
* </ul>
*
* <p>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.
* <p>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.
*
* <p>To stage this template locally, run {@code ./nom_build :core:sBP --environment=alpha \
* --pipeline=expandBilling}.
*
* <p>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 <a href="https://cloud.google.com/dataflow/docs/guides/templates/using-flex-templates">Using
* Flex Templates</a>
*/
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<KV<Integer, Long>> recurringIds = getRecurringsInScope(pipeline);
PCollection<Void> expanded = expandRecurrings(recurringIds);
PCollection<KV<Integer, Long>> recurrenceIds = getRecurrencesInScope(pipeline);
PCollection<Void> expanded = expandRecurrences(recurrenceIds);
if (!isDryRun && advanceCursor) {
advanceCursor(expanded);
}
}
PCollection<KV<Integer, Long>> getRecurringsInScope(Pipeline pipeline) {
PCollection<KV<Integer, Long>> 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<Void> expandRecurrings(PCollection<KV<Integer, Long>> recurringIds) {
return recurringIds
private PCollection<Void> expandRecurrences(PCollection<KV<Integer, Long>> recurrenceIds) {
return recurrenceIds
.apply(
"Group into batches",
GroupIntoBatches.<Integer, Long>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<ImmutableObject> 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<ImmutableObject> results) {
Recurring recurring = tm().loadByKey(Recurring.createVKey(recurringId));
private void expandOneRecurrence(
Long recurrenceId, ImmutableSet.Builder<ImmutableObject> 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<DateTime> 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<DateTime> 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<Void> 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

View file

@ -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")

View file

@ -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<BillingEvent> billingEvents = readFromCloudSql(options, pipeline);
PCollection<google.registry.beam.billing.BillingEvent> billingEvents =
readFromCloudSql(options, pipeline);
saveInvoiceCsv(billingEvents, options);
saveDetailedCsv(billingEvents, options);
}
static PCollection<BillingEvent> readFromCloudSql(
static PCollection<google.registry.beam.billing.BillingEvent> readFromCloudSql(
InvoicingPipelineOptions options, Pipeline pipeline) {
Read<Object[], BillingEvent> read =
RegistryJpaIO.<Object[], BillingEvent>read(
Read<Object[], google.registry.beam.billing.BillingEvent> read =
RegistryJpaIO.<Object[], google.registry.beam.billing.BillingEvent>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<BillingEvent> billingEventsWithNulls =
PCollection<google.registry.beam.billing.BillingEvent> billingEventsWithNulls =
pipeline.apply("Read BillingEvents from Cloud SQL", read);
// Remove null billing events
return billingEventsWithNulls.apply(Filter.by(Objects::nonNull));
}
private static Optional<BillingEvent> parseRow(Object[] row) {
OneTime oneTime = (OneTime) row[0];
private static Optional<google.registry.beam.billing.BillingEvent> 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<BillingEvent>, PCollection<String>> {
extends PTransform<
PCollection<google.registry.beam.billing.BillingEvent>, PCollection<String>> {
private static final long serialVersionUID = -8090619008258393728L;
@Override
public PCollection<String> expand(PCollection<BillingEvent> input) {
public PCollection<String> expand(
PCollection<google.registry.beam.billing.BillingEvent> 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<BillingEvent> billingEvents, InvoicingPipelineOptions options) {
PCollection<google.registry.beam.billing.BillingEvent> 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<BillingEvent> billingEvents, InvoicingPipelineOptions options) {
PCollection<google.registry.beam.billing.BillingEvent> billingEvents,
InvoicingPipelineOptions options) {
String yearMonth = options.getYearMonth();
billingEvents.apply(
"Write detailed report for each registrar-tld pair",
FileIO.<String, BillingEvent>writeDynamic()
FileIO.<String, google.registry.beam.billing.BillingEvent>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. */

View file

@ -80,12 +80,13 @@
</task>
<task>
<url><![CDATA[/_dr/task/expandRecurringBillingEvents?advanceCursor]]></url>
<name>expandRecurringBillingEvents</name>
<url><![CDATA[/_dr/task/expandBillingRecurrences?advanceCursor]]></url>
<name>expandBillingRecurrences</name>
<description>
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.
</description>
<schedule>0 3 * * *</schedule>
</task>

View file

@ -246,10 +246,10 @@
<url-pattern>/_dr/task/refreshDnsOnHostRename</url-pattern>
</servlet-mapping>
<!-- Action to expand recurring billing events into OneTimes. -->
<!-- Action to expand BillingRecurrences into BillingEvents. -->
<servlet-mapping>
<servlet-name>backend-servlet</servlet-name>
<url-pattern>/_dr/task/expandRecurringBillingEvents</url-pattern>
<url-pattern>/_dr/task/expandBillingRecurrences</url-pattern>
</servlet-mapping>
<!-- Background action to delete domains past end of autorenewal. -->

View file

@ -122,12 +122,13 @@
</task>
<task>
<url><![CDATA[/_dr/task/expandRecurringBillingEvents?advanceCursor]]></url>
<name>expandRecurringBillingEvents</name>
<url><![CDATA[/_dr/task/expandBillingRecurrences?advanceCursor]]></url>
<name>expandBillingRecurrences</name>
<description>
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.
</description>
<schedule>0 3 * * *</schedule>
</task>
@ -253,7 +254,7 @@
reports to the associated registrars' drive folders.
See GenerateInvoicesAction for more details.
</description>
<!--WARNING: This must occur AFTER expandRecurringBillingEvents and AFTER exportSnapshot, as
<!--WARNING: This must occur AFTER expandBillingRecurrences and AFTER exportSnapshot, as
it uses Bigquery as the source of truth for billable events. ExportSnapshot usually takes
about 2 hours to complete, so we give 11 hours to be safe. Normally, we give 24+ hours (see
icannReportingStaging), but the invoicing team prefers receiving the e-mail on the first of

View file

@ -82,12 +82,13 @@
</task>
<task>
<url><![CDATA[/_dr/task/expandRecurringBillingEvents?advanceCursor]]></url>
<name>expandRecurringBillingEvents</name>
<url><![CDATA[/_dr/task/expandBillingRecurrences?advanceCursor]]></url>
<name>expandBillingRecurrences</name>
<description>
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.
</description>
<schedule>0 3 * * *</schedule>
</task>

View file

@ -59,7 +59,7 @@ import google.registry.flows.domain.token.AllocationTokenFlowUtils.AllocationTok
import google.registry.flows.domain.token.AllocationTokenFlowUtils.AllocationTokenNotValidForTldException;
import google.registry.model.EppResource;
import google.registry.model.ForeignKeyUtils;
import google.registry.model.billing.BillingEvent;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainCommand.Check;
import google.registry.model.domain.fee.FeeCheckCommandExtension;
@ -271,8 +271,7 @@ public final class DomainCheckFlow implements Flow {
new ImmutableList.Builder<>();
ImmutableMap<String, Domain> domainObjs =
loadDomainsForRestoreChecks(feeCheck, domainNames, existingDomains);
ImmutableMap<String, BillingEvent.Recurring> recurrences =
loadRecurrencesForDomains(domainObjs);
ImmutableMap<String, BillingRecurrence> recurrences = loadRecurrencesForDomains(domainObjs);
for (FeeCheckCommandExtensionItem feeCheckItem : feeCheck.getItems()) {
for (String domainName : getDomainNamesToCheckForFee(feeCheckItem, domainNames.keySet())) {
@ -377,16 +376,15 @@ public final class DomainCheckFlow implements Flow {
Maps.transformEntries(existingDomainsToLoad, (k, v) -> (Domain) loadedDomains.get(v)));
}
private ImmutableMap<String, BillingEvent.Recurring> loadRecurrencesForDomains(
private ImmutableMap<String, BillingRecurrence> loadRecurrencesForDomains(
ImmutableMap<String, Domain> domainObjs) {
return tm().transact(
() -> {
ImmutableMap<VKey<? extends BillingEvent.Recurring>, BillingEvent.Recurring>
recurrences =
tm().loadByKeys(
domainObjs.values().stream()
.map(Domain::getAutorenewBillingEvent)
.collect(toImmutableSet()));
ImmutableMap<VKey<? extends BillingRecurrence>, BillingRecurrence> recurrences =
tm().loadByKeys(
domainObjs.values().stream()
.map(Domain::getAutorenewBillingEvent)
.collect(toImmutableSet()));
return ImmutableMap.copyOf(
Maps.transformValues(
domainObjs, d -> recurrences.get(d.getAutorenewBillingEvent())));

View file

@ -77,11 +77,11 @@ import google.registry.flows.domain.token.AllocationTokenFlowUtils;
import google.registry.flows.exceptions.ResourceAlreadyExistsForThisClientException;
import google.registry.flows.exceptions.ResourceCreateContentionException;
import google.registry.model.ImmutableObject;
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.Recurring;
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainCommand;
import google.registry.model.domain.DomainCommand.Create;
@ -348,8 +348,8 @@ public final class DomainCreateFlow implements TransactionalFlow {
HistoryEntryId domainHistoryId = new HistoryEntryId(repoId, historyRevisionId);
historyBuilder.setRevisionId(historyRevisionId);
// Bill for the create.
BillingEvent.OneTime createBillingEvent =
createOneTimeBillingEvent(
BillingEvent createBillingEvent =
createBillingEvent(
tld,
isAnchorTenant,
isSunriseCreate,
@ -360,7 +360,7 @@ public final class DomainCreateFlow implements TransactionalFlow {
allocationToken,
now);
// Create a new autorenew billing event and poll message starting at the expiration time.
BillingEvent.Recurring autorenewBillingEvent =
BillingRecurrence autorenewBillingEvent =
createAutorenewBillingEvent(
domainHistoryId,
registrationExpirationTime,
@ -572,7 +572,7 @@ public final class DomainCreateFlow implements TransactionalFlow {
return historyBuilder.setType(DOMAIN_CREATE).setPeriod(period).setDomain(domain).build();
}
private BillingEvent.OneTime createOneTimeBillingEvent(
private BillingEvent createBillingEvent(
Tld tld,
boolean isAnchorTenant,
boolean isSunriseCreate,
@ -594,7 +594,7 @@ public final class DomainCreateFlow implements TransactionalFlow {
// it if it's reserved for other reasons.
flagsBuilder.add(Flag.RESERVED);
}
return new BillingEvent.OneTime.Builder()
return new BillingEvent.Builder()
.setReason(Reason.CREATE)
.setTargetId(targetId)
.setRegistrarId(registrarId)
@ -612,11 +612,11 @@ public final class DomainCreateFlow implements TransactionalFlow {
.build();
}
private Recurring createAutorenewBillingEvent(
private BillingRecurrence createAutorenewBillingEvent(
HistoryEntryId domainHistoryId,
DateTime registrationExpirationTime,
RenewalPriceInfo renewalpriceInfo) {
return new BillingEvent.Recurring.Builder()
return new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId(targetId)
@ -640,9 +640,9 @@ public final class DomainCreateFlow implements TransactionalFlow {
.build();
}
private static BillingEvent.OneTime createEapBillingEvent(
FeesAndCredits feesAndCredits, BillingEvent.OneTime createBillingEvent) {
return new BillingEvent.OneTime.Builder()
private static BillingEvent createEapBillingEvent(
FeesAndCredits feesAndCredits, BillingEvent createBillingEvent) {
return new BillingEvent.Builder()
.setReason(Reason.FEE_EARLY_ACCESS)
.setTargetId(createBillingEvent.getTargetId())
.setRegistrarId(createBillingEvent.getRegistrarId())
@ -671,7 +671,7 @@ public final class DomainCreateFlow implements TransactionalFlow {
/**
* Determines the {@link RenewalPriceBehavior} and the renewal price that needs be stored in the
* {@link Recurring} billing events.
* {@link BillingRecurrence} billing events.
*
* <p>By default, the renewal price is calculated during the process of renewal. Renewal price
* should be the createCost if and only if the renewal price behavior in the {@link
@ -697,7 +697,7 @@ public final class DomainCreateFlow implements TransactionalFlow {
}
}
/** A class to store renewal info used in {@link Recurring} billing events. */
/** A class to store renewal info used in {@link BillingRecurrence} billing events. */
@AutoValue
public abstract static class RenewalPriceInfo {
static DomainCreateFlow.RenewalPriceInfo create(

View file

@ -61,8 +61,9 @@ import google.registry.flows.custom.DomainDeleteFlowCustomLogic.BeforeResponseRe
import google.registry.flows.custom.DomainDeleteFlowCustomLogic.BeforeSaveParameters;
import google.registry.flows.custom.EntityChanges;
import google.registry.model.ImmutableObject;
import google.registry.model.billing.BillingCancellation;
import google.registry.model.billing.BillingEvent;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.GracePeriod;
@ -232,15 +233,15 @@ public final class DomainDeleteFlow implements TransactionalFlow {
// No cancellation is written if the grace period was not for a billable event.
if (gracePeriod.hasBillingEvent()) {
entitiesToSave.add(
BillingEvent.Cancellation.forGracePeriod(gracePeriod, now, domainHistoryId, targetId));
if (gracePeriod.getOneTimeBillingEvent() != null) {
BillingCancellation.forGracePeriod(gracePeriod, now, domainHistoryId, targetId));
if (gracePeriod.getBillingEvent() != null) {
// Take the amount of registration time being refunded off the expiration time.
// This can be either add grace periods or renew grace periods.
BillingEvent.OneTime oneTime = tm().loadByKey(gracePeriod.getOneTimeBillingEvent());
newExpirationTime = newExpirationTime.minusYears(oneTime.getPeriodYears());
} else if (gracePeriod.getRecurringBillingEvent() != null) {
BillingEvent billingEvent = tm().loadByKey(gracePeriod.getBillingEvent());
newExpirationTime = newExpirationTime.minusYears(billingEvent.getPeriodYears());
} else if (gracePeriod.getBillingRecurrence() != null) {
// Take 1 year off the registration if in the autorenew grace period (no need to load the
// recurring billing event; all autorenews are for 1 year).
// recurrence billing event; all autorenews are for 1 year).
newExpirationTime = newExpirationTime.minusYears(1);
}
}
@ -252,11 +253,12 @@ public final class DomainDeleteFlow implements TransactionalFlow {
buildDomainHistory(newDomain, tld, now, durationUntilDelete, inAddGracePeriod);
handlePendingTransferOnDelete(existingDomain, newDomain, now, domainHistory);
// Close the autorenew billing event and poll message. This may delete the poll message. Store
// the updated recurring billing event, we'll need it later and can't reload it.
Recurring existingRecurring = tm().loadByKey(existingDomain.getAutorenewBillingEvent());
BillingEvent.Recurring recurringBillingEvent =
// the updated recurrence billing event, we'll need it later and can't reload it.
BillingRecurrence existingBillingRecurrence =
tm().loadByKey(existingDomain.getAutorenewBillingEvent());
BillingRecurrence billingRecurrence =
updateAutorenewRecurrenceEndTime(
existingDomain, existingRecurring, now, domainHistory.getHistoryEntryId());
existingDomain, existingBillingRecurrence, now, domainHistory.getHistoryEntryId());
// If there's a pending transfer, the gaining client's autorenew billing
// event and poll message will already have been deleted in
// ResourceDeleteFlow since it's listed in serverApproveEntities.
@ -280,7 +282,7 @@ public final class DomainDeleteFlow implements TransactionalFlow {
? SUCCESS_WITH_ACTION_PENDING
: SUCCESS)
.setResponseExtensions(
getResponseExtensions(recurringBillingEvent, existingDomain, now))
getResponseExtensions(billingRecurrence, existingDomain, now))
.build());
persistEntityChanges(entityChanges);
return responseBuilder
@ -380,7 +382,7 @@ public final class DomainDeleteFlow implements TransactionalFlow {
@Nullable
private ImmutableList<FeeTransformResponseExtension> getResponseExtensions(
BillingEvent.Recurring recurringBillingEvent, Domain existingDomain, DateTime now) {
BillingRecurrence billingRecurrence, Domain existingDomain, DateTime now) {
FeeTransformResponseExtension.Builder feeResponseBuilder = getDeleteResponseBuilder();
if (feeResponseBuilder == null) {
return ImmutableList.of();
@ -388,7 +390,7 @@ public final class DomainDeleteFlow implements TransactionalFlow {
ImmutableList.Builder<Credit> creditsBuilder = new ImmutableList.Builder<>();
for (GracePeriod gracePeriod : existingDomain.getGracePeriods()) {
if (gracePeriod.hasBillingEvent()) {
Money cost = getGracePeriodCost(recurringBillingEvent, gracePeriod, now);
Money cost = getGracePeriodCost(billingRecurrence, gracePeriod, now);
creditsBuilder.add(Credit.create(
cost.negated().getAmount(), FeeType.CREDIT, gracePeriod.getType().getXmlName()));
feeResponseBuilder.setCurrency(checkNotNull(cost.getCurrencyUnit()));
@ -402,14 +404,14 @@ public final class DomainDeleteFlow implements TransactionalFlow {
}
private Money getGracePeriodCost(
BillingEvent.Recurring recurringBillingEvent, GracePeriod gracePeriod, DateTime now) {
BillingRecurrence billingRecurrence, GracePeriod gracePeriod, DateTime now) {
if (gracePeriod.getType() == GracePeriodStatus.AUTO_RENEW) {
// If we updated the autorenew billing event, reuse it.
DateTime autoRenewTime =
recurringBillingEvent.getRecurrenceTimeOfYear().getLastInstanceBeforeOrAt(now);
billingRecurrence.getRecurrenceTimeOfYear().getLastInstanceBeforeOrAt(now);
return getDomainRenewCost(targetId, autoRenewTime, 1);
}
return tm().loadByKey(checkNotNull(gracePeriod.getOneTimeBillingEvent())).getCost();
return tm().loadByKey(checkNotNull(gracePeriod.getBillingEvent())).getCost();
}
@Nullable

View file

@ -77,10 +77,9 @@ import google.registry.flows.EppException.UnimplementedOptionException;
import google.registry.flows.domain.token.AllocationTokenFlowUtils;
import google.registry.flows.exceptions.ResourceHasClientUpdateProhibitedException;
import google.registry.model.EppResource;
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.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.DesignatedContact.Type;
@ -556,8 +555,8 @@ public class DomainFlowUtils {
* Fills in a builder with the data needed for an autorenew billing event for this domain. This
* does not copy over the id of the current autorenew billing event.
*/
public static BillingEvent.Recurring.Builder newAutorenewBillingEvent(Domain domain) {
return new BillingEvent.Recurring.Builder()
public static BillingRecurrence.Builder newAutorenewBillingEvent(Domain domain) {
return new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId(domain.getDomainName())
@ -584,11 +583,11 @@ public class DomainFlowUtils {
* (if opening the message interval). This may cause an autorenew billing event to have an end
* time earlier than its event time (i.e. if it's being ended before it was ever triggered).
*
* <p>Returns the new autorenew recurring billing event.
* <p>Returns the new autorenew recurrence billing event.
*/
public static Recurring updateAutorenewRecurrenceEndTime(
public static BillingRecurrence updateAutorenewRecurrenceEndTime(
Domain domain,
Recurring existingRecurring,
BillingRecurrence existingBillingRecurrence,
DateTime newEndTime,
@Nullable HistoryEntryId historyId) {
Optional<PollMessage.Autorenew> autorenewPollMessage =
@ -623,9 +622,10 @@ public class DomainFlowUtils {
tm().put(updatedAutorenewPollMessage);
}
Recurring newRecurring = existingRecurring.asBuilder().setRecurrenceEndTime(newEndTime).build();
tm().put(newRecurring);
return newRecurring;
BillingRecurrence newBillingRecurrence =
existingBillingRecurrence.asBuilder().setRecurrenceEndTime(newEndTime).build();
tm().put(newBillingRecurrence);
return newBillingRecurrence;
}
/**
@ -642,7 +642,7 @@ public class DomainFlowUtils {
DomainPricingLogic pricingLogic,
Optional<AllocationToken> allocationToken,
boolean isAvailable,
@Nullable Recurring recurringBillingEvent)
@Nullable BillingRecurrence billingRecurrence)
throws EppException {
DateTime now = currentDate;
// Use the custom effective date specified in the fee check request, if there is one.
@ -698,7 +698,7 @@ public class DomainFlowUtils {
fees =
pricingLogic
.getRenewPrice(
tld, domainNameString, now, years, recurringBillingEvent, allocationToken)
tld, domainNameString, now, years, billingRecurrence, allocationToken)
.getFees();
break;
case RESTORE:
@ -724,9 +724,7 @@ public class DomainFlowUtils {
}
builder.setAvailIfSupported(true);
fees =
pricingLogic
.getTransferPrice(tld, domainNameString, now, recurringBillingEvent)
.getFees();
pricingLogic.getTransferPrice(tld, domainNameString, now, billingRecurrence).getFees();
break;
case UPDATE:
builder.setAvailIfSupported(true);

View file

@ -29,7 +29,7 @@ import google.registry.flows.custom.DomainPricingCustomLogic.RenewPriceParameter
import google.registry.flows.custom.DomainPricingCustomLogic.RestorePriceParameters;
import google.registry.flows.custom.DomainPricingCustomLogic.TransferPriceParameters;
import google.registry.flows.custom.DomainPricingCustomLogic.UpdatePriceParameters;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.fee.BaseFee;
import google.registry.model.domain.fee.BaseFee.FeeType;
import google.registry.model.domain.fee.Fee;
@ -113,20 +113,20 @@ public final class DomainPricingLogic {
String domainName,
DateTime dateTime,
int years,
@Nullable Recurring recurringBillingEvent,
@Nullable BillingRecurrence billingRecurrence,
Optional<AllocationToken> allocationToken)
throws AllocationTokenInvalidForPremiumNameException {
checkArgument(years > 0, "Number of years must be positive");
Money renewCost;
DomainPrices domainPrices = getPricesForDomainName(domainName, dateTime);
boolean isRenewCostPremiumPrice;
// recurring billing event is null if the domain is still available. Billing events are created
// recurrence is null if the domain is still available. Billing events are created
// in the process of domain creation.
if (recurringBillingEvent == null) {
if (billingRecurrence == null) {
renewCost = getDomainRenewCostWithDiscount(domainPrices, years, allocationToken);
isRenewCostPremiumPrice = domainPrices.isPremium();
} else {
switch (recurringBillingEvent.getRenewalPriceBehavior()) {
switch (billingRecurrence.getRenewalPriceBehavior()) {
case DEFAULT:
renewCost = getDomainRenewCostWithDiscount(domainPrices, years, allocationToken);
isRenewCostPremiumPrice = domainPrices.isPremium();
@ -135,11 +135,11 @@ public final class DomainPricingLogic {
// as the creation price, which is stored in the billing event as the renewal price
case SPECIFIED:
checkArgumentPresent(
recurringBillingEvent.getRenewalPrice(),
billingRecurrence.getRenewalPrice(),
"Unexpected behavior: renewal price cannot be null when renewal behavior is"
+ " SPECIFIED");
// Don't apply allocation token to renewal price when SPECIFIED
renewCost = recurringBillingEvent.getRenewalPrice().get().multipliedBy(years);
renewCost = billingRecurrence.getRenewalPrice().get().multipliedBy(years);
isRenewCostPremiumPrice = false;
break;
// if the renewal price behavior is nonpremium, it means that the domain should be renewed
@ -157,7 +157,7 @@ public final class DomainPricingLogic {
throw new IllegalArgumentException(
String.format(
"Unknown RenewalPriceBehavior enum value: %s",
recurringBillingEvent.getRenewalPriceBehavior()));
billingRecurrence.getRenewalPriceBehavior()));
}
}
return customLogic.customizeRenewPrice(
@ -200,10 +200,10 @@ public final class DomainPricingLogic {
/** Returns a new transfer price for the pricer. */
FeesAndCredits getTransferPrice(
Tld tld, String domainName, DateTime dateTime, @Nullable Recurring recurringBillingEvent)
Tld tld, String domainName, DateTime dateTime, @Nullable BillingRecurrence billingRecurrence)
throws EppException {
FeesAndCredits renewPrice =
getRenewPrice(tld, domainName, dateTime, 1, recurringBillingEvent, Optional.empty());
getRenewPrice(tld, domainName, dateTime, 1, billingRecurrence, Optional.empty());
return customLogic.customizeTransferPrice(
TransferPriceParameters.newBuilder()
.setFeesAndCredits(

View file

@ -54,10 +54,9 @@ import google.registry.flows.custom.DomainRenewFlowCustomLogic.BeforeSaveParamet
import google.registry.flows.custom.EntityChanges;
import google.registry.flows.domain.token.AllocationTokenFlowUtils;
import google.registry.model.ImmutableObject;
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.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainCommand.Renew;
import google.registry.model.domain.DomainHistory;
@ -203,7 +202,7 @@ public final class DomainRenewFlow implements TransactionalFlow {
validateRegistrationPeriod(now, newExpirationTime);
Optional<FeeRenewCommandExtension> feeRenew =
eppInput.getSingleExtension(FeeRenewCommandExtension.class);
Recurring existingRecurringBillingEvent =
BillingRecurrence existingBillingRecurrence =
tm().loadByKey(existingDomain.getAutorenewBillingEvent());
FeesAndCredits feesAndCredits =
pricingLogic.getRenewPrice(
@ -211,7 +210,7 @@ public final class DomainRenewFlow implements TransactionalFlow {
targetId,
now,
years,
existingRecurringBillingEvent,
existingBillingRecurrence,
allocationToken);
validateFeeChallenge(feeRenew, feesAndCredits, defaultTokenUsed);
flowCustomLogic.afterValidation(
@ -223,15 +222,15 @@ public final class DomainRenewFlow implements TransactionalFlow {
HistoryEntryId domainHistoryId = createHistoryEntryId(existingDomain);
historyBuilder.setRevisionId(domainHistoryId.getRevisionId());
// Bill for this explicit renew itself.
BillingEvent.OneTime explicitRenewEvent =
BillingEvent explicitRenewEvent =
createRenewBillingEvent(
tldStr, feesAndCredits.getTotalCost(), years, domainHistoryId, allocationToken, now);
// Create a new autorenew billing event and poll message starting at the new expiration time.
BillingEvent.Recurring newAutorenewEvent =
BillingRecurrence newAutorenewEvent =
newAutorenewBillingEvent(existingDomain)
.setEventTime(newExpirationTime)
.setRenewalPrice(existingRecurringBillingEvent.getRenewalPrice().orElse(null))
.setRenewalPriceBehavior(existingRecurringBillingEvent.getRenewalPriceBehavior())
.setRenewalPrice(existingBillingRecurrence.getRenewalPrice().orElse(null))
.setRenewalPriceBehavior(existingBillingRecurrence.getRenewalPriceBehavior())
.setDomainHistoryId(domainHistoryId)
.build();
PollMessage.Autorenew newAutorenewPollMessage =
@ -240,8 +239,8 @@ public final class DomainRenewFlow implements TransactionalFlow {
.setDomainHistoryId(domainHistoryId)
.build();
// End the old autorenew billing event and poll message now. This may delete the poll message.
Recurring existingRecurring = tm().loadByKey(existingDomain.getAutorenewBillingEvent());
updateAutorenewRecurrenceEndTime(existingDomain, existingRecurring, now, domainHistoryId);
updateAutorenewRecurrenceEndTime(
existingDomain, existingBillingRecurrence, now, domainHistoryId);
Domain newDomain =
existingDomain
.asBuilder()
@ -338,14 +337,14 @@ public final class DomainRenewFlow implements TransactionalFlow {
}
}
private OneTime createRenewBillingEvent(
private BillingEvent createRenewBillingEvent(
String tld,
Money renewCost,
int years,
HistoryEntryId domainHistoryId,
Optional<AllocationToken> allocationToken,
DateTime now) {
return new BillingEvent.OneTime.Builder()
return new BillingEvent.Builder()
.setReason(Reason.RENEW)
.setTargetId(targetId)
.setRegistrarId(registrarId)

View file

@ -45,9 +45,9 @@ import google.registry.flows.FlowModule.TargetId;
import google.registry.flows.TransactionalFlow;
import google.registry.flows.annotations.ReportingSpec;
import google.registry.model.ImmutableObject;
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.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainCommand.Update;
import google.registry.model.domain.DomainHistory;
@ -161,7 +161,7 @@ public final class DomainRestoreRequestFlow implements TransactionalFlow {
entitiesToSave.add(
createRestoreBillingEvent(domainHistoryId, feesAndCredits.getRestoreCost(), now));
BillingEvent.Recurring autorenewEvent =
BillingRecurrence autorenewEvent =
newAutorenewBillingEvent(existingDomain)
.setEventTime(newExpirationTime)
.setRecurrenceEndTime(END_OF_TIME)
@ -231,7 +231,7 @@ public final class DomainRestoreRequestFlow implements TransactionalFlow {
private static Domain performRestore(
Domain existingDomain,
DateTime newExpirationTime,
BillingEvent.Recurring autorenewEvent,
BillingRecurrence autorenewEvent,
PollMessage.Autorenew autorenewPollMessage,
DateTime now,
String registrarId) {
@ -252,19 +252,19 @@ public final class DomainRestoreRequestFlow implements TransactionalFlow {
.build();
}
private OneTime createRenewBillingEvent(
private BillingEvent createRenewBillingEvent(
HistoryEntryId domainHistoryId, Money renewCost, DateTime now) {
return prepareBillingEvent(domainHistoryId, renewCost, now).setReason(Reason.RENEW).build();
}
private BillingEvent.OneTime createRestoreBillingEvent(
private BillingEvent createRestoreBillingEvent(
HistoryEntryId domainHistoryId, Money restoreCost, DateTime now) {
return prepareBillingEvent(domainHistoryId, restoreCost, now).setReason(Reason.RESTORE).build();
}
private OneTime.Builder prepareBillingEvent(
private BillingEvent.Builder prepareBillingEvent(
HistoryEntryId domainHistoryId, Money cost, DateTime now) {
return new BillingEvent.OneTime.Builder()
return new BillingEvent.Builder()
.setTargetId(targetId)
.setRegistrarId(registrarId)
.setEventTime(now)

View file

@ -45,11 +45,12 @@ import google.registry.flows.TransactionalFlow;
import google.registry.flows.annotations.ReportingSpec;
import google.registry.flows.domain.token.AllocationTokenFlowUtils;
import google.registry.model.ImmutableObject;
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.domain.Domain;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.GracePeriod;
@ -149,16 +150,18 @@ public final class DomainTransferApproveFlow implements TransactionalFlow {
String gainingRegistrarId = transferData.getGainingRegistrarId();
// Create a transfer billing event for 1 year, unless the superuser extension was used to set
// the transfer period to zero. There is not a transfer cost if the transfer period is zero.
Recurring existingRecurring = tm().loadByKey(existingDomain.getAutorenewBillingEvent());
BillingRecurrence existingBillingRecurrence =
tm().loadByKey(existingDomain.getAutorenewBillingEvent());
HistoryEntryId domainHistoryId = createHistoryEntryId(existingDomain);
historyBuilder.setRevisionId(domainHistoryId.getRevisionId());
boolean hasPackageToken = existingDomain.getCurrentPackageToken().isPresent();
Money renewalPrice = hasPackageToken ? null : existingRecurring.getRenewalPrice().orElse(null);
Optional<BillingEvent.OneTime> billingEvent =
Money renewalPrice =
hasPackageToken ? null : existingBillingRecurrence.getRenewalPrice().orElse(null);
Optional<BillingEvent> billingEvent =
transferData.getTransferPeriod().getValue() == 0
? Optional.empty()
: Optional.of(
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setReason(Reason.TRANSFER)
.setTargetId(targetId)
.setRegistrarId(gainingRegistrarId)
@ -170,9 +173,9 @@ public final class DomainTransferApproveFlow implements TransactionalFlow {
targetId,
transferData.getTransferRequestTime(),
// When removing a domain from a package it should return to the
// default recurring billing behavior so the existing recurring
// default recurrence billing behavior so the existing recurrence
// billing event should not be passed in.
hasPackageToken ? null : existingRecurring)
hasPackageToken ? null : existingBillingRecurrence)
.getRenewCost())
.setEventTime(now)
.setBillingTime(now.plus(Tld.get(tldStr).getTransferGracePeriodLength()))
@ -192,18 +195,18 @@ public final class DomainTransferApproveFlow implements TransactionalFlow {
// still needs to be charged for the auto-renew.
if (billingEvent.isPresent()) {
entitiesToSave.add(
BillingEvent.Cancellation.forGracePeriod(
autorenewGrace, now, domainHistoryId, targetId));
BillingCancellation.forGracePeriod(autorenewGrace, now, domainHistoryId, targetId));
}
}
// Close the old autorenew event and poll message at the transfer time (aka now). This may end
// up deleting the poll message.
updateAutorenewRecurrenceEndTime(existingDomain, existingRecurring, now, domainHistoryId);
updateAutorenewRecurrenceEndTime(
existingDomain, existingBillingRecurrence, now, domainHistoryId);
DateTime newExpirationTime =
computeExDateForApprovalTime(existingDomain, now, transferData.getTransferPeriod());
// Create a new autorenew event starting at the expiration time.
BillingEvent.Recurring autorenewEvent =
new BillingEvent.Recurring.Builder()
BillingRecurrence autorenewEvent =
new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId(targetId)
@ -212,7 +215,7 @@ public final class DomainTransferApproveFlow implements TransactionalFlow {
.setRenewalPriceBehavior(
hasPackageToken
? RenewalPriceBehavior.DEFAULT
: existingRecurring.getRenewalPriceBehavior())
: existingBillingRecurrence.getRenewalPriceBehavior())
.setRenewalPrice(renewalPrice)
.setRecurrenceEndTime(END_OF_TIME)
.setDomainHistoryId(domainHistoryId)
@ -248,12 +251,10 @@ public final class DomainTransferApproveFlow implements TransactionalFlow {
.setGracePeriods(
billingEvent
.map(
oneTime ->
event ->
ImmutableSet.of(
GracePeriod.forBillingEvent(
GracePeriodStatus.TRANSFER,
existingDomain.getRepoId(),
oneTime)))
GracePeriodStatus.TRANSFER, existingDomain.getRepoId(), event)))
.orElseGet(ImmutableSet::of))
.setLastEppUpdateTime(now)
.setLastEppUpdateRegistrarId(registrarId)

View file

@ -39,7 +39,7 @@ import google.registry.flows.FlowModule.Superuser;
import google.registry.flows.FlowModule.TargetId;
import google.registry.flows.TransactionalFlow;
import google.registry.flows.annotations.ReportingSpec;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.metadata.MetadataExtension;
@ -117,9 +117,10 @@ public final class DomainTransferCancelFlow implements TransactionalFlow {
targetId, newDomain.getTransferData(), null, domainHistoryId));
// Reopen the autorenew event and poll message that we closed for the implicit transfer. This
// may recreate the autorenew poll message if it was deleted when the transfer request was made.
Recurring existingRecurring = tm().loadByKey(existingDomain.getAutorenewBillingEvent());
BillingRecurrence existingBillingRecurrence =
tm().loadByKey(existingDomain.getAutorenewBillingEvent());
updateAutorenewRecurrenceEndTime(
existingDomain, existingRecurring, END_OF_TIME, domainHistory.getHistoryEntryId());
existingDomain, existingBillingRecurrence, END_OF_TIME, domainHistory.getHistoryEntryId());
// Delete the billing event and poll messages that were written in case the transfer would have
// been implicitly server approved.
tm().delete(existingDomain.getTransferData().getServerApproveEntities());

View file

@ -41,7 +41,7 @@ import google.registry.flows.FlowModule.Superuser;
import google.registry.flows.FlowModule.TargetId;
import google.registry.flows.TransactionalFlow;
import google.registry.flows.annotations.ReportingSpec;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.metadata.MetadataExtension;
@ -116,9 +116,10 @@ public final class DomainTransferRejectFlow implements TransactionalFlow {
targetId, newDomain.getTransferData(), null, now, domainHistoryId));
// Reopen the autorenew event and poll message that we closed for the implicit transfer. This
// may end up recreating the poll message if it was deleted upon the transfer request.
Recurring existingRecurring = tm().loadByKey(existingDomain.getAutorenewBillingEvent());
BillingRecurrence existingBillingRecurrence =
tm().loadByKey(existingDomain.getAutorenewBillingEvent());
updateAutorenewRecurrenceEndTime(
existingDomain, existingRecurring, END_OF_TIME, domainHistory.getHistoryEntryId());
existingDomain, existingBillingRecurrence, END_OF_TIME, domainHistory.getHistoryEntryId());
// Delete the billing event and poll messages that were written in case the transfer would have
// been implicitly server approved.
tm().delete(existingDomain.getTransferData().getServerApproveEntities());

View file

@ -52,7 +52,7 @@ import google.registry.flows.exceptions.InvalidTransferPeriodValueException;
import google.registry.flows.exceptions.ObjectAlreadySponsoredException;
import google.registry.flows.exceptions.TransferPeriodMustBeOneYearException;
import google.registry.flows.exceptions.TransferPeriodZeroAndFeeTransferExtensionException;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainCommand.Transfer;
import google.registry.model.domain.DomainHistory;
@ -195,13 +195,14 @@ public final class DomainTransferRequestFlow implements TransactionalFlow {
throw new TransferPeriodZeroAndFeeTransferExtensionException();
}
// If the period is zero, then there is no fee for the transfer.
Recurring existingRecurring = tm().loadByKey(existingDomain.getAutorenewBillingEvent());
BillingRecurrence existingBillingRecurrence =
tm().loadByKey(existingDomain.getAutorenewBillingEvent());
Optional<FeesAndCredits> feesAndCredits;
if (period.getValue() == 0) {
feesAndCredits = Optional.empty();
} else if (!existingDomain.getCurrentPackageToken().isPresent()) {
feesAndCredits =
Optional.of(pricingLogic.getTransferPrice(tld, targetId, now, existingRecurring));
Optional.of(pricingLogic.getTransferPrice(tld, targetId, now, existingBillingRecurrence));
} else {
// If existing domain is in a package, calculate the transfer price with default renewal price
// behavior
@ -243,7 +244,7 @@ public final class DomainTransferRequestFlow implements TransactionalFlow {
serverApproveNewExpirationTime,
domainHistoryId,
existingDomain,
existingRecurring,
existingBillingRecurrence,
trid,
gainingClientId,
feesAndCredits.map(FeesAndCredits::getTotalCost),
@ -274,7 +275,7 @@ public final class DomainTransferRequestFlow implements TransactionalFlow {
// cloneProjectedAtTime() will replace these old autorenew entities with the server approve ones
// that we've created in this flow and stored in pendingTransferData.
updateAutorenewRecurrenceEndTime(
existingDomain, existingRecurring, automaticTransferTime, domainHistoryId);
existingDomain, existingBillingRecurrence, automaticTransferTime, domainHistoryId);
Domain newDomain =
existingDomain
.asBuilder()

View file

@ -20,11 +20,12 @@ import static google.registry.util.DateTimeUtils.END_OF_TIME;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
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.domain.Domain;
import google.registry.model.domain.GracePeriod;
import google.registry.model.domain.Period;
@ -66,8 +67,8 @@ public final class DomainTransferUtils {
// Unless superuser sets period to 0, add a transfer billing event.
transferDataBuilder.setServerApproveBillingEvent(
serverApproveEntities.stream()
.filter(BillingEvent.OneTime.class::isInstance)
.map(BillingEvent.OneTime.class::cast)
.filter(BillingEvent.class::isInstance)
.map(BillingEvent.class::cast)
.collect(onlyElement())
.createVKey());
}
@ -75,8 +76,8 @@ public final class DomainTransferUtils {
.setTransferStatus(TransferStatus.PENDING)
.setServerApproveAutorenewEvent(
serverApproveEntities.stream()
.filter(BillingEvent.Recurring.class::isInstance)
.map(BillingEvent.Recurring.class::cast)
.filter(BillingRecurrence.class::isInstance)
.map(BillingRecurrence.class::cast)
.collect(onlyElement())
.createVKey())
.setServerApproveAutorenewPollMessage(
@ -110,7 +111,7 @@ public final class DomainTransferUtils {
DateTime serverApproveNewExpirationTime,
HistoryEntryId domainHistoryId,
Domain existingDomain,
Recurring existingRecurring,
BillingRecurrence existingBillingRecurrence,
Trid trid,
String gainingRegistrarId,
Optional<Money> transferCost,
@ -146,12 +147,12 @@ public final class DomainTransferUtils {
.add(
createGainingClientAutorenewEvent(
existingDomain.getCurrentPackageToken().isPresent()
? existingRecurring
? existingBillingRecurrence
.asBuilder()
.setRenewalPriceBehavior(RenewalPriceBehavior.DEFAULT)
.setRenewalPrice(null)
.build()
: existingRecurring,
: existingBillingRecurrence,
serverApproveNewExpirationTime,
domainHistoryId,
targetId,
@ -246,21 +247,21 @@ public final class DomainTransferUtils {
.build();
}
private static BillingEvent.Recurring createGainingClientAutorenewEvent(
Recurring existingRecurring,
private static BillingRecurrence createGainingClientAutorenewEvent(
BillingRecurrence existingBillingRecurrence,
DateTime serverApproveNewExpirationTime,
HistoryEntryId domainHistoryId,
String targetId,
String gainingRegistrarId) {
return new BillingEvent.Recurring.Builder()
return new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId(targetId)
.setRegistrarId(gainingRegistrarId)
.setEventTime(serverApproveNewExpirationTime)
.setRecurrenceEndTime(END_OF_TIME)
.setRenewalPriceBehavior(existingRecurring.getRenewalPriceBehavior())
.setRenewalPrice(existingRecurring.getRenewalPrice().orElse(null))
.setRenewalPriceBehavior(existingBillingRecurrence.getRenewalPriceBehavior())
.setRenewalPrice(existingBillingRecurrence.getRenewalPrice().orElse(null))
.setDomainHistoryId(domainHistoryId)
.build();
}
@ -281,7 +282,7 @@ public final class DomainTransferUtils {
* <p>For details on the policy justification, see b/19430703#comment17 and <a
* href="https://www.icann.org/news/advisory-2002-06-06-en">this ICANN advisory</a>.
*/
private static Optional<BillingEvent.Cancellation> createOptionalAutorenewCancellation(
private static Optional<BillingCancellation> createOptionalAutorenewCancellation(
DateTime automaticTransferTime,
DateTime now,
HistoryEntryId domainHistoryId,
@ -294,8 +295,7 @@ public final class DomainTransferUtils {
domainAtTransferTime.getGracePeriodsOfType(GracePeriodStatus.AUTO_RENEW), null);
if (autorenewGracePeriod != null && transferCost.isPresent()) {
return Optional.of(
BillingEvent.Cancellation.forGracePeriod(
autorenewGracePeriod, now, domainHistoryId, targetId)
BillingCancellation.forGracePeriod(autorenewGracePeriod, now, domainHistoryId, targetId)
.asBuilder()
.setEventTime(automaticTransferTime)
.build());
@ -303,14 +303,14 @@ public final class DomainTransferUtils {
return Optional.empty();
}
private static BillingEvent.OneTime createTransferBillingEvent(
private static BillingEvent createTransferBillingEvent(
DateTime automaticTransferTime,
HistoryEntryId domainHistoryId,
String targetId,
String gainingRegistrarId,
Tld registry,
Money transferCost) {
return new BillingEvent.OneTime.Builder()
return new BillingEvent.Builder()
.setReason(Reason.TRANSFER)
.setTargetId(targetId)
.setRegistrarId(gainingRegistrarId)

View file

@ -64,8 +64,8 @@ import google.registry.flows.custom.EntityChanges;
import google.registry.flows.domain.DomainFlowUtils.MissingRegistrantException;
import google.registry.flows.domain.DomainFlowUtils.NameserversNotSpecifiedForTldWithNameserverAllowListException;
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.DesignatedContact;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainCommand.Update;
@ -187,7 +187,7 @@ public final class DomainUpdateFlow implements TransactionalFlow {
}
ImmutableSet.Builder<ImmutableObject> entitiesToSave = new ImmutableSet.Builder<>();
entitiesToSave.add(newDomain, domainHistory);
Optional<BillingEvent.OneTime> statusUpdateBillingEvent =
Optional<BillingEvent> statusUpdateBillingEvent =
createBillingEventForStatusUpdates(existingDomain, newDomain, domainHistory, now);
statusUpdateBillingEvent.ifPresent(entitiesToSave::add);
Optional<PollMessage.OneTime> serverStatusUpdatePollMessage =
@ -324,7 +324,7 @@ public final class DomainUpdateFlow implements TransactionalFlow {
}
/** Some status updates cost money. Bill only once no matter how many of them are changed. */
private Optional<BillingEvent.OneTime> createBillingEventForStatusUpdates(
private Optional<BillingEvent> createBillingEventForStatusUpdates(
Domain existingDomain, Domain newDomain, DomainHistory historyEntry, DateTime now) {
Optional<MetadataExtension> metadataExtension =
eppInput.getSingleExtension(MetadataExtension.class);
@ -334,7 +334,7 @@ public final class DomainUpdateFlow implements TransactionalFlow {
if (statusValue.isChargedStatus()) {
// Only charge once.
return Optional.of(
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setReason(Reason.SERVER_STATUS)
.setTargetId(targetId)
.setRegistrarId(registrarId)

View file

@ -26,8 +26,8 @@ import google.registry.flows.EppException;
import google.registry.flows.EppException.AssociationProhibitsOperationException;
import google.registry.flows.EppException.AuthorizationErrorException;
import google.registry.flows.EppException.StatusProhibitsOperationException;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
import google.registry.model.billing.BillingBase.RenewalPriceBehavior;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainCommand;
import google.registry.model.domain.fee.FeeQueryCommandExtensionItem.CommandName;
@ -236,16 +236,16 @@ public class AllocationTokenFlowUtils {
return domain;
}
Recurring newRecurringBillingEvent =
BillingRecurrence newBillingRecurrence =
tm().loadByKey(domain.getAutorenewBillingEvent())
.asBuilder()
.setRenewalPriceBehavior(RenewalPriceBehavior.DEFAULT)
.setRenewalPrice(null)
.build();
// the Recurring billing event is reloaded later in the renew flow, so we synchronize changed
// RecurringBillingEvent with storage manually
tm().put(newRecurringBillingEvent);
// the Recurrence is reloaded later in the renew flow, so we synchronize changed
// Recurrences with storage manually
tm().put(newBillingRecurrence);
tm().getEntityManager().flush();
tm().getEntityManager().clear();
@ -253,7 +253,7 @@ public class AllocationTokenFlowUtils {
return domain
.asBuilder()
.setCurrentPackageToken(null)
.setAutorenewBillingEvent(newRecurringBillingEvent.createVKey())
.setAutorenewBillingEvent(newBillingRecurrence.createVKey())
.build();
}

View file

@ -0,0 +1,264 @@
// Copyright 2017 The Nomulus Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package google.registry.model.billing;
import static com.google.common.base.Preconditions.checkNotNull;
import static google.registry.util.CollectionUtils.forceEmptyToNull;
import static google.registry.util.CollectionUtils.nullToEmptyImmutableCopy;
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
import com.google.common.collect.ImmutableSet;
import google.registry.model.Buildable;
import google.registry.model.ImmutableObject;
import google.registry.model.UnsafeSerializable;
import google.registry.model.annotations.IdAllocation;
import google.registry.model.domain.DomainHistory;
import google.registry.model.reporting.HistoryEntry.HistoryEntryId;
import google.registry.model.transfer.TransferData.TransferServerApproveEntity;
import google.registry.persistence.VKey;
import java.util.Set;
import javax.annotation.Nullable;
import javax.persistence.Column;
import javax.persistence.EnumType;
import javax.persistence.Enumerated;
import javax.persistence.Id;
import javax.persistence.MappedSuperclass;
import org.joda.time.DateTime;
/** A billable event in a domain's lifecycle. */
@MappedSuperclass
public abstract class BillingBase extends ImmutableObject
implements Buildable, TransferServerApproveEntity, UnsafeSerializable {
/** The reason for the bill, which maps 1:1 to skus in go/registry-billing-skus. */
public enum Reason {
CREATE(true),
@Deprecated // DO NOT USE THIS REASON. IT REMAINS BECAUSE OF HISTORICAL DATA. SEE b/31676071.
ERROR(false),
FEE_EARLY_ACCESS(true),
RENEW(true),
RESTORE(true),
SERVER_STATUS(false),
TRANSFER(true);
private final boolean requiresPeriod;
Reason(boolean requiresPeriod) {
this.requiresPeriod = requiresPeriod;
}
/**
* Returns whether billing events with this reason have a period years associated with them.
*
* <p>Note that this is an "if an only if" condition.
*/
public boolean hasPeriodYears() {
return requiresPeriod;
}
}
/** Set of flags that can be applied to billing events. */
public enum Flag {
ALLOCATION,
ANCHOR_TENANT,
AUTO_RENEW,
/** Landrush billing events are historical only and are no longer created. */
LANDRUSH,
/**
* This flag is used on create {@link BillingEvent} billing events for domains that were
* reserved.
*
* <p>This can happen when allocation tokens are used or superusers override a domain
* reservation. These cases can need special handling in billing/invoicing. Anchor tenants will
* never have this flag applied; they will have ANCHOR_TENANT instead.
*/
RESERVED,
SUNRISE,
/**
* This flag will be added to any {@link BillingEvent} events that are created via, e.g., an
* automated process to expand {@link BillingRecurrence} events.
*/
SYNTHETIC
}
/**
* Sets of renewal price behaviors that can be applied to billing recurrences.
*
* <p>When a client renews a domain, they could be charged differently, depending on factors such
* as the client type and the domain itself.
*/
public enum RenewalPriceBehavior {
/**
* This indicates the renewal price is the default price.
*
* <p>By default, if the domain is premium, then premium price will be used. Otherwise, the
* standard price of the TLD will be used.
*/
DEFAULT,
/**
* This indicates the domain will be renewed at standard price even if it's a premium domain.
*
* <p>We chose to name this "NONPREMIUM" rather than simply "STANDARD" to avoid confusion
* between "STANDARD" and "DEFAULT".
*
* <p>This price behavior is used with anchor tenants.
*/
NONPREMIUM,
/**
* This indicates that the renewalPrice in {@link BillingRecurrence} will be used for domain
* renewal.
*
* <p>The renewalPrice has a non-null value iff the price behavior is set to "SPECIFIED". This
* behavior is used with internal registrations.
*/
SPECIFIED
}
/** Entity id. */
@IdAllocation @Id Long id;
/** The registrar to bill. */
@Column(name = "registrarId", nullable = false)
String clientId;
/** Revision id of the entry in DomainHistory table that ths bill belongs to. */
@Column(nullable = false)
Long domainHistoryRevisionId;
/** ID of the EPP resource that the bill is for. */
@Column(nullable = false)
String domainRepoId;
/** When this event was created. For recurrence events, this is also the recurrence start time. */
@Column(nullable = false)
DateTime eventTime;
/** The reason for the bill. */
@Enumerated(EnumType.STRING)
@Column(nullable = false)
Reason reason;
/** The fully qualified domain name of the domain that the bill is for. */
@Column(name = "domain_name", nullable = false)
String targetId;
@Nullable Set<Flag> flags;
public String getRegistrarId() {
return clientId;
}
public long getDomainHistoryRevisionId() {
return domainHistoryRevisionId;
}
public String getDomainRepoId() {
return domainRepoId;
}
public DateTime getEventTime() {
return eventTime;
}
public long getId() {
return id;
}
public Reason getReason() {
return reason;
}
public String getTargetId() {
return targetId;
}
public HistoryEntryId getHistoryEntryId() {
return new HistoryEntryId(domainRepoId, domainHistoryRevisionId);
}
public ImmutableSet<Flag> getFlags() {
return nullToEmptyImmutableCopy(flags);
}
@Override
public abstract VKey<? extends BillingBase> createVKey();
/** Override Buildable.asBuilder() to give this method stronger typing. */
@Override
public abstract Builder<?, ?> asBuilder();
/** An abstract builder for {@link BillingBase}. */
public abstract static class Builder<T extends BillingBase, B extends Builder<?, ?>>
extends GenericBuilder<T, B> {
protected Builder() {}
protected Builder(T instance) {
super(instance);
}
public B setReason(Reason reason) {
getInstance().reason = reason;
return thisCastToDerived();
}
public B setId(long id) {
getInstance().id = id;
return thisCastToDerived();
}
public B setRegistrarId(String registrarId) {
getInstance().clientId = registrarId;
return thisCastToDerived();
}
public B setEventTime(DateTime eventTime) {
getInstance().eventTime = eventTime;
return thisCastToDerived();
}
public B setTargetId(String targetId) {
getInstance().targetId = targetId;
return thisCastToDerived();
}
public B setFlags(ImmutableSet<Flag> flags) {
getInstance().flags = forceEmptyToNull(checkArgumentNotNull(flags, "flags"));
return thisCastToDerived();
}
public B setDomainHistoryId(HistoryEntryId domainHistoryId) {
getInstance().domainHistoryRevisionId = domainHistoryId.getRevisionId();
getInstance().domainRepoId = domainHistoryId.getRepoId();
return thisCastToDerived();
}
public B setDomainHistory(DomainHistory domainHistory) {
return setDomainHistoryId(domainHistory.getHistoryEntryId());
}
@Override
public T build() {
T instance = getInstance();
checkNotNull(instance.reason, "Reason must be set");
checkNotNull(instance.clientId, "Registrar ID must be set");
checkNotNull(instance.eventTime, "Event time must be set");
checkNotNull(instance.targetId, "Target ID must be set");
checkNotNull(instance.domainHistoryRevisionId, "Domain History Revision ID must be set");
checkNotNull(instance.domainRepoId, "Domain Repo ID must be set");
return super.build();
}
}
}

View file

@ -0,0 +1,166 @@
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package google.registry.model.billing;
import static com.google.common.base.MoreObjects.firstNonNull;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import com.google.common.collect.ImmutableMap;
import google.registry.model.domain.GracePeriod;
import google.registry.model.domain.rgp.GracePeriodStatus;
import google.registry.model.reporting.HistoryEntry.HistoryEntryId;
import google.registry.persistence.VKey;
import google.registry.persistence.WithVKey;
import javax.persistence.AttributeOverride;
import javax.persistence.Column;
import javax.persistence.Entity;
import javax.persistence.Index;
import javax.persistence.Table;
import org.joda.time.DateTime;
/**
* An event representing a cancellation of one of the other two billable event types.
*
* <p>This is implemented as a separate event rather than a bit on BillingEvent in order to preserve
* the immutability of billing events.
*/
@Entity
@Table(
indexes = {
@Index(columnList = "registrarId"),
@Index(columnList = "eventTime"),
@Index(columnList = "domainRepoId"),
@Index(columnList = "billingTime"),
@Index(columnList = "billing_event_id"),
@Index(columnList = "billing_recurrence_id")
})
@AttributeOverride(name = "id", column = @Column(name = "billing_cancellation_id"))
@WithVKey(Long.class)
public class BillingCancellation extends BillingBase {
/** The billing time of the charge that is being cancelled. */
DateTime billingTime;
/** The one-time billing event to cancel, or null for autorenew cancellations. */
@Column(name = "billing_event_id")
VKey<BillingEvent> billingEvent;
/** The Recurrence to cancel, or null for non-autorenew cancellations. */
@Column(name = "billing_recurrence_id")
VKey<BillingRecurrence> billingRecurrence;
public DateTime getBillingTime() {
return billingTime;
}
public VKey<? extends BillingBase> getEventKey() {
return firstNonNull(billingEvent, billingRecurrence);
}
/** The mapping from billable grace period types to originating billing event reasons. */
static final ImmutableMap<GracePeriodStatus, Reason> GRACE_PERIOD_TO_REASON =
ImmutableMap.of(
GracePeriodStatus.ADD, Reason.CREATE,
GracePeriodStatus.AUTO_RENEW, Reason.RENEW,
GracePeriodStatus.RENEW, Reason.RENEW,
GracePeriodStatus.TRANSFER, Reason.TRANSFER);
/**
* Creates a cancellation billing event (parented on the provided history key, and with the
* corresponding event time) that will cancel out the provided grace period's billing event, using
* the supplied targetId and deriving other metadata (clientId, billing time, and the cancellation
* reason) from the grace period.
*/
public static google.registry.model.billing.BillingCancellation forGracePeriod(
GracePeriod gracePeriod,
DateTime eventTime,
HistoryEntryId domainHistoryId,
String targetId) {
checkArgument(
gracePeriod.hasBillingEvent(),
"Cannot create cancellation for grace period without billing event");
Builder builder =
new Builder()
.setReason(checkNotNull(GRACE_PERIOD_TO_REASON.get(gracePeriod.getType())))
.setTargetId(targetId)
.setRegistrarId(gracePeriod.getRegistrarId())
.setEventTime(eventTime)
// The charge being cancelled will take place at the grace period's expiration time.
.setBillingTime(gracePeriod.getExpirationTime())
.setDomainHistoryId(domainHistoryId);
// Set the grace period's billing event using the appropriate Cancellation builder method.
if (gracePeriod.getBillingEvent() != null) {
builder.setBillingEvent(gracePeriod.getBillingEvent());
} else if (gracePeriod.getBillingRecurrence() != null) {
builder.setBillingRecurrence(gracePeriod.getBillingRecurrence());
}
return builder.build();
}
@Override
public VKey<google.registry.model.billing.BillingCancellation> createVKey() {
return createVKey(getId());
}
public static VKey<google.registry.model.billing.BillingCancellation> createVKey(long id) {
return VKey.create(google.registry.model.billing.BillingCancellation.class, id);
}
@Override
public Builder asBuilder() {
return new Builder(clone(this));
}
/**
* A builder for {@link google.registry.model.billing.BillingCancellation} since it is immutable.
*/
public static class Builder
extends BillingBase.Builder<google.registry.model.billing.BillingCancellation, Builder> {
public Builder() {}
private Builder(google.registry.model.billing.BillingCancellation instance) {
super(instance);
}
public Builder setBillingTime(DateTime billingTime) {
getInstance().billingTime = billingTime;
return this;
}
public Builder setBillingEvent(VKey<BillingEvent> billingEvent) {
getInstance().billingEvent = billingEvent;
return this;
}
public Builder setBillingRecurrence(VKey<BillingRecurrence> billingRecurrence) {
getInstance().billingRecurrence = billingRecurrence;
return this;
}
@Override
public google.registry.model.billing.BillingCancellation build() {
google.registry.model.billing.BillingCancellation instance = getInstance();
checkNotNull(instance.billingTime, "Must set billing time");
checkNotNull(instance.reason, "Must set reason");
checkState(
(instance.billingEvent == null) != (instance.billingRecurrence == null),
"Cancellations must have exactly one billing event key set");
return super.build();
}
}
}

View file

@ -1,4 +1,4 @@
// Copyright 2017 The Nomulus Authors. All Rights Reserved.
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
@ -14,740 +14,197 @@
package google.registry.model.billing;
import static com.google.common.base.MoreObjects.firstNonNull;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static google.registry.util.CollectionUtils.forceEmptyToNull;
import static google.registry.util.CollectionUtils.nullToEmptyImmutableCopy;
import static google.registry.util.DateTimeUtils.END_OF_TIME;
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import google.registry.model.Buildable;
import google.registry.model.ImmutableObject;
import google.registry.model.UnsafeSerializable;
import google.registry.model.annotations.IdAllocation;
import google.registry.model.common.TimeOfYear;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.GracePeriod;
import google.registry.model.domain.rgp.GracePeriodStatus;
import google.registry.model.domain.token.AllocationToken;
import google.registry.model.reporting.HistoryEntry.HistoryEntryId;
import google.registry.model.transfer.TransferData.TransferServerApproveEntity;
import google.registry.persistence.VKey;
import google.registry.persistence.WithVKey;
import google.registry.persistence.converter.JodaMoneyType;
import java.util.Optional;
import java.util.Set;
import javax.annotation.Nullable;
import javax.persistence.AttributeOverride;
import javax.persistence.AttributeOverrides;
import javax.persistence.Column;
import javax.persistence.Embedded;
import javax.persistence.Entity;
import javax.persistence.EnumType;
import javax.persistence.Enumerated;
import javax.persistence.Id;
import javax.persistence.Index;
import javax.persistence.MappedSuperclass;
import javax.persistence.Table;
import org.hibernate.annotations.Columns;
import org.hibernate.annotations.Type;
import org.joda.money.Money;
import org.joda.time.DateTime;
/** A billable event in a domain's lifecycle. */
@MappedSuperclass
public abstract class BillingEvent extends ImmutableObject
implements Buildable, TransferServerApproveEntity, UnsafeSerializable {
/** A one-time billable event. */
@Entity
@Table(
indexes = {
@Index(columnList = "registrarId"),
@Index(columnList = "eventTime"),
@Index(columnList = "billingTime"),
@Index(columnList = "syntheticCreationTime"),
@Index(columnList = "domainRepoId"),
@Index(columnList = "allocationToken"),
@Index(columnList = "cancellation_matching_billing_recurrence_id")
})
@AttributeOverride(name = "id", column = @Column(name = "billing_event_id"))
@WithVKey(Long.class)
public class BillingEvent extends BillingBase {
/** The reason for the bill, which maps 1:1 to skus in go/registry-billing-skus. */
public enum Reason {
CREATE(true),
@Deprecated // DO NOT USE THIS REASON. IT REMAINS BECAUSE OF HISTORICAL DATA. SEE b/31676071.
ERROR(false),
FEE_EARLY_ACCESS(true),
RENEW(true),
RESTORE(true),
SERVER_STATUS(false),
TRANSFER(true);
/** The billable value. */
@Type(type = JodaMoneyType.TYPE_NAME)
@Columns(columns = {@Column(name = "cost_amount"), @Column(name = "cost_currency")})
Money cost;
private final boolean requiresPeriod;
Reason(boolean requiresPeriod) {
this.requiresPeriod = requiresPeriod;
}
/**
* Returns whether billing events with this reason have a period years associated with them.
*
* <p>Note that this is an "if an only if" condition.
*/
public boolean hasPeriodYears() {
return requiresPeriod;
}
}
/** Set of flags that can be applied to billing events. */
public enum Flag {
ALLOCATION,
ANCHOR_TENANT,
AUTO_RENEW,
/** Landrush billing events are historical only and are no longer created. */
LANDRUSH,
/**
* This flag is used on create {@link OneTime} billing events for domains that were reserved.
*
* <p>This can happen when allocation tokens are used or superusers override a domain
* reservation. These cases can need special handling in billing/invoicing. Anchor tenants will
* never have this flag applied; they will have ANCHOR_TENANT instead.
*/
RESERVED,
SUNRISE,
/**
* This flag will be added to any {@link OneTime} events that are created via, e.g., an
* automated process to expand {@link Recurring} events.
*/
SYNTHETIC
}
/** When the cost should be billed. */
DateTime billingTime;
/**
* Sets of renewal price behaviors that can be applied to billing recurrences.
*
* <p>When a client renews a domain, they could be charged differently, depending on factors such
* as the client type and the domain itself.
* The period in years of the action being billed for, if applicable, otherwise null. Used for
* financial reporting.
*/
public enum RenewalPriceBehavior {
/**
* This indicates the renewal price is the default price.
*
* <p>By default, if the domain is premium, then premium price will be used. Otherwise, the
* standard price of the TLD will be used.
*/
DEFAULT,
/**
* This indicates the domain will be renewed at standard price even if it's a premium domain.
*
* <p>We chose to name this "NONPREMIUM" rather than simply "STANDARD" to avoid confusion
* between "STANDARD" and "DEFAULT".
*
* <p>This price behavior is used with anchor tenants.
*/
NONPREMIUM,
/**
* This indicates that the renewalPrice in {@link Recurring} will be used for domain renewal.
*
* <p>The renewalPrice has a non-null value iff the price behavior is set to "SPECIFIED". This
* behavior is used with internal registrations.
*/
SPECIFIED
Integer periodYears;
/**
* For {@link Flag#SYNTHETIC} events, when this event was persisted to the database (i.e. the
* cursor position at the time the recurrence expansion job was last run). In the event a job
* needs to be undone, a query on this field will return the complete set of potentially bad
* events.
*/
DateTime syntheticCreationTime;
/**
* For {@link Flag#SYNTHETIC} events, a {@link VKey} to the {@link BillingRecurrence} from which
* this {@link google.registry.model.billing.BillingEvent} was created. This is needed in order to
* properly match billing events against {@link BillingCancellation}s.
*/
@Column(name = "cancellation_matching_billing_recurrence_id")
VKey<BillingRecurrence> cancellationMatchingBillingEvent;
/**
* For {@link Flag#SYNTHETIC} events, the {@link DomainHistory} revision ID of the {@link
* BillingRecurrence} from which this {@link google.registry.model.billing.BillingEvent} was
* created. This is needed in order to recreate the {@link VKey} when reading from SQL.
*/
@Column(name = "recurrence_history_revision_id")
Long recurrenceHistoryRevisionId;
/**
* The {@link AllocationToken} used in the creation of this event, or null if one was not used.
*/
@Nullable VKey<AllocationToken> allocationToken;
public Money getCost() {
return cost;
}
/** Entity id. */
@IdAllocation @Id Long id;
/** The registrar to bill. */
@Column(name = "registrarId", nullable = false)
String clientId;
/** Revision id of the entry in DomainHistory table that ths bill belongs to. */
@Column(nullable = false)
Long domainHistoryRevisionId;
/** ID of the EPP resource that the bill is for. */
@Column(nullable = false)
String domainRepoId;
/** When this event was created. For recurring events, this is also the recurrence start time. */
@Column(nullable = false)
DateTime eventTime;
/** The reason for the bill. */
@Enumerated(EnumType.STRING)
@Column(nullable = false)
Reason reason;
/** The fully qualified domain name of the domain that the bill is for. */
@Column(name = "domain_name", nullable = false)
String targetId;
@Nullable Set<Flag> flags;
public String getRegistrarId() {
return clientId;
public DateTime getBillingTime() {
return billingTime;
}
public long getDomainHistoryRevisionId() {
return domainHistoryRevisionId;
public Integer getPeriodYears() {
return periodYears;
}
public String getDomainRepoId() {
return domainRepoId;
public DateTime getSyntheticCreationTime() {
return syntheticCreationTime;
}
public DateTime getEventTime() {
return eventTime;
public VKey<BillingRecurrence> getCancellationMatchingBillingEvent() {
return cancellationMatchingBillingEvent;
}
public long getId() {
return id;
public Long getRecurrenceHistoryRevisionId() {
return recurrenceHistoryRevisionId;
}
public Reason getReason() {
return reason;
}
public String getTargetId() {
return targetId;
}
public HistoryEntryId getHistoryEntryId() {
return new HistoryEntryId(domainRepoId, domainHistoryRevisionId);
}
public ImmutableSet<Flag> getFlags() {
return nullToEmptyImmutableCopy(flags);
public Optional<VKey<AllocationToken>> getAllocationToken() {
return Optional.ofNullable(allocationToken);
}
@Override
public abstract VKey<? extends BillingEvent> createVKey();
public VKey<google.registry.model.billing.BillingEvent> createVKey() {
return createVKey(getId());
}
public static VKey<google.registry.model.billing.BillingEvent> createVKey(long id) {
return VKey.create(google.registry.model.billing.BillingEvent.class, id);
}
/** Override Buildable.asBuilder() to give this method stronger typing. */
@Override
public abstract Builder<?, ?> asBuilder();
public Builder asBuilder() {
return new Builder(clone(this));
}
/** An abstract builder for {@link BillingEvent}. */
public abstract static class Builder<T extends BillingEvent, B extends Builder<?, ?>>
extends GenericBuilder<T, B> {
/** A builder for {@link google.registry.model.billing.BillingEvent} since it is immutable. */
public static class Builder
extends BillingBase.Builder<google.registry.model.billing.BillingEvent, Builder> {
protected Builder() {}
public Builder() {}
protected Builder(T instance) {
private Builder(google.registry.model.billing.BillingEvent instance) {
super(instance);
}
public B setReason(Reason reason) {
getInstance().reason = reason;
return thisCastToDerived();
public Builder setCost(Money cost) {
getInstance().cost = cost;
return this;
}
public B setId(long id) {
getInstance().id = id;
return thisCastToDerived();
public Builder setPeriodYears(Integer periodYears) {
checkNotNull(periodYears);
checkArgument(periodYears > 0);
getInstance().periodYears = periodYears;
return this;
}
public B setRegistrarId(String registrarId) {
getInstance().clientId = registrarId;
return thisCastToDerived();
public Builder setBillingTime(DateTime billingTime) {
getInstance().billingTime = billingTime;
return this;
}
public B setEventTime(DateTime eventTime) {
getInstance().eventTime = eventTime;
return thisCastToDerived();
public Builder setSyntheticCreationTime(DateTime syntheticCreationTime) {
getInstance().syntheticCreationTime = syntheticCreationTime;
return this;
}
public B setTargetId(String targetId) {
getInstance().targetId = targetId;
return thisCastToDerived();
public Builder setCancellationMatchingBillingEvent(
BillingRecurrence cancellationMatchingBillingEvent) {
getInstance().cancellationMatchingBillingEvent =
cancellationMatchingBillingEvent.createVKey();
getInstance().recurrenceHistoryRevisionId =
cancellationMatchingBillingEvent.getDomainHistoryRevisionId();
return this;
}
public B setFlags(ImmutableSet<Flag> flags) {
getInstance().flags = forceEmptyToNull(checkArgumentNotNull(flags, "flags"));
return thisCastToDerived();
}
public B setDomainHistoryId(HistoryEntryId domainHistoryId) {
getInstance().domainHistoryRevisionId = domainHistoryId.getRevisionId();
getInstance().domainRepoId = domainHistoryId.getRepoId();
return thisCastToDerived();
}
public B setDomainHistory(DomainHistory domainHistory) {
return setDomainHistoryId(domainHistory.getHistoryEntryId());
public Builder setAllocationToken(@Nullable VKey<AllocationToken> allocationToken) {
getInstance().allocationToken = allocationToken;
return this;
}
@Override
public T build() {
T instance = getInstance();
checkNotNull(instance.reason, "Reason must be set");
checkNotNull(instance.clientId, "Registrar ID must be set");
checkNotNull(instance.eventTime, "Event time must be set");
checkNotNull(instance.targetId, "Target ID must be set");
checkNotNull(instance.domainHistoryRevisionId, "Domain History Revision ID must be set");
checkNotNull(instance.domainRepoId, "Domain Repo ID must be set");
public google.registry.model.billing.BillingEvent build() {
google.registry.model.billing.BillingEvent instance = getInstance();
checkNotNull(instance.billingTime);
checkNotNull(instance.cost);
checkState(!instance.cost.isNegative(), "Costs should be non-negative.");
// TODO(mcilwain): Enforce this check on all billing events (not just more recent ones)
// post-migration after we add the missing period years values in SQL.
if (instance.eventTime.isAfter(DateTime.parse("2019-01-01T00:00:00Z"))) {
checkState(
instance.reason.hasPeriodYears() == (instance.periodYears != null),
"Period years must be set if and only if reason is "
+ "CREATE, FEE_EARLY_ACCESS, RENEW, RESTORE or TRANSFER.");
}
checkState(
instance.getFlags().contains(Flag.SYNTHETIC) == (instance.syntheticCreationTime != null),
"Synthetic creation time must be set if and only if the SYNTHETIC flag is set.");
checkState(
instance.getFlags().contains(Flag.SYNTHETIC)
== (instance.cancellationMatchingBillingEvent != null),
"Cancellation matching billing event must be set if and only if the SYNTHETIC flag "
+ "is set.");
return super.build();
}
}
/** A one-time billable event. */
@Entity(name = "BillingEvent")
@Table(
indexes = {
@Index(columnList = "registrarId"),
@Index(columnList = "eventTime"),
@Index(columnList = "billingTime"),
@Index(columnList = "syntheticCreationTime"),
@Index(columnList = "domainRepoId"),
@Index(columnList = "allocationToken"),
@Index(columnList = "cancellation_matching_billing_recurrence_id")
})
@AttributeOverride(name = "id", column = @Column(name = "billing_event_id"))
@WithVKey(Long.class)
public static class OneTime extends BillingEvent {
/** The billable value. */
@Type(type = JodaMoneyType.TYPE_NAME)
@Columns(columns = {@Column(name = "cost_amount"), @Column(name = "cost_currency")})
Money cost;
/** When the cost should be billed. */
DateTime billingTime;
/**
* The period in years of the action being billed for, if applicable, otherwise null. Used for
* financial reporting.
*/
Integer periodYears;
/**
* For {@link Flag#SYNTHETIC} events, when this event was persisted to the database (i.e. the
* cursor position at the time the recurrence expansion job was last run). In the event a job
* needs to be undone, a query on this field will return the complete set of potentially bad
* events.
*/
DateTime syntheticCreationTime;
/**
* For {@link Flag#SYNTHETIC} events, a {@link VKey} to the {@link Recurring} from which this
* {@link OneTime} was created. This is needed in order to properly match billing events against
* {@link Cancellation}s.
*/
@Column(name = "cancellation_matching_billing_recurrence_id")
VKey<Recurring> cancellationMatchingBillingEvent;
/**
* For {@link Flag#SYNTHETIC} events, the {@link DomainHistory} revision ID of the {@link
* Recurring} from which this {@link OneTime} was created. This is needed in order to recreate
* the {@link VKey} when reading from SQL.
*/
@Column(name = "recurrence_history_revision_id")
Long recurringEventHistoryRevisionId;
/**
* The {@link AllocationToken} used in the creation of this event, or null if one was not used.
*/
@Nullable VKey<AllocationToken> allocationToken;
public Money getCost() {
return cost;
}
public DateTime getBillingTime() {
return billingTime;
}
public Integer getPeriodYears() {
return periodYears;
}
public DateTime getSyntheticCreationTime() {
return syntheticCreationTime;
}
public VKey<Recurring> getCancellationMatchingBillingEvent() {
return cancellationMatchingBillingEvent;
}
public Long getRecurringEventHistoryRevisionId() {
return recurringEventHistoryRevisionId;
}
public Optional<VKey<AllocationToken>> getAllocationToken() {
return Optional.ofNullable(allocationToken);
}
@Override
public VKey<OneTime> createVKey() {
return createVKey(getId());
}
public static VKey<OneTime> createVKey(long id) {
return VKey.create(OneTime.class, id);
}
@Override
public Builder asBuilder() {
return new Builder(clone(this));
}
/** A builder for {@link OneTime} since it is immutable. */
public static class Builder extends BillingEvent.Builder<OneTime, Builder> {
public Builder() {}
private Builder(OneTime instance) {
super(instance);
}
public Builder setCost(Money cost) {
getInstance().cost = cost;
return this;
}
public Builder setPeriodYears(Integer periodYears) {
checkNotNull(periodYears);
checkArgument(periodYears > 0);
getInstance().periodYears = periodYears;
return this;
}
public Builder setBillingTime(DateTime billingTime) {
getInstance().billingTime = billingTime;
return this;
}
public Builder setSyntheticCreationTime(DateTime syntheticCreationTime) {
getInstance().syntheticCreationTime = syntheticCreationTime;
return this;
}
public Builder setCancellationMatchingBillingEvent(
Recurring cancellationMatchingBillingEvent) {
getInstance().cancellationMatchingBillingEvent =
cancellationMatchingBillingEvent.createVKey();
getInstance().recurringEventHistoryRevisionId =
cancellationMatchingBillingEvent.getDomainHistoryRevisionId();
return this;
}
public Builder setAllocationToken(@Nullable VKey<AllocationToken> allocationToken) {
getInstance().allocationToken = allocationToken;
return this;
}
@Override
public OneTime build() {
OneTime instance = getInstance();
checkNotNull(instance.billingTime);
checkNotNull(instance.cost);
checkState(!instance.cost.isNegative(), "Costs should be non-negative.");
// TODO(mcilwain): Enforce this check on all billing events (not just more recent ones)
// post-migration after we add the missing period years values in SQL.
if (instance.eventTime.isAfter(DateTime.parse("2019-01-01T00:00:00Z"))) {
checkState(
instance.reason.hasPeriodYears() == (instance.periodYears != null),
"Period years must be set if and only if reason is "
+ "CREATE, FEE_EARLY_ACCESS, RENEW, RESTORE or TRANSFER.");
}
checkState(
instance.getFlags().contains(Flag.SYNTHETIC)
== (instance.syntheticCreationTime != null),
"Synthetic creation time must be set if and only if the SYNTHETIC flag is set.");
checkState(
instance.getFlags().contains(Flag.SYNTHETIC)
== (instance.cancellationMatchingBillingEvent != null),
"Cancellation matching billing event must be set if and only if the SYNTHETIC flag "
+ "is set.");
return super.build();
}
}
}
/**
* A recurring billable event.
*
* <p>Unlike {@link OneTime} events, these do not store an explicit cost, since the cost of the
* recurring event might change and each time we bill for it, we need to bill at the current cost,
* not the value that was in use at the time the recurrence was created.
*/
@Entity(name = "BillingRecurrence")
@Table(
indexes = {
@Index(columnList = "registrarId"),
@Index(columnList = "eventTime"),
@Index(columnList = "domainRepoId"),
@Index(columnList = "recurrenceEndTime"),
@Index(columnList = "recurrenceLastExpansion"),
@Index(columnList = "recurrence_time_of_year")
})
@AttributeOverride(name = "id", column = @Column(name = "billing_recurrence_id"))
@WithVKey(Long.class)
public static class Recurring extends BillingEvent {
/**
* The billing event recurs every year between {@link #eventTime} and this time on the [month,
* day, time] specified in {@link #recurrenceTimeOfYear}.
*/
DateTime recurrenceEndTime;
/**
* The most recent {@link DateTime} when this recurrence was expanded.
*
* <p>We only bother checking recurrences for potential expansion if this is at least one year
* in the past. If it's more recent than that, it means that the recurrence was already expanded
* too recently to need to be checked again (as domains autorenew each year).
*/
@Column(nullable = false)
DateTime recurrenceLastExpansion;
/**
* The eventTime recurs every year on this [month, day, time] between {@link #eventTime} and
* {@link #recurrenceEndTime}, inclusive of the start but not of the end.
*
* <p>This field is denormalized from {@link #eventTime} to allow for an efficient index, but it
* always has the same data as that field.
*
* <p>Note that this is a recurrence of the event time, not the billing time. The billing time
* can be calculated by adding the relevant grace period length to this date. The reason for
* this requirement is that the event time recurs on a {@link org.joda.time.Period} schedule
* (same day of year, which can be 365 or 366 days later) which is what {@link TimeOfYear} can
* model, whereas the billing time is a fixed {@link org.joda.time.Duration} later.
*/
@Embedded
@AttributeOverrides(
@AttributeOverride(name = "timeString", column = @Column(name = "recurrence_time_of_year")))
TimeOfYear recurrenceTimeOfYear;
/**
* The renewal price for domain renewal if and only if it's specified.
*
* <p>This price column remains null except when the renewal price behavior of the billing is
* SPECIFIED. This column is used for internal registrations.
*/
@Nullable
@Type(type = JodaMoneyType.TYPE_NAME)
@Columns(
columns = {@Column(name = "renewalPriceAmount"), @Column(name = "renewalPriceCurrency")})
Money renewalPrice;
@Enumerated(EnumType.STRING)
@Column(name = "renewalPriceBehavior", nullable = false)
RenewalPriceBehavior renewalPriceBehavior = RenewalPriceBehavior.DEFAULT;
public DateTime getRecurrenceEndTime() {
return recurrenceEndTime;
}
public DateTime getRecurrenceLastExpansion() {
return recurrenceLastExpansion;
}
public TimeOfYear getRecurrenceTimeOfYear() {
return recurrenceTimeOfYear;
}
public RenewalPriceBehavior getRenewalPriceBehavior() {
return renewalPriceBehavior;
}
public Optional<Money> getRenewalPrice() {
return Optional.ofNullable(renewalPrice);
}
@Override
public VKey<Recurring> createVKey() {
return createVKey(getId());
}
public static VKey<Recurring> createVKey(Long id) {
return VKey.create(Recurring.class, id);
}
@Override
public Builder asBuilder() {
return new Builder(clone(this));
}
/** A builder for {@link Recurring} since it is immutable. */
public static class Builder extends BillingEvent.Builder<Recurring, Builder> {
public Builder() {}
private Builder(Recurring instance) {
super(instance);
}
public Builder setRecurrenceEndTime(DateTime recurrenceEndTime) {
getInstance().recurrenceEndTime = recurrenceEndTime;
return this;
}
public Builder setRecurrenceLastExpansion(DateTime recurrenceLastExpansion) {
getInstance().recurrenceLastExpansion = recurrenceLastExpansion;
return this;
}
public Builder setRenewalPriceBehavior(RenewalPriceBehavior renewalPriceBehavior) {
getInstance().renewalPriceBehavior = renewalPriceBehavior;
return this;
}
public Builder setRenewalPrice(@Nullable Money renewalPrice) {
getInstance().renewalPrice = renewalPrice;
return this;
}
@Override
public Recurring build() {
Recurring instance = getInstance();
checkNotNull(instance.eventTime);
checkNotNull(instance.reason);
// Don't require recurrenceLastExpansion to be individually set on every new Recurrence.
// The correct default value if not otherwise set is the event time of the recurrence minus
// 1 year.
instance.recurrenceLastExpansion =
Optional.ofNullable(instance.recurrenceLastExpansion)
.orElse(instance.eventTime.minusYears(1));
checkArgument(
instance.renewalPriceBehavior == RenewalPriceBehavior.SPECIFIED
^ instance.renewalPrice == null,
"Renewal price can have a value if and only if the renewal price behavior is"
+ " SPECIFIED");
instance.recurrenceTimeOfYear = TimeOfYear.fromDateTime(instance.eventTime);
instance.recurrenceEndTime =
Optional.ofNullable(instance.recurrenceEndTime).orElse(END_OF_TIME);
return super.build();
}
}
}
/**
* An event representing a cancellation of one of the other two billable event types.
*
* <p>This is implemented as a separate event rather than a bit on BillingEvent in order to
* preserve the immutability of billing events.
*/
@Entity(name = "BillingCancellation")
@Table(
indexes = {
@Index(columnList = "registrarId"),
@Index(columnList = "eventTime"),
@Index(columnList = "domainRepoId"),
@Index(columnList = "billingTime"),
@Index(columnList = "billing_event_id"),
@Index(columnList = "billing_recurrence_id")
})
@AttributeOverride(name = "id", column = @Column(name = "billing_cancellation_id"))
@WithVKey(Long.class)
public static class Cancellation extends BillingEvent {
/** The billing time of the charge that is being cancelled. */
DateTime billingTime;
/**
* The one-time billing event to cancel, or null for autorenew cancellations.
*
* <p>Although the type is {@link VKey} the name "ref" is preserved for historical reasons.
*/
@Column(name = "billing_event_id")
VKey<OneTime> refOneTime;
/**
* The recurring billing event to cancel, or null for non-autorenew cancellations.
*
* <p>Although the type is {@link VKey} the name "ref" is preserved for historical reasons.
*/
@Column(name = "billing_recurrence_id")
VKey<Recurring> refRecurring;
public DateTime getBillingTime() {
return billingTime;
}
public VKey<? extends BillingEvent> getEventKey() {
return firstNonNull(refOneTime, refRecurring);
}
/** The mapping from billable grace period types to originating billing event reasons. */
static final ImmutableMap<GracePeriodStatus, Reason> GRACE_PERIOD_TO_REASON =
ImmutableMap.of(
GracePeriodStatus.ADD, Reason.CREATE,
GracePeriodStatus.AUTO_RENEW, Reason.RENEW,
GracePeriodStatus.RENEW, Reason.RENEW,
GracePeriodStatus.TRANSFER, Reason.TRANSFER);
/**
* Creates a cancellation billing event (parented on the provided history key, and with the
* corresponding event time) that will cancel out the provided grace period's billing event,
* using the supplied targetId and deriving other metadata (clientId, billing time, and the
* cancellation reason) from the grace period.
*/
public static Cancellation forGracePeriod(
GracePeriod gracePeriod,
DateTime eventTime,
HistoryEntryId domainHistoryId,
String targetId) {
checkArgument(
gracePeriod.hasBillingEvent(),
"Cannot create cancellation for grace period without billing event");
Builder builder =
new Builder()
.setReason(checkNotNull(GRACE_PERIOD_TO_REASON.get(gracePeriod.getType())))
.setTargetId(targetId)
.setRegistrarId(gracePeriod.getRegistrarId())
.setEventTime(eventTime)
// The charge being cancelled will take place at the grace period's expiration time.
.setBillingTime(gracePeriod.getExpirationTime())
.setDomainHistoryId(domainHistoryId);
// Set the grace period's billing event using the appropriate Cancellation builder method.
if (gracePeriod.getOneTimeBillingEvent() != null) {
builder.setOneTimeEventKey(gracePeriod.getOneTimeBillingEvent());
} else if (gracePeriod.getRecurringBillingEvent() != null) {
builder.setRecurringEventKey(gracePeriod.getRecurringBillingEvent());
}
return builder.build();
}
@Override
public VKey<Cancellation> createVKey() {
return createVKey(getId());
}
public static VKey<Cancellation> createVKey(long id) {
return VKey.create(Cancellation.class, id);
}
@Override
public Builder asBuilder() {
return new Builder(clone(this));
}
/** A builder for {@link Cancellation} since it is immutable. */
public static class Builder extends BillingEvent.Builder<Cancellation, Builder> {
public Builder() {}
private Builder(Cancellation instance) {
super(instance);
}
public Builder setBillingTime(DateTime billingTime) {
getInstance().billingTime = billingTime;
return this;
}
public Builder setOneTimeEventKey(VKey<OneTime> eventKey) {
getInstance().refOneTime = eventKey;
return this;
}
public Builder setRecurringEventKey(VKey<Recurring> eventKey) {
getInstance().refRecurring = eventKey;
return this;
}
@Override
public Cancellation build() {
Cancellation instance = getInstance();
checkNotNull(instance.billingTime, "Must set billing time");
checkNotNull(instance.reason, "Must set reason");
checkState(
(instance.refOneTime == null) != (instance.refRecurring == null),
"Cancellations must have exactly one billing event key set");
return super.build();
}
}
}
}

View file

@ -0,0 +1,199 @@
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package google.registry.model.billing;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static google.registry.util.DateTimeUtils.END_OF_TIME;
import google.registry.model.common.TimeOfYear;
import google.registry.persistence.VKey;
import google.registry.persistence.WithVKey;
import google.registry.persistence.converter.JodaMoneyType;
import java.util.Optional;
import javax.annotation.Nullable;
import javax.persistence.AttributeOverride;
import javax.persistence.AttributeOverrides;
import javax.persistence.Column;
import javax.persistence.Embedded;
import javax.persistence.Entity;
import javax.persistence.EnumType;
import javax.persistence.Enumerated;
import javax.persistence.Index;
import javax.persistence.Table;
import org.hibernate.annotations.Columns;
import org.hibernate.annotations.Type;
import org.joda.money.Money;
import org.joda.time.DateTime;
/**
* A recurring billable event.
*
* <p>Unlike {@link BillingEvent} events, these do not store an explicit cost, since the cost of the
* recurring event might change and each time we bill for it, we need to bill at the current cost,
* not the value that was in use at the time the recurrence was created.
*/
@Entity
@Table(
indexes = {
@Index(columnList = "registrarId"),
@Index(columnList = "eventTime"),
@Index(columnList = "domainRepoId"),
@Index(columnList = "recurrenceEndTime"),
@Index(columnList = "recurrenceLastExpansion"),
@Index(columnList = "recurrence_time_of_year")
})
@AttributeOverride(name = "id", column = @Column(name = "billing_recurrence_id"))
@WithVKey(Long.class)
public class BillingRecurrence extends BillingBase {
/**
* The billing event recurs every year between {@link #eventTime} and this time on the [month,
* day, time] specified in {@link #recurrenceTimeOfYear}.
*/
DateTime recurrenceEndTime;
/**
* The most recent {@link DateTime} when this recurrence was expanded.
*
* <p>We only bother checking recurrences for potential expansion if this is at least one year in
* the past. If it's more recent than that, it means that the recurrence was already expanded too
* recently to need to be checked again (as domains autorenew each year).
*/
@Column(nullable = false)
DateTime recurrenceLastExpansion;
/**
* The eventTime recurs every year on this [month, day, time] between {@link #eventTime} and
* {@link #recurrenceEndTime}, inclusive of the start but not of the end.
*
* <p>This field is denormalized from {@link #eventTime} to allow for an efficient index, but it
* always has the same data as that field.
*
* <p>Note that this is a recurrence of the event time, not the billing time. The billing time can
* be calculated by adding the relevant grace period length to this date. The reason for this
* requirement is that the event time recurs on a {@link org.joda.time.Period} schedule (same day
* of year, which can be 365 or 366 days later) which is what {@link TimeOfYear} can model,
* whereas the billing time is a fixed {@link org.joda.time.Duration} later.
*/
@Embedded
@AttributeOverrides(
@AttributeOverride(name = "timeString", column = @Column(name = "recurrence_time_of_year")))
TimeOfYear recurrenceTimeOfYear;
/**
* The renewal price for domain renewal if and only if it's specified.
*
* <p>This price column remains null except when the renewal price behavior of the billing is
* SPECIFIED. This column is used for internal registrations.
*/
@Nullable
@Type(type = JodaMoneyType.TYPE_NAME)
@Columns(columns = {@Column(name = "renewalPriceAmount"), @Column(name = "renewalPriceCurrency")})
Money renewalPrice;
@Enumerated(EnumType.STRING)
@Column(name = "renewalPriceBehavior", nullable = false)
RenewalPriceBehavior renewalPriceBehavior = RenewalPriceBehavior.DEFAULT;
public DateTime getRecurrenceEndTime() {
return recurrenceEndTime;
}
public DateTime getRecurrenceLastExpansion() {
return recurrenceLastExpansion;
}
public TimeOfYear getRecurrenceTimeOfYear() {
return recurrenceTimeOfYear;
}
public RenewalPriceBehavior getRenewalPriceBehavior() {
return renewalPriceBehavior;
}
public Optional<Money> getRenewalPrice() {
return Optional.ofNullable(renewalPrice);
}
@Override
public VKey<google.registry.model.billing.BillingRecurrence> createVKey() {
return createVKey(getId());
}
public static VKey<google.registry.model.billing.BillingRecurrence> createVKey(Long id) {
return VKey.create(google.registry.model.billing.BillingRecurrence.class, id);
}
@Override
public Builder asBuilder() {
return new Builder(clone(this));
}
/**
* A builder for {@link google.registry.model.billing.BillingRecurrence} since it is immutable.
*/
public static class Builder
extends BillingBase.Builder<google.registry.model.billing.BillingRecurrence, Builder> {
public Builder() {}
private Builder(google.registry.model.billing.BillingRecurrence instance) {
super(instance);
}
public Builder setRecurrenceEndTime(DateTime recurrenceEndTime) {
getInstance().recurrenceEndTime = recurrenceEndTime;
return this;
}
public Builder setRecurrenceLastExpansion(DateTime recurrenceLastExpansion) {
getInstance().recurrenceLastExpansion = recurrenceLastExpansion;
return this;
}
public Builder setRenewalPriceBehavior(RenewalPriceBehavior renewalPriceBehavior) {
getInstance().renewalPriceBehavior = renewalPriceBehavior;
return this;
}
public Builder setRenewalPrice(@Nullable Money renewalPrice) {
getInstance().renewalPrice = renewalPrice;
return this;
}
@Override
public google.registry.model.billing.BillingRecurrence build() {
google.registry.model.billing.BillingRecurrence instance = getInstance();
checkNotNull(instance.eventTime);
checkNotNull(instance.reason);
// Don't require recurrenceLastExpansion to be individually set on every new Recurrence.
// The correct default value if not otherwise set is the event time of the recurrence minus
// 1 year.
instance.recurrenceLastExpansion =
Optional.ofNullable(instance.recurrenceLastExpansion)
.orElse(instance.eventTime.minusYears(1));
checkArgument(
instance.renewalPriceBehavior == RenewalPriceBehavior.SPECIFIED
^ instance.renewalPrice == null,
"Renewal price can have a value if and only if the renewal price behavior is"
+ " SPECIFIED");
instance.recurrenceTimeOfYear = TimeOfYear.fromDateTime(instance.eventTime);
instance.recurrenceEndTime =
Optional.ofNullable(instance.recurrenceEndTime).orElse(END_OF_TIME);
return super.build();
}
}
}

View file

@ -44,7 +44,7 @@ import com.google.gson.annotations.Expose;
import google.registry.flows.ResourceFlowUtils;
import google.registry.model.EppResource;
import google.registry.model.EppResource.ResourceWithTransferData;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.contact.Contact;
import google.registry.model.domain.launch.LaunchNotice;
import google.registry.model.domain.rgp.GracePeriodStatus;
@ -200,7 +200,7 @@ public class DomainBase extends EppResource
* should be created, and this field should be updated to point to the new one.
*/
@Column(name = "billing_recurrence_id")
VKey<Recurring> autorenewBillingEvent;
VKey<BillingRecurrence> autorenewBillingEvent;
/**
* The recurring poll message associated with this domain's autorenewals.
@ -286,7 +286,7 @@ public class DomainBase extends EppResource
return deletePollMessage;
}
public VKey<Recurring> getAutorenewBillingEvent() {
public VKey<BillingRecurrence> getAutorenewBillingEvent() {
return autorenewBillingEvent;
}
@ -520,7 +520,7 @@ public class DomainBase extends EppResource
builder
.setRegistrationExpirationTime(newExpirationTime)
.addGracePeriod(
GracePeriod.createForRecurring(
GracePeriod.createForRecurrence(
GracePeriodStatus.AUTO_RENEW,
domain.getRepoId(),
lastAutorenewTime.plus(Tld.get(domain.getTld()).getAutoRenewGracePeriodLength()),
@ -847,7 +847,7 @@ public class DomainBase extends EppResource
return thisCastToDerived();
}
public B setAutorenewBillingEvent(VKey<Recurring> autorenewBillingEvent) {
public B setAutorenewBillingEvent(VKey<BillingRecurrence> autorenewBillingEvent) {
getInstance().autorenewBillingEvent = autorenewBillingEvent;
return thisCastToDerived();
}

View file

@ -20,7 +20,7 @@ import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
import com.google.common.annotations.VisibleForTesting;
import google.registry.model.billing.BillingEvent;
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;
@ -60,23 +60,23 @@ public class GracePeriod extends GracePeriodBase {
String domainRepoId,
DateTime expirationTime,
String registrarId,
@Nullable VKey<BillingEvent.OneTime> billingEventOneTime,
@Nullable VKey<BillingEvent.Recurring> billingEventRecurring,
@Nullable VKey<BillingEvent> billingEvent,
@Nullable VKey<BillingRecurrence> billingRecurrence,
@Nullable Long gracePeriodId) {
checkArgument(
billingEventOneTime == null || billingEventRecurring == null,
billingEvent == null || billingRecurrence == null,
"A grace period can have at most one billing event");
checkArgument(
(billingEventRecurring != null) == GracePeriodStatus.AUTO_RENEW.equals(type),
"Recurring billing events must be present on (and only on) autorenew grace periods");
(billingRecurrence != null) == GracePeriodStatus.AUTO_RENEW.equals(type),
"BillingRecurrences must be present on (and only on) autorenew grace periods");
GracePeriod instance = new GracePeriod();
instance.gracePeriodId = gracePeriodId == null ? allocateId() : gracePeriodId;
instance.type = checkArgumentNotNull(type);
instance.domainRepoId = checkArgumentNotNull(domainRepoId);
instance.expirationTime = checkArgumentNotNull(expirationTime);
instance.clientId = checkArgumentNotNull(registrarId);
instance.billingEventOneTime = billingEventOneTime;
instance.billingEventRecurring = billingEventRecurring;
instance.billingEvent = billingEvent;
instance.billingRecurrence = billingRecurrence;
return instance;
}
@ -92,7 +92,7 @@ public class GracePeriod extends GracePeriodBase {
String domainRepoId,
DateTime expirationTime,
String registrarId,
@Nullable VKey<BillingEvent.OneTime> billingEventOneTime) {
@Nullable VKey<BillingEvent> billingEventOneTime) {
return createInternal(
type, domainRepoId, expirationTime, registrarId, billingEventOneTime, null, null);
}
@ -111,7 +111,7 @@ public class GracePeriod extends GracePeriodBase {
String domainRepoId,
DateTime expirationTime,
String registrarId,
@Nullable VKey<BillingEvent.OneTime> billingEventOneTime,
@Nullable VKey<BillingEvent> billingEventOneTime,
@Nullable Long gracePeriodId) {
return createInternal(
type, domainRepoId, expirationTime, registrarId, billingEventOneTime, null, gracePeriodId);
@ -123,40 +123,40 @@ public class GracePeriod extends GracePeriodBase {
history.domainRepoId,
history.expirationTime,
history.clientId,
history.billingEventOneTime,
history.billingEventRecurring,
history.billingEvent,
history.billingRecurrence,
history.gracePeriodId);
}
/** Creates a GracePeriod for a Recurring billing event. */
public static GracePeriod createForRecurring(
/** Creates a GracePeriod for a Recurrence billing event. */
public static GracePeriod createForRecurrence(
GracePeriodStatus type,
String domainRepoId,
DateTime expirationTime,
String registrarId,
VKey<Recurring> billingEventRecurring) {
checkArgumentNotNull(billingEventRecurring, "billingEventRecurring cannot be null");
VKey<BillingRecurrence> billingEventRecurrence) {
checkArgumentNotNull(billingEventRecurrence, "billingEventRecurrence cannot be null");
return createInternal(
type, domainRepoId, expirationTime, registrarId, null, billingEventRecurring, null);
type, domainRepoId, expirationTime, registrarId, null, billingEventRecurrence, null);
}
/** Creates a GracePeriod for a Recurring billing event and a given {@link #gracePeriodId}. */
/** Creates a GracePeriod for a Recurrence billing event and a given {@link #gracePeriodId}. */
@VisibleForTesting
public static GracePeriod createForRecurring(
public static GracePeriod createForRecurrence(
GracePeriodStatus type,
String domainRepoId,
DateTime expirationTime,
String registrarId,
VKey<Recurring> billingEventRecurring,
VKey<BillingRecurrence> billingEventRecurrence,
@Nullable Long gracePeriodId) {
checkArgumentNotNull(billingEventRecurring, "billingEventRecurring cannot be null");
checkArgumentNotNull(billingEventRecurrence, "billingEventRecurrence cannot be null");
return createInternal(
type,
domainRepoId,
expirationTime,
registrarId,
null,
billingEventRecurring,
billingEventRecurrence,
gracePeriodId);
}
@ -168,7 +168,7 @@ public class GracePeriod extends GracePeriodBase {
/** Constructs a GracePeriod of the given type from the provided one-time BillingEvent. */
public static GracePeriod forBillingEvent(
GracePeriodStatus type, String domainRepoId, BillingEvent.OneTime billingEvent) {
GracePeriodStatus type, String domainRepoId, BillingEvent billingEvent) {
return create(
type,
domainRepoId,
@ -205,8 +205,8 @@ public class GracePeriod extends GracePeriodBase {
instance.domainRepoId = gracePeriod.domainRepoId;
instance.expirationTime = gracePeriod.expirationTime;
instance.clientId = gracePeriod.clientId;
instance.billingEventOneTime = gracePeriod.billingEventOneTime;
instance.billingEventRecurring = gracePeriod.billingEventRecurring;
instance.billingEvent = gracePeriod.billingEvent;
instance.billingRecurrence = gracePeriod.billingRecurrence;
return instance;
}
}

View file

@ -17,6 +17,7 @@ package google.registry.model.domain;
import google.registry.model.ImmutableObject;
import google.registry.model.UnsafeSerializable;
import google.registry.model.billing.BillingEvent;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.rgp.GracePeriodStatus;
import google.registry.persistence.VKey;
import javax.persistence.Access;
@ -56,21 +57,21 @@ public class GracePeriodBase extends ImmutableObject implements UnsafeSerializab
/**
* The one-time billing event corresponding to the action that triggered this grace period, or
* null if not applicable. Not set for autorenew grace periods (which instead use the field {@code
* billingEventRecurring}) or for redemption grace periods (since deletes have no cost).
* billingEventRecurrence}) or for redemption grace periods (since deletes have no cost).
*/
// NB: Would @IgnoreSave(IfNull.class), but not allowed for @Embed collections.
@Access(AccessType.FIELD)
@Column(name = "billing_event_id")
VKey<BillingEvent.OneTime> billingEventOneTime = null;
VKey<BillingEvent> billingEvent = null;
/**
* The recurring billing event corresponding to the action that triggered this grace period, if
* applicable - i.e. if the action was an autorenew - or null in all other cases.
* The Recurrence corresponding to the action that triggered this grace period, if applicable -
* i.e. if the action was an autorenew - or null in all other cases.
*/
// NB: Would @IgnoreSave(IfNull.class), but not allowed for @Embed collections.
@Access(AccessType.FIELD)
@Column(name = "billing_recurrence_id")
VKey<BillingEvent.Recurring> billingEventRecurring = null;
VKey<BillingRecurrence> billingRecurrence = null;
public long getGracePeriodId() {
return gracePeriodId;
@ -100,22 +101,22 @@ public class GracePeriodBase extends ImmutableObject implements UnsafeSerializab
/** Returns true if this GracePeriod has an associated BillingEvent; i.e. if it's refundable. */
public boolean hasBillingEvent() {
return billingEventOneTime != null || billingEventRecurring != null;
return billingEvent != null || billingRecurrence != null;
}
/**
* Returns the one time billing event. The value will only be non-null if the type of this grace
* period is not AUTO_RENEW.
*/
public VKey<BillingEvent.OneTime> getOneTimeBillingEvent() {
return billingEventOneTime;
public VKey<BillingEvent> getBillingEvent() {
return billingEvent;
}
/**
* Returns the recurring billing event. The value will only be non-null if the type of this grace
* period is AUTO_RENEW.
* Returns the Recurrence. The value will only be non-null if the type of this grace period is
* AUTO_RENEW.
*/
public VKey<BillingEvent.Recurring> getRecurringBillingEvent() {
return billingEventRecurring;
public VKey<BillingRecurrence> getBillingRecurrence() {
return billingRecurrence;
}
}

View file

@ -43,7 +43,7 @@ import google.registry.model.Buildable;
import google.registry.model.CacheUtils;
import google.registry.model.CreateAutoTimestamp;
import google.registry.model.UpdateAutoTimestampEntity;
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
import google.registry.model.billing.BillingBase.RenewalPriceBehavior;
import google.registry.model.common.TimedTransitionProperty;
import google.registry.model.domain.fee.FeeQueryCommandExtensionItem.CommandName;
import google.registry.model.reporting.HistoryEntry.HistoryEntryId;

View file

@ -17,6 +17,7 @@ package google.registry.model.reporting;
import static com.google.common.base.Preconditions.checkArgument;
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
import google.registry.batch.ExpandBillingRecurrencesAction;
import google.registry.model.Buildable;
import google.registry.model.EppResource;
import google.registry.model.ImmutableObject;
@ -81,7 +82,7 @@ public abstract class HistoryEntry extends ImmutableObject
DOMAIN_ALLOCATE,
/**
* Used for domain registration autorenews explicitly logged by {@link
* google.registry.batch.ExpandRecurringBillingEventsAction}.
* ExpandBillingRecurrencesAction}.
*/
DOMAIN_AUTORENEW,
DOMAIN_CREATE,

View file

@ -17,7 +17,9 @@ package google.registry.model.transfer;
import static google.registry.util.CollectionUtils.isNullOrEmpty;
import com.google.common.collect.ImmutableSet;
import google.registry.model.billing.BillingCancellation;
import google.registry.model.billing.BillingEvent;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Period;
import google.registry.model.domain.Period.Unit;
import google.registry.model.poll.PollMessage;
@ -69,7 +71,7 @@ public class DomainTransferData extends TransferData {
DateTime transferredRegistrationExpirationTime;
@Column(name = "transfer_billing_cancellation_id")
public VKey<BillingEvent.Cancellation> billingCancellationId;
public VKey<BillingCancellation> billingCancellationId;
/**
* The regular one-time billing event that will be charged for a server-approved transfer.
@ -78,7 +80,7 @@ public class DomainTransferData extends TransferData {
* being transferred is not a domain.
*/
@Column(name = "transfer_billing_event_id")
VKey<BillingEvent.OneTime> serverApproveBillingEvent;
VKey<BillingEvent> serverApproveBillingEvent;
/**
* The autorenew billing event that should be associated with this resource after the transfer.
@ -87,7 +89,7 @@ public class DomainTransferData extends TransferData {
* being transferred is not a domain.
*/
@Column(name = "transfer_billing_recurrence_id")
VKey<BillingEvent.Recurring> serverApproveAutorenewEvent;
VKey<BillingRecurrence> serverApproveAutorenewEvent;
/**
* The autorenew poll message that should be associated with this resource after the transfer.
@ -120,12 +122,12 @@ public class DomainTransferData extends TransferData {
}
@Nullable
public VKey<BillingEvent.OneTime> getServerApproveBillingEvent() {
public VKey<BillingEvent> getServerApproveBillingEvent() {
return serverApproveBillingEvent;
}
@Nullable
public VKey<BillingEvent.Recurring> getServerApproveAutorenewEvent() {
public VKey<BillingRecurrence> getServerApproveAutorenewEvent() {
return serverApproveAutorenewEvent;
}
@ -176,9 +178,9 @@ public class DomainTransferData extends TransferData {
domainTransferData.billingCancellationId = null;
} else {
domainTransferData.billingCancellationId =
(VKey<BillingEvent.Cancellation>)
(VKey<BillingCancellation>)
serverApproveEntities.stream()
.filter(k -> k.getKind().equals(BillingEvent.Cancellation.class))
.filter(k -> k.getKind().equals(BillingCancellation.class))
.findFirst()
.orElse(null);
}
@ -204,14 +206,13 @@ public class DomainTransferData extends TransferData {
return this;
}
public Builder setServerApproveBillingEvent(
VKey<BillingEvent.OneTime> serverApproveBillingEvent) {
public Builder setServerApproveBillingEvent(VKey<BillingEvent> serverApproveBillingEvent) {
getInstance().serverApproveBillingEvent = serverApproveBillingEvent;
return this;
}
public Builder setServerApproveAutorenewEvent(
VKey<BillingEvent.Recurring> serverApproveAutorenewEvent) {
VKey<BillingRecurrence> serverApproveAutorenewEvent) {
getInstance().serverApproveAutorenewEvent = serverApproveAutorenewEvent;
return this;
}

View file

@ -21,7 +21,7 @@ import google.registry.batch.CannedScriptExecutionAction;
import google.registry.batch.DeleteExpiredDomainsAction;
import google.registry.batch.DeleteLoadTestDataAction;
import google.registry.batch.DeleteProberDataAction;
import google.registry.batch.ExpandRecurringBillingEventsAction;
import google.registry.batch.ExpandBillingRecurrencesAction;
import google.registry.batch.RelockDomainAction;
import google.registry.batch.ResaveAllEppResourcesPipelineAction;
import google.registry.batch.ResaveEntityAction;
@ -115,7 +115,7 @@ interface BackendRequestComponent {
DeleteProberDataAction deleteProberDataAction();
ExpandRecurringBillingEventsAction expandRecurringBillingEventsAction();
ExpandBillingRecurrencesAction expandBillingRecurrencesAction();
ExportDomainListsAction exportDomainListsAction();

View file

@ -14,7 +14,7 @@
package google.registry.persistence.converter;
import google.registry.model.billing.BillingEvent.Flag;
import google.registry.model.billing.BillingBase.Flag;
import java.util.Set;
import javax.persistence.AttributeConverter;
import javax.persistence.Converter;

View file

@ -26,8 +26,8 @@ import com.google.common.collect.Sets;
import google.registry.batch.CloudTasksUtils;
import google.registry.batch.RelockDomainAction;
import google.registry.config.RegistryConfig.Config;
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;
@ -397,8 +397,8 @@ public final class DomainLockUtils {
tm().update(domain);
tm().insert(domainHistory);
if (!lock.isSuperuser()) { // admin actions shouldn't affect billing
BillingEvent.OneTime oneTime =
new BillingEvent.OneTime.Builder()
BillingEvent billingEvent =
new BillingEvent.Builder()
.setReason(Reason.SERVER_STATUS)
.setTargetId(domain.getForeignKey())
.setRegistrarId(domain.getCurrentSponsorRegistrarId())
@ -407,7 +407,7 @@ public final class DomainLockUtils {
.setBillingTime(now)
.setDomainHistory(domainHistory)
.build();
tm().insert(oneTime);
tm().insert(billingEvent);
}
}
}

View file

@ -17,7 +17,7 @@ package google.registry.tools;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static com.google.common.collect.Sets.difference;
import static google.registry.model.billing.BillingEvent.RenewalPriceBehavior.DEFAULT;
import static google.registry.model.billing.BillingBase.RenewalPriceBehavior.DEFAULT;
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;
@ -39,7 +39,7 @@ import com.google.common.collect.ImmutableSortedMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Streams;
import com.google.common.io.Files;
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
import google.registry.model.billing.BillingBase.RenewalPriceBehavior;
import google.registry.model.domain.fee.FeeQueryCommandExtensionItem.CommandName;
import google.registry.model.domain.token.AllocationToken;
import google.registry.model.domain.token.AllocationToken.RegistrationBehavior;

View file

@ -15,7 +15,7 @@
package google.registry.tools;
import com.google.common.collect.ImmutableMap;
import google.registry.tools.javascrap.CreateCancellationsForOneTimesCommand;
import google.registry.tools.javascrap.CreateCancellationsForBillingEventsCommand;
/** Container class to create and run remote commands against a server instance. */
public final class RegistryTool {
@ -35,7 +35,9 @@ public final class RegistryTool {
.put("convert_idn", ConvertIdnCommand.class)
.put("count_domains", CountDomainsCommand.class)
.put("create_anchor_tenant", CreateAnchorTenantCommand.class)
.put("create_cancellations_for_one_times", CreateCancellationsForOneTimesCommand.class)
.put(
"create_cancellations_for_billing_events",
CreateCancellationsForBillingEventsCommand.class)
.put("create_cdns_tld", CreateCdnsTld.class)
.put("create_contact", CreateContactCommand.class)
.put("create_domain", CreateDomainCommand.class)

View file

@ -40,7 +40,6 @@ import google.registry.request.Modules.UrlConnectionServiceModule;
import google.registry.request.Modules.UrlFetchServiceModule;
import google.registry.request.Modules.UserServiceModule;
import google.registry.tools.AuthModule.LocalCredentialModule;
import google.registry.tools.javascrap.CreateCancellationsForOneTimesCommand;
import google.registry.util.UtilsModule;
import google.registry.whois.NonCachingWhoisModule;
import javax.annotation.Nullable;
@ -92,8 +91,6 @@ interface RegistryToolComponent {
void inject(CreateAnchorTenantCommand command);
void inject(CreateCancellationsForOneTimesCommand command);
void inject(CreateCdnsTld command);
void inject(CreateContactCommand command);

View file

@ -32,8 +32,7 @@ import com.google.common.collect.ImmutableMultimap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import google.registry.model.ForeignKeyUtils;
import google.registry.model.billing.BillingEvent;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.Period;
@ -205,7 +204,7 @@ class UnrenewDomainCommand extends ConfirmingCommand {
.setEventTime(now)
.build();
// Create a new autorenew billing event and poll message starting at the new expiration time.
BillingEvent.Recurring newAutorenewEvent =
BillingRecurrence newAutorenewEvent =
newAutorenewBillingEvent(domain)
.setEventTime(newExpirationTime)
.setDomainHistory(domainHistory)
@ -216,9 +215,9 @@ class UnrenewDomainCommand extends ConfirmingCommand {
.setHistoryEntry(domainHistory)
.build();
// End the old autorenew billing event and poll message now.
Recurring existingRecurring = tm().loadByKey(domain.getAutorenewBillingEvent());
BillingRecurrence existingBillingRecurrence = tm().loadByKey(domain.getAutorenewBillingEvent());
updateAutorenewRecurrenceEndTime(
domain, existingRecurring, now, domainHistory.getHistoryEntryId());
domain, existingBillingRecurrence, now, domainHistory.getHistoryEntryId());
Domain newDomain =
domain
.asBuilder()

View file

@ -28,7 +28,7 @@ import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedMap;
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
import google.registry.model.billing.BillingBase.RenewalPriceBehavior;
import google.registry.model.domain.fee.FeeQueryCommandExtensionItem.CommandName;
import google.registry.model.domain.token.AllocationToken;
import google.registry.model.domain.token.AllocationToken.RegistrationBehavior;

View file

@ -25,8 +25,8 @@ import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import google.registry.model.EppResourceUtils;
import google.registry.model.billing.BillingEvent.Recurring;
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
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.reporting.HistoryEntry;
@ -38,7 +38,7 @@ import org.joda.money.Money;
import org.joda.time.DateTime;
/**
* Command to update {@link Recurring} billing events with a new behavior and/or price.
* Command to update {@link BillingRecurrence} billing events with a new behavior and/or price.
*
* <p>More specifically, this closes the existing recurrence object and creates a new, similar,
* object as well as a corresponding synthetic {@link DomainHistory} object. This is done to
@ -85,11 +85,11 @@ public class UpdateRecurrenceCommand extends ConfirmingCommand {
+ " SPECIFIED");
}
}
ImmutableMap<Domain, Recurring> domainsAndRecurrings =
tm().transact(this::loadDomainsAndRecurrings);
ImmutableMap<Domain, BillingRecurrence> domainsAndRecurrences =
tm().transact(this::loadDomainsAndRecurrences);
if (renewalPriceBehavior == null) {
// Allow users to specify only a price only if all renewals are already SPECIFIED
domainsAndRecurrings.forEach(
domainsAndRecurrences.forEach(
(d, r) ->
checkArgument(
r.getRenewalPriceBehavior().equals(RenewalPriceBehavior.SPECIFIED),
@ -103,41 +103,45 @@ public class UpdateRecurrenceCommand extends ConfirmingCommand {
"Update the following with behavior %s%s?\n%s",
renewalPriceBehavior,
specifiedPriceString,
Joiner.on('\n').withKeyValueSeparator(':').join(domainsAndRecurrings));
Joiner.on('\n').withKeyValueSeparator(':').join(domainsAndRecurrences));
}
@Override
protected String execute() throws Exception {
ImmutableList<Recurring> newRecurrings = tm().transact(this::internalExecute);
return "Updated new recurring(s): " + newRecurrings;
ImmutableList<BillingRecurrence> newBillingRecurrences = tm().transact(this::internalExecute);
return "Updated new recurrence(s): " + newBillingRecurrences;
}
private ImmutableList<Recurring> internalExecute() {
ImmutableMap<Domain, Recurring> domainsAndRecurrings = loadDomainsAndRecurrings();
private ImmutableList<BillingRecurrence> internalExecute() {
ImmutableMap<Domain, BillingRecurrence> domainsAndRecurrences = loadDomainsAndRecurrences();
DateTime now = tm().getTransactionTime();
ImmutableList.Builder<Recurring> resultBuilder = new ImmutableList.Builder<>();
domainsAndRecurrings.forEach(
(domain, existingRecurring) -> {
// Make a new history ID to break the (recurring, history, domain) circular dep chain
ImmutableList.Builder<BillingRecurrence> resultBuilder = new ImmutableList.Builder<>();
domainsAndRecurrences.forEach(
(domain, existingRecurrence) -> {
// Make a new history ID to break the (recurrence, history, domain) circular dep chain
long newHistoryId = allocateId();
HistoryEntryId newDomainHistoryId = new HistoryEntryId(domain.getRepoId(), newHistoryId);
Recurring endingNow = existingRecurring.asBuilder().setRecurrenceEndTime(now).build();
Recurring.Builder newRecurringBuilder =
existingRecurring
BillingRecurrence endingNow =
existingRecurrence.asBuilder().setRecurrenceEndTime(now).build();
BillingRecurrence.Builder newRecurrenceBuilder =
existingRecurrence
.asBuilder()
// set the ID to be 0 (null) to create a new object
.setId(0)
.setDomainHistoryId(newDomainHistoryId);
if (renewalPriceBehavior != null) {
newRecurringBuilder.setRenewalPriceBehavior(renewalPriceBehavior);
newRecurringBuilder.setRenewalPrice(null);
newRecurrenceBuilder.setRenewalPriceBehavior(renewalPriceBehavior);
newRecurrenceBuilder.setRenewalPrice(null);
}
if (specifiedRenewalPrice != null) {
newRecurringBuilder.setRenewalPrice(specifiedRenewalPrice);
newRecurrenceBuilder.setRenewalPrice(specifiedRenewalPrice);
}
Recurring newRecurring = newRecurringBuilder.build();
BillingRecurrence newBillingRecurrence = newRecurrenceBuilder.build();
Domain newDomain =
domain.asBuilder().setAutorenewBillingEvent(newRecurring.createVKey()).build();
domain
.asBuilder()
.setAutorenewBillingEvent(newBillingRecurrence.createVKey())
.build();
DomainHistory newDomainHistory =
new DomainHistory.Builder()
.setRevisionId(newDomainHistoryId.getRevisionId())
@ -149,14 +153,14 @@ public class UpdateRecurrenceCommand extends ConfirmingCommand {
.setType(HistoryEntry.Type.SYNTHETIC)
.setModificationTime(now)
.build();
tm().putAll(endingNow, newRecurring, newDomain, newDomainHistory);
resultBuilder.add(newRecurring);
tm().putAll(endingNow, newBillingRecurrence, newDomain, newDomainHistory);
resultBuilder.add(newBillingRecurrence);
});
return resultBuilder.build();
}
private ImmutableMap<Domain, Recurring> loadDomainsAndRecurrings() {
ImmutableMap.Builder<Domain, Recurring> result = new ImmutableMap.Builder<>();
private ImmutableMap<Domain, BillingRecurrence> loadDomainsAndRecurrences() {
ImmutableMap.Builder<Domain, BillingRecurrence> result = new ImmutableMap.Builder<>();
DateTime now = tm().getTransactionTime();
for (String domainName : mainParameters) {
Domain domain =
@ -183,12 +187,12 @@ public class UpdateRecurrenceCommand extends ConfirmingCommand {
"Domain %s autorenew ended prior to now at %s",
domainName,
endTime));
Recurring recurring = tm().loadByKey(domain.getAutorenewBillingEvent());
BillingRecurrence billingRecurrence = tm().loadByKey(domain.getAutorenewBillingEvent());
checkArgument(
recurring.getRecurrenceEndTime().equals(END_OF_TIME),
billingRecurrence.getRecurrenceEndTime().equals(END_OF_TIME),
"Domain %s's recurrence's end date is not END_OF_TIME",
domainName);
result.put(domain, recurring);
result.put(domain, billingRecurrence);
}
return result.build();
}

View file

@ -19,9 +19,9 @@ import static google.registry.persistence.transaction.TransactionManagerFactory.
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.google.common.collect.ImmutableSet;
import google.registry.model.billing.BillingBase;
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.persistence.VKey;
import google.registry.persistence.transaction.QueryComposer.Comparator;
import google.registry.tools.ConfirmingCommand;
@ -29,14 +29,16 @@ import google.registry.tools.params.LongParameter;
import java.util.List;
/**
* Command to create {@link Cancellation}s for specified {@link OneTime} billing events.
* Command to create {@link BillingCancellation}s for specified {@link BillingEvent} billing events.
*
* <p>This can be used to fix situations where we've inadvertently billed registrars. It's generally
* easier and better to issue cancellations within the Nomulus system than to attempt to issue
* refunds after the fact.
*/
@Parameters(separators = " =", commandDescription = "Manually create Cancellations for OneTimes.")
public class CreateCancellationsForOneTimesCommand extends ConfirmingCommand {
@Parameters(
separators = " =",
commandDescription = "Manually create Cancellations for BillingEvents.")
public class CreateCancellationsForBillingEventsCommand extends ConfirmingCommand {
@Parameter(
description = "Space-delimited billing event ID(s) to cancel",
@ -44,80 +46,83 @@ public class CreateCancellationsForOneTimesCommand extends ConfirmingCommand {
validateWith = LongParameter.class)
private List<Long> mainParameters;
private ImmutableSet<OneTime> oneTimesToCancel;
private ImmutableSet<BillingEvent> billingEventsToCancel;
@Override
protected void init() {
ImmutableSet.Builder<Long> missingIdsBuilder = new ImmutableSet.Builder<>();
ImmutableSet.Builder<Long> alreadyCancelledIdsBuilder = new ImmutableSet.Builder<>();
ImmutableSet.Builder<OneTime> oneTimesBuilder = new ImmutableSet.Builder<>();
ImmutableSet.Builder<BillingEvent> billingEventsBuilder = new ImmutableSet.Builder<>();
tm().transact(
() -> {
for (Long billingEventId : ImmutableSet.copyOf(mainParameters)) {
VKey<OneTime> key = VKey.create(OneTime.class, billingEventId);
VKey<BillingEvent> key = VKey.create(BillingEvent.class, billingEventId);
if (tm().exists(key)) {
OneTime oneTime = tm().loadByKey(key);
if (alreadyCancelled(oneTime)) {
BillingEvent billingEvent = tm().loadByKey(key);
if (alreadyCancelled(billingEvent)) {
alreadyCancelledIdsBuilder.add(billingEventId);
} else {
oneTimesBuilder.add(oneTime);
billingEventsBuilder.add(billingEvent);
}
} else {
missingIdsBuilder.add(billingEventId);
}
}
});
oneTimesToCancel = oneTimesBuilder.build();
System.out.printf("Found %d OneTime(s) to cancel\n", oneTimesToCancel.size());
billingEventsToCancel = billingEventsBuilder.build();
System.out.printf("Found %d BillingEvent(s) to cancel\n", billingEventsToCancel.size());
ImmutableSet<Long> missingIds = missingIdsBuilder.build();
if (!missingIds.isEmpty()) {
System.out.printf("Missing OneTime(s) for IDs %s\n", missingIds);
System.out.printf("Missing BillingEvent(s) for IDs %s\n", missingIds);
}
ImmutableSet<Long> alreadyCancelledIds = alreadyCancelledIdsBuilder.build();
if (!alreadyCancelledIds.isEmpty()) {
System.out.printf(
"The following OneTime IDs were already cancelled: %s\n", alreadyCancelledIds);
"The following BillingEvent IDs were already cancelled: %s\n", alreadyCancelledIds);
}
}
@Override
protected String prompt() {
return String.format("Create cancellations for %d OneTime(s)?", oneTimesToCancel.size());
return String.format(
"Create cancellations for %d BillingEvent(s)?", billingEventsToCancel.size());
}
@Override
protected String execute() throws Exception {
int cancelledOneTimes = 0;
for (OneTime oneTime : oneTimesToCancel) {
cancelledOneTimes +=
int cancelledBillingEvents = 0;
for (BillingEvent billingEvent : billingEventsToCancel) {
cancelledBillingEvents +=
tm().transact(
() -> {
if (alreadyCancelled(oneTime)) {
if (alreadyCancelled(billingEvent)) {
System.out.printf(
"OneTime %d already cancelled, this is unexpected.\n", oneTime.getId());
"BillingEvent %d already cancelled, this is unexpected.\n",
billingEvent.getId());
return 0;
}
tm().put(
new Cancellation.Builder()
.setOneTimeEventKey(oneTime.createVKey())
.setBillingTime(oneTime.getBillingTime())
.setDomainHistoryId(oneTime.getHistoryEntryId())
.setRegistrarId(oneTime.getRegistrarId())
.setEventTime(oneTime.getEventTime())
.setReason(BillingEvent.Reason.ERROR)
.setTargetId(oneTime.getTargetId())
new BillingCancellation.Builder()
.setBillingEvent(billingEvent.createVKey())
.setBillingTime(billingEvent.getBillingTime())
.setDomainHistoryId(billingEvent.getHistoryEntryId())
.setRegistrarId(billingEvent.getRegistrarId())
.setEventTime(billingEvent.getEventTime())
.setReason(BillingBase.Reason.ERROR)
.setTargetId(billingEvent.getTargetId())
.build());
System.out.printf(
"Added Cancellation for OneTime with ID %d\n", oneTime.getId());
"Added BillingCancellation for BillingEvent with ID %d\n",
billingEvent.getId());
return 1;
});
}
return String.format("Created %d Cancellation event(s)", cancelledOneTimes);
return String.format("Created %d BillingCancellation event(s)", cancelledBillingEvents);
}
private boolean alreadyCancelled(OneTime oneTime) {
return tm().createQueryComposer(Cancellation.class)
.where("refOneTime", Comparator.EQ, oneTime.getId())
private boolean alreadyCancelled(BillingEvent billingEvent) {
return tm().createQueryComposer(BillingCancellation.class)
.where("billingEvent", Comparator.EQ, billingEvent.getId())
.first()
.isPresent();
}

View file

@ -38,9 +38,9 @@
<mapping-file>META-INF/orm.xml</mapping-file>
<class>google.registry.model.billing.BillingEvent$Cancellation</class>
<class>google.registry.model.billing.BillingEvent$OneTime</class>
<class>google.registry.model.billing.BillingEvent$Recurring</class>
<class>google.registry.model.billing.BillingCancellation</class>
<class>google.registry.model.billing.BillingEvent</class>
<class>google.registry.model.billing.BillingRecurrence</class>
<class>google.registry.model.common.Cursor</class>
<class>google.registry.model.common.DatabaseMigrationStateSchedule</class>
<class>google.registry.model.common.DnsRefreshRequest</class>
@ -105,9 +105,9 @@
<class>google.registry.persistence.converter.TransferServerApproveEntitySetConverter</class>
<!-- Generated converters for VKey -->
<class>google.registry.model.billing.VKeyConverter_Cancellation</class>
<class>google.registry.model.billing.VKeyConverter_OneTime</class>
<class>google.registry.model.billing.VKeyConverter_Recurring</class>
<class>google.registry.model.billing.VKeyConverter_BillingCancellation</class>
<class>google.registry.model.billing.VKeyConverter_BillingEvent</class>
<class>google.registry.model.billing.VKeyConverter_BillingRecurrence</class>
<class>google.registry.model.contact.VKeyConverter_Contact</class>
<class>google.registry.model.domain.VKeyConverter_Domain</class>
<class>google.registry.model.domain.token.VKeyConverter_AllocationToken</class>

View file

@ -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

View file

@ -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;

View file

@ -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")

View file

@ -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"))

View file

@ -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<String, String> expectedParameters = new HashMap<>();
private final ArgumentCaptor<LaunchFlexTemplateRequest> 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);

View file

@ -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 {}
}

View file

@ -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<BillingEvent> INPUT_EVENTS =
private static final ImmutableList<google.registry.beam.billing.BillingEvent> 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<BillingEvent> billingEvents;
private PCollection<google.registry.beam.billing.BillingEvent> 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<BillingEvent> billingEvents = InvoicingPipeline.readFromCloudSql(options, pipeline);
PCollection<google.registry.beam.billing.BillingEvent> 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<BillingEvent> billingEvents = InvoicingPipeline.readFromCloudSql(options, pipeline);
persistBillingEvent(25, domain, registrar, Reason.RENEW, 3, Money.of(CAD, 20.5));
PCollection<google.registry.beam.billing.BillingEvent> 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<BillingEvent>, PCollection<BillingEvent>> {
extends PTransform<
PCollection<google.registry.beam.billing.BillingEvent>,
PCollection<google.registry.beam.billing.BillingEvent>> {
private static final long serialVersionUID = 2695033474967615250L;
@Override
public PCollection<BillingEvent> expand(PCollection<BillingEvent> input) {
public PCollection<google.registry.beam.billing.BillingEvent> expand(
PCollection<google.registry.beam.billing.BillingEvent> 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(),

View file

@ -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.

View file

@ -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<OneTime> findKeyToActualOneTimeBillingEvent(OneTime expectedBillingEvent) {
Optional<OneTime> actualCreateBillingEvent =
loadAllOf(BillingEvent.OneTime.class).stream()
private static VKey<BillingEvent> findKeyToActualOneTimeBillingEvent(
BillingEvent expectedBillingEvent) {
Optional<BillingEvent> actualCreateBillingEvent =
loadAllOf(BillingEvent.class).stream()
.filter(
b ->
Objects.equals(

View file

@ -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<F extends Flow> {
* 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<GracePeriod, BillingEvent> canonicalizeGracePeriods(
ImmutableMap<GracePeriod, ? extends BillingEvent> gracePeriods) {
ImmutableMap.Builder<GracePeriod, BillingEvent> builder = new ImmutableMap.Builder<>();
for (Map.Entry<GracePeriod, ? extends BillingEvent> entry : gracePeriods.entrySet()) {
private static ImmutableMap<GracePeriod, BillingBase> canonicalizeGracePeriods(
ImmutableMap<GracePeriod, ? extends BillingBase> gracePeriods) {
ImmutableMap.Builder<GracePeriod, BillingBase> builder = new ImmutableMap.Builder<>();
for (Map.Entry<GracePeriod, ? extends BillingBase> entry : gracePeriods.entrySet()) {
builder.put(
GracePeriod.create(
entry.getKey().getType(),
@ -186,7 +186,7 @@ public abstract class FlowTestCase<F extends Flow> {
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<F extends Flow> {
() ->
tm().loadByKey(
firstNonNull(
gracePeriod.getOneTimeBillingEvent(),
gracePeriod.getRecurringBillingEvent())));
gracePeriod.getBillingEvent(), gracePeriod.getBillingRecurrence())));
}
/**
@ -204,7 +203,7 @@ public abstract class FlowTestCase<F extends Flow> {
* on the grace periods and IDs on the billing events are ignored.
*/
protected static void assertGracePeriods(
Iterable<GracePeriod> actual, ImmutableMap<GracePeriod, ? extends BillingEvent> expected) {
Iterable<GracePeriod> actual, ImmutableMap<GracePeriod, ? extends BillingBase> expected) {
assertThat(canonicalizeGracePeriods(Maps.toMap(actual, FlowTestCase::expandGracePeriod)))
.isEqualTo(canonicalizeGracePeriods(expected));
}

View file

@ -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<DomainCheckFlow, Dom
.setModificationTime(existingDomain.getCreationTime())
.setRegistrarId(existingDomain.getCreationRegistrarId())
.build());
BillingEvent.Recurring renewEvent =
BillingRecurrence renewEvent =
persistResource(
new BillingEvent.Recurring.Builder()
new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId(existingDomain.getDomainName())

View file

@ -19,12 +19,12 @@ import static com.google.common.io.BaseEncoding.base16;
import static com.google.common.truth.Truth.assertThat;
import static com.google.common.truth.Truth8.assertThat;
import static google.registry.flows.FlowTestCase.UserPrivileges.SUPERUSER;
import static google.registry.model.billing.BillingEvent.Flag.ANCHOR_TENANT;
import static google.registry.model.billing.BillingEvent.Flag.RESERVED;
import static google.registry.model.billing.BillingEvent.Flag.SUNRISE;
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.ANCHOR_TENANT;
import static google.registry.model.billing.BillingBase.Flag.RESERVED;
import static google.registry.model.billing.BillingBase.Flag.SUNRISE;
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.Fee.FEE_EXTENSION_URIS;
import static google.registry.model.domain.token.AllocationToken.TokenType.DEFAULT_PROMO;
import static google.registry.model.domain.token.AllocationToken.TokenType.PACKAGE;
@ -145,10 +145,12 @@ import google.registry.flows.domain.token.AllocationTokenFlowUtils.InvalidAlloca
import google.registry.flows.exceptions.OnlyToolCanPassMetadataException;
import google.registry.flows.exceptions.ResourceAlreadyExistsForThisClientException;
import google.registry.flows.exceptions.ResourceCreateContentionException;
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.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;
@ -263,13 +265,13 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
}
private void assertSuccessfulCreate(
String domainTld, ImmutableSet<BillingEvent.Flag> expectedBillingFlags) throws Exception {
String domainTld, ImmutableSet<BillingBase.Flag> expectedBillingFlags) throws Exception {
assertSuccessfulCreate(domainTld, expectedBillingFlags, null);
}
private void assertSuccessfulCreate(
String domainTld,
ImmutableSet<BillingEvent.Flag> expectedBillingFlags,
ImmutableSet<BillingBase.Flag> expectedBillingFlags,
@Nullable AllocationToken allocationToken)
throws Exception {
Domain domain = reloadResourceByForeignKey();
@ -311,9 +313,9 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
RenewalPriceInfo renewalPriceInfo =
DomainCreateFlow.getRenewalPriceInfo(
isAnchorTenant, Optional.ofNullable(allocationToken), feesAndCredits);
// There should be one bill for the create and one for the recurring autorenew event.
BillingEvent.OneTime createBillingEvent =
new BillingEvent.OneTime.Builder()
// There should be one bill for the create and one for the recurrence autorenew event.
BillingEvent createBillingEvent =
new BillingEvent.Builder()
.setReason(Reason.CREATE)
.setTargetId(getUniqueIdFromCommand())
.setRegistrarId("TheRegistrar")
@ -326,8 +328,8 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
.setAllocationToken(allocationToken == null ? null : allocationToken.createVKey())
.build();
BillingEvent.Recurring renewBillingEvent =
new BillingEvent.Recurring.Builder()
BillingRecurrence renewBillingEvent =
new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId(getUniqueIdFromCommand())
@ -339,14 +341,14 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
.setRenewalPrice(renewalPriceInfo.renewalPrice())
.build();
ImmutableSet.Builder<BillingEvent> expectedBillingEvents =
new ImmutableSet.Builder<BillingEvent>().add(createBillingEvent).add(renewBillingEvent);
ImmutableSet.Builder<BillingBase> expectedBillingEvents =
new ImmutableSet.Builder<BillingBase>().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<DomainCreateFlow, Domain
ImmutableMap.of("DOMAIN", "example.tld", "YEARS", "2"));
runFlowAssertResponse(
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "example.tld")));
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.getCost()).isEqualTo(Money.of(USD, BigDecimal.valueOf(19.5)));
}
@ -1620,8 +1622,8 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
.put("CRDATE", "1999-04-03T22:00:00.0Z")
.put("EXDATE", "2004-04-03T22:00:00.0Z")
.build()));
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.getCost()).isEqualTo(expectedPrice);
}
@ -1653,8 +1655,8 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
loadFile(
"domain_create_response_premium.xml",
ImmutableMap.of("EXDATE", "2002-04-03T22:00:00.0Z", "FEE", "104.00")));
BillingEvent.OneTime billingEvent =
Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class));
BillingEvent billingEvent =
Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class));
assertThat(billingEvent.getTargetId()).isEqualTo("rich.example");
// 1yr @ $100 + 2yrs @ $100 * (1 - 0.98) = $104
assertThat(billingEvent.getCost()).isEqualTo(Money.of(USD, 104.00));
@ -1686,8 +1688,8 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
loadFile(
"domain_create_response_premium.xml",
ImmutableMap.of("EXDATE", "2002-04-03T22:00:00.0Z", "FEE", "204.44")));
BillingEvent.OneTime billingEvent =
Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.OneTime.class));
BillingEvent billingEvent =
Iterables.getOnlyElement(DatabaseHelper.loadAllOf(BillingEvent.class));
assertThat(billingEvent.getTargetId()).isEqualTo("rich.example");
// 2yrs @ $100 + 1yr @ $100 * (1 - 0.95555) = $204.44
assertThat(billingEvent.getCost()).isEqualTo(Money.of(USD, 204.44));
@ -1855,8 +1857,8 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
ImmutableMap.of("DOMAIN", "example.tld", "YEARS", "2"));
runFlowAssertResponse(
loadFile("domain_create_response.xml", ImmutableMap.of("DOMAIN", "example.tld")));
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.getCost()).isEqualTo(Money.of(USD, BigDecimal.valueOf(19.5)));
assertThat(billingEvent.getAllocationToken().get().getKey()).isEqualTo("abc123");
@ -1971,7 +1973,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
.setDefaultPromoTokens(
ImmutableList.of(defaultToken1.createVKey(), defaultToken2.createVKey()))
.build());
BillingEvent.OneTime billingEvent = runTest_defaultToken("bbbbb");
BillingEvent billingEvent = runTest_defaultToken("bbbbb");
assertThat(billingEvent.getCost()).isEqualTo(Money.of(USD, BigDecimal.valueOf(19.5)));
}
@ -2002,7 +2004,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
ImmutableList.of(defaultToken1.createVKey(), defaultToken2.createVKey()))
.build());
DatabaseHelper.deleteResource(defaultToken1);
BillingEvent.OneTime billingEvent = runTest_defaultToken("bbbbb");
BillingEvent billingEvent = runTest_defaultToken("bbbbb");
assertThat(billingEvent.getCost()).isEqualTo(Money.of(USD, BigDecimal.valueOf(19.5)));
}
@ -2041,7 +2043,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
doSuccessfulTest();
}
BillingEvent.OneTime runTest_defaultToken(String token) throws Exception {
BillingEvent runTest_defaultToken(String token) throws Exception {
setEppInput("domain_create.xml", ImmutableMap.of("DOMAIN", "example.tld"));
runFlowAssertResponse(
loadFile(
@ -2051,8 +2053,8 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
.put("CRDATE", "1999-04-03T22:00:00.0Z")
.put("EXDATE", "2001-04-03T22:00:00.0Z")
.build()));
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(token);
return billingEvent;
@ -3241,7 +3243,7 @@ class DomainCreateFlowTest extends ResourceFlowTestCase<DomainCreateFlow, Domain
.hasMessageThat()
.isEqualTo(
"No enum constant"
+ " google.registry.model.billing.BillingEvent.RenewalPriceBehavior.INVALID");
+ " google.registry.model.billing.BillingBase.RenewalPriceBehavior.INVALID");
}
@Test

View file

@ -76,9 +76,11 @@ import google.registry.flows.domain.DomainFlowUtils.BadCommandForRegistryPhaseEx
import google.registry.flows.domain.DomainFlowUtils.NotAuthorizedForTldException;
import google.registry.flows.exceptions.OnlyToolCanPassMetadataException;
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.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.BillingRecurrence;
import google.registry.model.contact.Contact;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
@ -136,7 +138,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
private void setUpSuccessfulTest() throws Exception {
createReferencedEntities(A_MONTH_FROM_NOW);
BillingEvent.Recurring autorenewBillingEvent =
BillingRecurrence autorenewBillingEvent =
persistResource(createAutorenewBillingEvent("TheRegistrar").build());
PollMessage.Autorenew autorenewPollMessage =
persistResource(createAutorenewPollMessage("TheRegistrar").build());
@ -193,7 +195,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
private void setUpAutorenewGracePeriod() throws Exception {
createReferencedEntities(A_MONTH_AGO.plusYears(1));
BillingEvent.Recurring autorenewBillingEvent =
BillingRecurrence autorenewBillingEvent =
persistResource(
createAutorenewBillingEvent("TheRegistrar").setEventTime(A_MONTH_AGO).build());
PollMessage.Autorenew autorenewPollMessage =
@ -205,7 +207,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
.asBuilder()
.setGracePeriods(
ImmutableSet.of(
GracePeriod.createForRecurring(
GracePeriod.createForRecurrence(
GracePeriodStatus.AUTO_RENEW,
domain.getRepoId(),
A_MONTH_AGO.plusDays(45),
@ -218,25 +220,23 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
}
private void assertAutorenewClosedAndCancellationCreatedFor(
BillingEvent.OneTime graceBillingEvent, DomainHistory historyEntryDomainDelete) {
BillingEvent graceBillingEvent, DomainHistory historyEntryDomainDelete) {
assertAutorenewClosedAndCancellationCreatedFor(
graceBillingEvent, historyEntryDomainDelete, clock.nowUtc());
}
private void assertAutorenewClosedAndCancellationCreatedFor(
BillingEvent.OneTime graceBillingEvent,
DomainHistory historyEntryDomainDelete,
DateTime eventTime) {
BillingEvent graceBillingEvent, DomainHistory historyEntryDomainDelete, DateTime eventTime) {
assertBillingEvents(
createAutorenewBillingEvent("TheRegistrar").setRecurrenceEndTime(eventTime).build(),
graceBillingEvent,
new BillingEvent.Cancellation.Builder()
new BillingCancellation.Builder()
.setReason(graceBillingEvent.getReason())
.setTargetId("example.tld")
.setRegistrarId("TheRegistrar")
.setEventTime(eventTime)
.setBillingTime(TIME_BEFORE_FLOW.plusDays(1))
.setOneTimeEventKey(graceBillingEvent.createVKey())
.setBillingEvent(graceBillingEvent.createVKey())
.setDomainHistory(historyEntryDomainDelete)
.build());
}
@ -248,8 +248,8 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
createAutorenewBillingEvent(registrarId).setRecurrenceEndTime(clock.nowUtc()).build());
}
private BillingEvent.OneTime createBillingEvent(Reason reason, Money cost) {
return new BillingEvent.OneTime.Builder()
private BillingEvent createBillingEvent(Reason reason, Money cost) {
return new BillingEvent.Builder()
.setReason(reason)
.setTargetId("example.tld")
.setRegistrarId("TheRegistrar")
@ -261,8 +261,8 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
.build();
}
private BillingEvent.Recurring.Builder createAutorenewBillingEvent(String registrarId) {
return new BillingEvent.Recurring.Builder()
private BillingRecurrence.Builder createAutorenewBillingEvent(String registrarId) {
return new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId("example.tld")
@ -344,7 +344,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
throws Exception {
// Persist the billing event so it can be retrieved for cancellation generation and checking.
setUpSuccessfulTest();
BillingEvent.OneTime graceBillingEvent =
BillingEvent graceBillingEvent =
persistResource(createBillingEvent(Reason.CREATE, Money.of(USD, 123)));
setUpGracePeriods(
GracePeriod.forBillingEvent(gracePeriodStatus, domain.getRepoId(), graceBillingEvent));
@ -418,7 +418,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
String responseFilename, Map<String, String> 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<DomainDeleteFlow, Domain
sessionMetadata.setServiceExtensionUris(ImmutableSet.of());
setUpSuccessfulTest();
// Persist the billing event so it can be retrieved for cancellation generation and checking.
BillingEvent.OneTime graceBillingEvent =
BillingEvent graceBillingEvent =
persistResource(createBillingEvent(Reason.CREATE, Money.of(USD, 123)));
// Use a grace period so that the delete is immediate, simplifying the assertions below.
setUpGracePeriods(
@ -1238,7 +1238,7 @@ class DomainDeleteFlowTest extends ResourceFlowTestCase<DomainDeleteFlow, Domain
void testSuccess_freeCreation_deletionDuringGracePeriod() throws Exception {
// Deletion during the add grace period should still work even if the credit is 0
setUpSuccessfulTest();
BillingEvent.OneTime graceBillingEvent =
BillingEvent graceBillingEvent =
persistResource(createBillingEvent(Reason.CREATE, Money.of(USD, 0)));
setUpGracePeriods(
GracePeriod.forBillingEvent(GracePeriodStatus.ADD, domain.getRepoId(), graceBillingEvent));

View file

@ -16,9 +16,9 @@ package google.registry.flows.domain;
import static com.google.common.io.BaseEncoding.base16;
import static com.google.common.truth.Truth.assertThat;
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.eppcommon.EppXmlTransformer.marshal;
import static google.registry.model.tld.Tld.TldState.QUIET_PERIOD;
import static google.registry.testing.DatabaseHelper.assertNoBillingEvents;
@ -51,10 +51,10 @@ import google.registry.flows.domain.DomainFlowUtils.CurrencyUnitMismatchExceptio
import google.registry.flows.domain.DomainFlowUtils.FeeChecksDontSupportPhasesException;
import google.registry.flows.domain.DomainFlowUtils.RestoresAreAlwaysForOneYearException;
import google.registry.flows.domain.DomainFlowUtils.TransfersAreAlwaysForOneYearException;
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.BillingBase.Flag;
import google.registry.model.billing.BillingBase.Reason;
import google.registry.model.billing.BillingBase.RenewalPriceBehavior;
import google.registry.model.billing.BillingRecurrence;
import google.registry.model.contact.Contact;
import google.registry.model.contact.ContactAuthInfo;
import google.registry.model.domain.DesignatedContact;
@ -384,9 +384,9 @@ class DomainInfoFlowTest extends ResourceFlowTestCase<DomainInfoFlow, Domain> {
.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<DomainInfoFlow, Domain> {
.setRecurrenceEndTime(END_OF_TIME)
.setDomainHistory(historyEntry)
.build());
VKey<Recurring> recurringVKey = renewEvent.createVKey();
VKey<BillingRecurrence> 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);
}

View file

@ -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<Money> 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()

View file

@ -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<DomainRenewFlow, Domain>
.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<DomainRenewFlow, Domain>
.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<DomainRenewFlow, Domain>
.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<DomainRenewFlow, Domain>
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<DomainRenewFlow, Domain>
"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<DomainRenewFlow, Domain>
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<DomainRenewFlow, Domain>
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<DomainRenewFlow, Domain>
"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<DomainRenewFlow, Domain>
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<DomainRenewFlow, Domain>
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<DomainRenewFlow, Domain>
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<DomainRenewFlow, Domain>
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<DomainRenewFlow, Domain>
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());

View file

@ -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<DomainRestoreReq
.setMsg("Domain was auto-renewed.")
.setHistoryEntry(historyEntryDomainRestore)
.build());
// There should be a onetime for the restore and a new recurring billing event, but no renew
// onetime.
// There should be a onetime for the restore and a new recurrence, but no renew onetime.
assertBillingEvents(
new BillingEvent.Recurring.Builder()
new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId("example.tld")
@ -212,7 +212,7 @@ class DomainRestoreRequestFlowTest extends ResourceFlowTestCase<DomainRestoreReq
.setRecurrenceEndTime(END_OF_TIME)
.setDomainHistory(historyEntryDomainRestore)
.build(),
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setReason(Reason.RESTORE)
.setTargetId("example.tld")
.setRegistrarId("TheRegistrar")
@ -269,10 +269,10 @@ class DomainRestoreRequestFlowTest extends ResourceFlowTestCase<DomainRestoreReq
.setMsg("Domain was auto-renewed.")
.setHistoryEntry(historyEntryDomainRestore)
.build());
// There should be a bill for the restore and an explicit renew, along with a new recurring
// There should be a bill for the restore and an explicit renew, along with a new recurrence
// autorenew event.
assertBillingEvents(
new BillingEvent.Recurring.Builder()
new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId("example.tld")
@ -281,7 +281,7 @@ class DomainRestoreRequestFlowTest extends ResourceFlowTestCase<DomainRestoreReq
.setRecurrenceEndTime(END_OF_TIME)
.setDomainHistory(historyEntryDomainRestore)
.build(),
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setReason(Reason.RESTORE)
.setTargetId("example.tld")
.setRegistrarId("TheRegistrar")
@ -291,7 +291,7 @@ class DomainRestoreRequestFlowTest extends ResourceFlowTestCase<DomainRestoreReq
.setBillingTime(clock.nowUtc())
.setDomainHistory(historyEntryDomainRestore)
.build(),
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setReason(Reason.RENEW)
.setTargetId("example.tld")
.setRegistrarId("TheRegistrar")

View file

@ -60,11 +60,12 @@ import google.registry.flows.domain.token.AllocationTokenFlowUtils.AllocationTok
import google.registry.flows.domain.token.AllocationTokenFlowUtils.AlreadyRedeemedAllocationTokenException;
import google.registry.flows.domain.token.AllocationTokenFlowUtils.InvalidAllocationTokenException;
import google.registry.flows.exceptions.NotPendingTransferException;
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.OneTime;
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.contact.ContactAuthInfo;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainAuthInfo;
@ -170,7 +171,7 @@ class DomainTransferApproveFlowTest
String expectedXmlFilename,
DateTime expectedExpirationTime,
int expectedYearsToCharge,
BillingEvent.Cancellation.Builder... expectedCancellationBillingEvents)
BillingCancellation.Builder... expectedCancellationBillingEvents)
throws Exception {
runSuccessfulFlowWithAssertions(
tld, commandFilename, expectedXmlFilename, expectedExpirationTime);
@ -269,7 +270,7 @@ class DomainTransferApproveFlowTest
private void assertHistoryEntriesContainBillingEventsAndGracePeriods(
String tld,
int expectedYearsToCharge,
BillingEvent.Cancellation.Builder... expectedCancellationBillingEvents)
BillingCancellation.Builder... expectedCancellationBillingEvents)
throws Exception {
Tld registry = Tld.get(tld);
domain = reloadResourceByForeignKey();
@ -277,8 +278,8 @@ class DomainTransferApproveFlowTest
getOnlyHistoryEntryOfType(domain, DOMAIN_TRANSFER_APPROVE, DomainHistory.class);
// We expect three billing events: one for the transfer, a closed autorenew for the losing
// client and an open autorenew for the gaining client that begins at the new expiration time.
OneTime transferBillingEvent =
new BillingEvent.OneTime.Builder()
BillingEvent transferBillingEvent =
new BillingEvent.Builder()
.setReason(Reason.TRANSFER)
.setTargetId(domain.getDomainName())
.setEventTime(clock.nowUtc())
@ -306,7 +307,7 @@ class DomainTransferApproveFlowTest
domain.getRegistrationExpirationTime().minusYears(1))
.setDomainHistory(historyEntryTransferApproved)
.build()))
.toArray(BillingEvent[]::new));
.toArray(BillingBase[]::new));
// There should be a grace period for the new transfer billing event.
assertGracePeriods(
domain.getGracePeriods(),
@ -321,7 +322,7 @@ class DomainTransferApproveFlowTest
}
private void assertHistoryEntriesDoNotContainTransferBillingEventsOrGracePeriods(
BillingEvent.Cancellation.Builder... expectedCancellationBillingEvents) throws Exception {
BillingCancellation.Builder... expectedCancellationBillingEvents) throws Exception {
domain = reloadResourceByForeignKey();
final DomainHistory historyEntryTransferApproved =
getOnlyHistoryEntryOfType(domain, DOMAIN_TRANSFER_APPROVE, DomainHistory.class);
@ -344,7 +345,7 @@ class DomainTransferApproveFlowTest
domain.getRegistrationExpirationTime().minusYears(1))
.setDomainHistory(historyEntryTransferApproved)
.build()))
.toArray(BillingEvent[]::new));
.toArray(BillingBase[]::new));
// There should be no grace period.
assertGracePeriods(domain.getGracePeriods(), ImmutableMap.of());
}
@ -437,7 +438,7 @@ class DomainTransferApproveFlowTest
getOnlyHistoryEntryOfType(domain, DOMAIN_TRANSFER_APPROVE, DomainHistory.class);
assertBillingEventsForResource(
domain,
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setBillingTime(now.plusDays(5))
.setEventTime(now)
.setRegistrarId("NewRegistrar")
@ -510,13 +511,13 @@ class DomainTransferApproveFlowTest
oldExpirationTime.plusYears(1),
1,
// Expect the grace period for autorenew to be cancelled.
new BillingEvent.Cancellation.Builder()
new BillingCancellation.Builder()
.setReason(Reason.RENEW)
.setTargetId("example.tld")
.setRegistrarId("TheRegistrar")
.setEventTime(clock.nowUtc()) // The cancellation happens at the moment of transfer.
.setBillingTime(oldExpirationTime.plus(Tld.get("tld").getAutoRenewGracePeriodLength()))
.setRecurringEventKey(domain.getAutorenewBillingEvent()));
.setBillingRecurrence(domain.getAutorenewBillingEvent()));
}
@Test
@ -543,7 +544,7 @@ class DomainTransferApproveFlowTest
getOnlyHistoryEntryOfType(domain, DOMAIN_TRANSFER_APPROVE, DomainHistory.class);
assertBillingEventsForResource(
domain,
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setBillingTime(now.plusDays(5))
.setEventTime(now)
.setRegistrarId("NewRegistrar")
@ -590,7 +591,7 @@ class DomainTransferApproveFlowTest
getOnlyHistoryEntryOfType(domain, DOMAIN_TRANSFER_APPROVE, DomainHistory.class);
assertBillingEventsForResource(
domain,
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setBillingTime(now.plusDays(5))
.setEventTime(now)
.setRegistrarId("NewRegistrar")
@ -845,7 +846,7 @@ class DomainTransferApproveFlowTest
@Test
void testSuccess_superuserExtension_transferPeriodZero_autorenewGraceActive() throws Exception {
Domain domain = reloadResourceByForeignKey();
VKey<Recurring> existingAutorenewEvent = domain.getAutorenewBillingEvent();
VKey<BillingRecurrence> 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()),

View file

@ -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<F extends Flow, R extends EppResource>
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<F extends Flow, R extends EppResource>
}
/** 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<F extends Flow, R extends EppResource>
}
/** 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())

View file

@ -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<Money> transferCost,
ImmutableSet<GracePeriod> 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<BillingEvent.OneTime> optionalTransferBillingEvent;
Optional<BillingEvent> 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<BillingEvent> extraBillingEvents =
ImmutableSet<BillingBase> 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<BillingEvent> expectedBillingEvents =
ImmutableSet<BillingBase> 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<BillingEvent> expectedServeApproveBillingEvents =
ImmutableSet<BillingBase> 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<String, String> substitutions,
Optional<Money> transferCost,
BillingEvent.Cancellation.Builder... extraExpectedBillingEvents)
BillingCancellation.Builder... extraExpectedBillingEvents)
throws Exception {
setEppInput(commandFilename, substitutions);
ImmutableSet<GracePeriod> 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<Money> 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<BillingEvent.Recurring> existingAutorenewEvent = domain.getAutorenewBillingEvent();
VKey<BillingRecurrence> 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<BillingEvent.Recurring> existingAutorenewEvent = domain.getAutorenewBillingEvent();
VKey<BillingRecurrence> 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<BillingEvent.Recurring> existingAutorenewEvent = domain.getAutorenewBillingEvent();
VKey<BillingRecurrence> 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")

View file

@ -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<DomainUpdateFlow, Domain
if (isBillable) {
assertBillingEvents(
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setReason(Reason.SERVER_STATUS)
.setTargetId("example.tld")
.setRegistrarId("TheRegistrar")

View file

@ -31,9 +31,9 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedMap;
import google.registry.model.EntityTestCase;
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.BillingBase.Flag;
import google.registry.model.billing.BillingBase.Reason;
import google.registry.model.billing.BillingBase.RenewalPriceBehavior;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.GracePeriod;
@ -49,22 +49,22 @@ import org.joda.time.DateTime;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
/** Unit tests for {@link BillingEvent}. */
public class BillingEventTest extends EntityTestCase {
/** Unit tests for {@link BillingBase}. */
public class BillingBaseTest extends EntityTestCase {
private final DateTime now = DateTime.now(UTC);
BillingEventTest() {
BillingBaseTest() {
super(JpaEntityCoverageCheck.ENABLED);
}
private DomainHistory domainHistory;
private DomainHistory domainHistory2;
private Domain domain;
private BillingEvent.OneTime oneTime;
private BillingEvent.OneTime oneTimeSynthetic;
private BillingEvent.Recurring recurring;
private BillingEvent.Cancellation cancellationOneTime;
private BillingEvent.Cancellation cancellationRecurring;
private BillingEvent billingEvent;
private BillingEvent billingEventSynthetic;
private BillingRecurrence billingRecurrence;
private BillingCancellation cancellationOneTime;
private BillingCancellation cancellationRecurrence;
@BeforeEach
void setUp() {
@ -105,39 +105,38 @@ public class BillingEventTest extends EntityTestCase {
.build())
.build());
oneTime =
billingEvent =
persistResource(
commonInit(
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setDomainHistory(domainHistory)
.setReason(Reason.CREATE)
.setFlags(ImmutableSet.of(BillingEvent.Flag.ANCHOR_TENANT))
.setFlags(ImmutableSet.of(BillingBase.Flag.ANCHOR_TENANT))
.setPeriodYears(2)
.setCost(Money.of(USD, 1))
.setEventTime(now)
.setBillingTime(now.plusDays(5))
.setAllocationToken(allocationToken.createVKey())));
recurring =
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))
.setRecurrenceEndTime(END_OF_TIME)));
oneTimeSynthetic =
billingEventSynthetic =
persistResource(
commonInit(
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setDomainHistory(domainHistory)
.setReason(Reason.CREATE)
.setFlags(
ImmutableSet.of(
BillingEvent.Flag.ANCHOR_TENANT, BillingEvent.Flag.SYNTHETIC))
ImmutableSet.of(BillingBase.Flag.ANCHOR_TENANT, BillingBase.Flag.SYNTHETIC))
.setSyntheticCreationTime(now.plusDays(10))
.setCancellationMatchingBillingEvent(recurring)
.setCancellationMatchingBillingEvent(billingRecurrence)
.setPeriodYears(2)
.setCost(Money.of(USD, 1))
.setEventTime(now)
@ -146,56 +145,57 @@ public class BillingEventTest extends EntityTestCase {
cancellationOneTime =
persistResource(
commonInit(
new BillingEvent.Cancellation.Builder()
new BillingCancellation.Builder()
.setDomainHistory(domainHistory2)
.setReason(Reason.CREATE)
.setEventTime(now.plusDays(1))
.setBillingTime(now.plusDays(5))
.setOneTimeEventKey(oneTime.createVKey())));
.setBillingEvent(billingEvent.createVKey())));
cancellationRecurring =
cancellationRecurrence =
persistResource(
commonInit(
new BillingEvent.Cancellation.Builder()
new BillingCancellation.Builder()
.setDomainHistory(domainHistory2)
.setReason(Reason.RENEW)
.setEventTime(now.plusDays(1))
.setBillingTime(now.plusYears(1).plusDays(45))
.setRecurringEventKey(recurring.createVKey())));
.setBillingRecurrence(billingRecurrence.createVKey())));
}
private static <E extends BillingEvent, B extends BillingEvent.Builder<E, B>> E commonInit(
private static <E extends BillingBase, B extends BillingBase.Builder<E, B>> 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)

View file

@ -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;

View file

@ -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<BillingEvent.OneTime> oneTimeBillKey;
private VKey<BillingEvent.Recurring> recurringBillKey;
private VKey<BillingEvent> oneTimeBillKey;
private VKey<BillingRecurrence> recurrenceBillKey;
private DomainHistory domainHistory;
private VKey<Contact> 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<PollMessage.Autorenew> autorenewPollKey = VKey.create(PollMessage.Autorenew.class, 3L);
VKey<PollMessage.OneTime> 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<BillingEvent.Recurring> newAutorenewEvent) {
Domain domain, DateTime newExpirationTime, VKey<BillingRecurrence> 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<BillingEvent.Recurring> serverApproveAutorenewEvent =
VKey<BillingRecurrence> 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<BillEventInfo> 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<BillingEvent.Recurring> billingEventRecurring;
Long billingEventRecurringHistoryId;
VKey<BillingEvent.OneTime> billingEventOneTime;
VKey<BillingRecurrence> recurrence;
Long recurrenceHistoryId;
VKey<BillingEvent> billingEventOneTime;
Long billingEventOneTimeHistoryId;
BillEventInfo(
VKey<BillingEvent.Recurring> billingEventRecurring,
VKey<BillingEvent.OneTime> billingEventOneTime) {
this.billingEventRecurring = billingEventRecurring;
BillEventInfo(VKey<BillingRecurrence> recurrence, VKey<BillingEvent> billingEventOneTime) {
this.recurrence = recurrence;
this.billingEventOneTime = billingEventOneTime;
}
}

View file

@ -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<BillingEvent.Recurring> recurringKey;
private BillingEvent onetime;
private VKey<BillingRecurrence> 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");
}
}

View file

@ -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;

View file

@ -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;

View file

@ -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<BillingEvent.OneTime> transferBillingEventKey;
private VKey<BillingEvent.Cancellation> otherServerApproveBillingEventKey;
private VKey<BillingEvent.Recurring> recurringBillingEventKey;
private VKey<BillingEvent> transferBillingEventKey;
private VKey<BillingCancellation> otherServerApproveBillingEventKey;
private VKey<BillingRecurrence> recurrenceKey;
private VKey<PollMessage.Autorenew> autorenewPollMessageKey;
private VKey<PollMessage.OneTime> 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

View file

@ -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")

View file

@ -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)

View file

@ -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,

View file

@ -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<BillingEvent> billingEvents = getBillingEvents(domain);
Iterable<BillingBase> billingEvents = getBillingEvents(domain);
Iterable<? extends HistoryEntry> historyEntries =
HistoryEntryDao.loadHistoryObjectsForResource(domain.createVKey());
Iterable<PollMessage> 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<BillingEvent> getBillingEvents() {
public static Iterable<BillingBase> 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<BillingEvent> getBillingEvents(EppResource resource) {
private static Iterable<BillingBase> 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<BillingEvent> actual, Iterable<BillingEvent> expected) {
Iterable<BillingBase> actual, Iterable<BillingBase> 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<BillingEvent> expected) {
public static void assertBillingEvents(Set<BillingBase> 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 {
*
* <p>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. */

View file

@ -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;

View file

@ -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<DomainHistory> historyEntries) {
Set<BillingEvent> expectedEvents =
Set<BillingBase> expectedEvents =
historyEntries.stream()
.map(
entry ->
new BillingEvent.OneTime.Builder()
new BillingEvent.Builder()
.setReason(Reason.SERVER_STATUS)
.setTargetId(domain.getForeignKey())
.setRegistrarId(domain.getCurrentSponsorRegistrarId())

View file

@ -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"),

View file

@ -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;

View file

@ -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;

View file

@ -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<UnrenewDomainComma
assertBillingEventsEqual(
loadByKey(domain.getAutorenewBillingEvent()),
new BillingEvent.Recurring.Builder()
new BillingRecurrence.Builder()
.setDomainHistory(synthetic)
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))

View file

@ -15,9 +15,9 @@
package google.registry.tools;
import static com.google.common.truth.Truth.assertThat;
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.TokenStatus.CANCELLED;
import static google.registry.model.domain.token.AllocationToken.TokenStatus.ENDED;
import static google.registry.model.domain.token.AllocationToken.TokenStatus.NOT_STARTED;

View file

@ -28,9 +28,9 @@ import static google.registry.util.DateTimeUtils.END_OF_TIME;
import static org.junit.jupiter.api.Assertions.assertThrows;
import com.google.common.collect.Iterables;
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.reporting.HistoryEntry;
@ -42,7 +42,7 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
/** Tests for {@link UpdateRecurrenceCommand}. */
public class UpdateRecurrenceCommandTest extends CommandTestCase<UpdateRecurrenceCommand> {
public class UpdateBillingRecurrenceCommandTest extends CommandTestCase<UpdateRecurrenceCommand> {
@BeforeEach
void beforeEach() {
@ -52,19 +52,21 @@ public class UpdateRecurrenceCommandTest extends CommandTestCase<UpdateRecurrenc
@Test
void testSuccess_setsSpecified() throws Exception {
persistDomain();
Recurring existingRecurring = Iterables.getOnlyElement(loadAllOf(Recurring.class));
assertThat(existingRecurring.getRecurrenceEndTime()).isEqualTo(END_OF_TIME);
assertThat(existingRecurring.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT);
BillingRecurrence existingBillingRecurrence =
Iterables.getOnlyElement(loadAllOf(BillingRecurrence.class));
assertThat(existingBillingRecurrence.getRecurrenceEndTime()).isEqualTo(END_OF_TIME);
assertThat(existingBillingRecurrence.getRenewalPriceBehavior())
.isEqualTo(RenewalPriceBehavior.DEFAULT);
runCommandForced(
"domain.tld",
"--renewal_price_behavior",
"SPECIFIED",
"--specified_renewal_price",
"USD 9001");
assertThat(loadByEntity(existingRecurring).getRecurrenceEndTime())
assertThat(loadByEntity(existingBillingRecurrence).getRecurrenceEndTime())
.isEqualTo(fakeClock.nowUtc());
assertNewBillingEventAndHistory(
existingRecurring.getId(),
existingBillingRecurrence.getId(),
RenewalPriceBehavior.SPECIFIED,
Money.of(CurrencyUnit.USD, 9001));
}
@ -72,46 +74,52 @@ public class UpdateRecurrenceCommandTest extends CommandTestCase<UpdateRecurrenc
@Test
void testSuccess_setsNonPremium() throws Exception {
persistDomain();
Recurring existingRecurring = Iterables.getOnlyElement(loadAllOf(Recurring.class));
assertThat(existingRecurring.getRecurrenceEndTime()).isEqualTo(END_OF_TIME);
assertThat(existingRecurring.getRenewalPriceBehavior()).isEqualTo(RenewalPriceBehavior.DEFAULT);
BillingRecurrence existingBillingRecurrence =
Iterables.getOnlyElement(loadAllOf(BillingRecurrence.class));
assertThat(existingBillingRecurrence.getRecurrenceEndTime()).isEqualTo(END_OF_TIME);
assertThat(existingBillingRecurrence.getRenewalPriceBehavior())
.isEqualTo(RenewalPriceBehavior.DEFAULT);
runCommandForced("domain.tld", "--renewal_price_behavior", "NONPREMIUM");
assertThat(loadByEntity(existingRecurring).getRecurrenceEndTime())
assertThat(loadByEntity(existingBillingRecurrence).getRecurrenceEndTime())
.isEqualTo(fakeClock.nowUtc());
assertNewBillingEventAndHistory(
existingRecurring.getId(), RenewalPriceBehavior.NONPREMIUM, null);
existingBillingRecurrence.getId(), RenewalPriceBehavior.NONPREMIUM, null);
}
@Test
void testSuccess_setsDefault() throws Exception {
persistDomain();
Recurring existingRecurring = Iterables.getOnlyElement(loadAllOf(Recurring.class));
BillingRecurrence existingBillingRecurrence =
Iterables.getOnlyElement(loadAllOf(BillingRecurrence.class));
persistResource(
existingRecurring
existingBillingRecurrence
.asBuilder()
.setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED)
.setRenewalPrice(Money.of(CurrencyUnit.USD, 100))
.build());
assertThat(existingRecurring.getRecurrenceEndTime()).isEqualTo(END_OF_TIME);
assertThat(existingBillingRecurrence.getRecurrenceEndTime()).isEqualTo(END_OF_TIME);
runCommandForced("domain.tld", "--renewal_price_behavior", "DEFAULT");
assertThat(loadByEntity(existingRecurring).getRecurrenceEndTime())
assertThat(loadByEntity(existingBillingRecurrence).getRecurrenceEndTime())
.isEqualTo(fakeClock.nowUtc());
assertNewBillingEventAndHistory(existingRecurring.getId(), RenewalPriceBehavior.DEFAULT, null);
assertNewBillingEventAndHistory(
existingBillingRecurrence.getId(), RenewalPriceBehavior.DEFAULT, null);
}
@Test
void testSuccess_setsPrice_whenSpecifiedAlready() throws Exception {
Domain domain = persistDomain();
Recurring recurring = loadByKey(domain.getAutorenewBillingEvent());
BillingRecurrence billingRecurrence = loadByKey(domain.getAutorenewBillingEvent());
persistResource(
recurring
billingRecurrence
.asBuilder()
.setRenewalPrice(Money.of(CurrencyUnit.USD, 20))
.setRenewalPriceBehavior(RenewalPriceBehavior.SPECIFIED)
.build());
runCommandForced("domain.tld", "--specified_renewal_price", "USD 9001");
assertNewBillingEventAndHistory(
recurring.getId(), RenewalPriceBehavior.SPECIFIED, Money.of(CurrencyUnit.USD, 9001));
billingRecurrence.getId(),
RenewalPriceBehavior.SPECIFIED,
Money.of(CurrencyUnit.USD, 9001));
}
@Test
@ -163,8 +171,8 @@ public class UpdateRecurrenceCommandTest extends CommandTestCase<UpdateRecurrenc
@Test
void testFailure_billingAlreadyClosed() {
Domain domain = persistDomain();
Recurring recurring = loadByKey(domain.getAutorenewBillingEvent());
persistResource(recurring.asBuilder().setRecurrenceEndTime(fakeClock.nowUtc()).build());
BillingRecurrence billingRecurrence = loadByKey(domain.getAutorenewBillingEvent());
persistResource(billingRecurrence.asBuilder().setRecurrenceEndTime(fakeClock.nowUtc()).build());
assertThat(
assertThrows(
IllegalArgumentException.class,
@ -190,12 +198,16 @@ public class UpdateRecurrenceCommandTest extends CommandTestCase<UpdateRecurrenc
private void assertNewBillingEventAndHistory(
long previousId, RenewalPriceBehavior expectedBehavior, @Nullable Money expectedPrice) {
Recurring newRecurring =
loadAllOf(Recurring.class).stream().filter(r -> 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()

View file

@ -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<UpdateDomainCommand
.setDomain(domain)
.setRegistrarId(domain.getCreationRegistrarId())
.build());
BillingEvent.Recurring autorenewBillingEvent =
BillingRecurrence autorenewBillingEvent =
persistResource(
new BillingEvent.Recurring.Builder()
new BillingRecurrence.Builder()
.setReason(Reason.RENEW)
.setFlags(ImmutableSet.of(Flag.AUTO_RENEW))
.setTargetId("example.tld")
@ -342,7 +342,7 @@ class UpdateDomainCommandTest extends EppToolCommandTestCase<UpdateDomainCommand
.setAutorenewBillingEvent(autorenewBillingEvent.createVKey())
.setGracePeriods(
ImmutableSet.of(
GracePeriod.createForRecurring(
GracePeriod.createForRecurrence(
AUTO_RENEW,
domain.getRepoId(),
fakeClock.nowUtc().plusDays(40),

View file

@ -20,7 +20,7 @@ import static google.registry.testing.DatabaseHelper.persistResource;
import static org.junit.jupiter.api.Assertions.assertThrows;
import com.google.common.truth.Truth;
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;

View file

@ -24,9 +24,9 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
import com.beust.jcommander.ParameterException;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import google.registry.model.billing.BillingEvent.Cancellation;
import google.registry.model.billing.BillingEvent.OneTime;
import google.registry.model.billing.BillingEvent.Reason;
import google.registry.model.billing.BillingBase.Reason;
import google.registry.model.billing.BillingCancellation;
import google.registry.model.billing.BillingEvent;
import google.registry.model.contact.Contact;
import google.registry.model.domain.Domain;
import google.registry.model.domain.DomainHistory;
@ -39,12 +39,12 @@ import org.joda.money.Money;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
/** Tests for {@link CreateCancellationsForOneTimesCommand}. */
public class CreateCancellationsForOneTimesCommandTest
extends CommandTestCase<CreateCancellationsForOneTimesCommand> {
/** Tests for {@link CreateCancellationsForBillingEventsCommand}. */
public class CreateCancellationsForBillingEventsCommandTest
extends CommandTestCase<CreateCancellationsForBillingEventsCommand> {
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();
}
}

View file

@ -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())

View file

@ -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

View file

@ -261,7 +261,7 @@ td.section {
</tr>
<tr>
<td class="property_name">generated on</td>
<td class="property_value">2023-03-28 19:28:38.164363</td>
<td class="property_value">2023-04-21 18:52:34.656577</td>
</tr>
<tr>
<td class="property_name">last flyway file</td>
@ -284,7 +284,7 @@ td.section {
generated on
</text>
<text text-anchor="start" x="3835.5" y="-10.8" font-family="Helvetica,sans-Serif" font-size="14.00">
2023-03-28 19:28:38.164363
2023-04-21 18:52:34.656577
</text>
<polygon fill="none" stroke="#888888" points="3748,-4 3748,-44 4013,-44 4013,-4 3748,-4" /> <!-- allocationtoken_a08ccbef -->
<g id="node1" class="node">

View file

@ -261,7 +261,7 @@ td.section {
</tr>
<tr>
<td class="property_name">generated on</td>
<td class="property_value">2023-03-28 19:28:35.770605</td>
<td class="property_value">2023-04-21 18:52:32.303901</td>
</tr>
<tr>
<td class="property_name">last flyway file</td>
@ -284,7 +284,7 @@ td.section {
generated on
</text>
<text text-anchor="start" x="4519.5" y="-10.8" font-family="Helvetica,sans-Serif" font-size="14.00">
2023-03-28 19:28:35.770605
2023-04-21 18:52:32.303901
</text>
<polygon fill="none" stroke="#888888" points="4432,-4 4432,-44 4697,-44 4697,-4 4432,-4" /> <!-- allocationtoken_a08ccbef -->
<g id="node1" class="node">

View file

@ -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)
);

View file

@ -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<HistoryEntry>` to identify the
mutation that spawned the `BillingEvent`.
All `BillingBase` entities contain a parent `VKey<HistoryEntry>` 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).

View file

@ -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 \