mirror of
https://github.com/google/nomulus.git
synced 2025-04-29 19:47:51 +02:00
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:
parent
75acd574cc
commit
668a7a47a2
97 changed files with 2086 additions and 1946 deletions
|
@ -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 :
|
||||
[
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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");
|
|
@ -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. */
|
||||
|
|
|
@ -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
|
|
@ -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")
|
|
@ -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. */
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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. -->
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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())));
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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"))
|
||||
|
|
|
@ -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);
|
|
@ -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 {}
|
||||
}
|
|
@ -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(),
|
||||
|
|
|
@ -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.
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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)
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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. */
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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"),
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()
|
|
@ -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),
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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())
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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">
|
||||
|
|
|
@ -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">
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
|
||||
|
|
|
@ -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).
|
||||
|
|
|
@ -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 \
|
||||
|
|
Loading…
Add table
Reference in a new issue