mirror of
https://github.com/google/nomulus
synced 2026-04-21 08:40:44 +00:00
Remove Duplicate billing events from the invoicing pipeline (#2326)
The Distinct transform removes duplicates based on the serialized format of the elements. By providing a deterministic coder, we can guarantee that no duplicates exist.
This commit is contained in:
@@ -16,12 +16,14 @@ package google.registry.beam.billing;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
|
||||
import google.registry.beam.billing.BillingEvent.BillingEventCoder;
|
||||
import google.registry.beam.billing.BillingEvent.InvoiceGroupingKey;
|
||||
import google.registry.beam.billing.BillingEvent.InvoiceGroupingKey.InvoiceGroupingKeyCoder;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import org.apache.beam.sdk.coders.NullableCoder;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
@@ -120,13 +122,22 @@ class BillingEventTest {
|
||||
@Test
|
||||
void testInvoiceGroupingKeyCoder_deterministicSerialization() throws IOException {
|
||||
InvoiceGroupingKey invoiceKey = event.getInvoiceGroupingKey();
|
||||
InvoiceGroupingKeyCoder coder = new InvoiceGroupingKeyCoder();
|
||||
InvoiceGroupingKeyCoder coder = InvoiceGroupingKeyCoder.of();
|
||||
ByteArrayOutputStream outStream = new ByteArrayOutputStream();
|
||||
coder.encode(invoiceKey, outStream);
|
||||
InputStream inStream = new ByteArrayInputStream(outStream.toByteArray());
|
||||
assertThat(coder.decode(inStream)).isEqualTo(invoiceKey);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testBillingEventCoder_deterministicSerialization() throws IOException {
|
||||
NullableCoder<BillingEvent> coder = BillingEventCoder.ofNullable();
|
||||
ByteArrayOutputStream outStream = new ByteArrayOutputStream();
|
||||
coder.encode(event, outStream);
|
||||
InputStream inStream = new ByteArrayInputStream(outStream.toByteArray());
|
||||
assertThat(coder.decode(inStream)).isEqualTo(event);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGetDetailReportHeader() {
|
||||
assertThat(BillingEvent.getHeader())
|
||||
|
||||
@@ -36,6 +36,7 @@ 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.beam.billing.BillingEvent.BillingEventCoder;
|
||||
import google.registry.model.billing.BillingBase.Flag;
|
||||
import google.registry.model.billing.BillingBase.Reason;
|
||||
import google.registry.model.billing.BillingCancellation;
|
||||
@@ -51,13 +52,13 @@ import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationT
|
||||
import google.registry.testing.FakeClock;
|
||||
import google.registry.util.ResourceUtils;
|
||||
import java.io.File;
|
||||
import java.io.Serial;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.logging.Logger;
|
||||
import org.apache.beam.sdk.coders.SerializableCoder;
|
||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||
import org.apache.beam.sdk.testing.PAssert;
|
||||
import org.apache.beam.sdk.transforms.Create;
|
||||
@@ -251,9 +252,7 @@ class InvoicingPipelineTest {
|
||||
options.setYearMonth(YEAR_MONTH);
|
||||
options.setInvoiceFilePrefix(INVOICE_FILE_PREFIX);
|
||||
billingEvents =
|
||||
pipeline.apply(
|
||||
Create.of(INPUT_EVENTS)
|
||||
.withCoder(SerializableCoder.of(google.registry.beam.billing.BillingEvent.class)));
|
||||
pipeline.apply(Create.of(INPUT_EVENTS).withCoder(BillingEventCoder.ofNullable()));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -346,26 +345,26 @@ class InvoicingPipelineTest {
|
||||
|
||||
@Test
|
||||
void testSuccess_makeCloudSqlQuery() throws Exception {
|
||||
// Pipeline must be run due to the TestPipelineExtension
|
||||
// The Pipeline must run due to TestPipelineExtension's checks.
|
||||
pipeline.run().waitUntilFinish();
|
||||
// Test that comments are removed from the .sql file correctly
|
||||
assertThat(InvoicingPipeline.makeCloudSqlQuery("2017-10"))
|
||||
.isEqualTo(
|
||||
'\n'
|
||||
+ "SELECT b, r FROM BillingEvent b\n"
|
||||
+ "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.billingEvent\n"
|
||||
+ "LEFT JOIN BillingCancellation cr ON b.cancellationMatchingBillingEvent ="
|
||||
+ " cr.billingRecurrence\n"
|
||||
+ "WHERE r.billingAccountMap IS NOT NULL\n"
|
||||
+ "AND r.type = 'REAL'\n"
|
||||
+ "AND t.invoicingEnabled IS TRUE\n"
|
||||
+ "AND b.billingTime BETWEEN CAST('2017-10-01' AS timestamp) AND CAST('2017-11-01'"
|
||||
+ " AS timestamp)\n"
|
||||
+ "AND c.id IS NULL\n"
|
||||
+ "AND cr.id IS NULL\n");
|
||||
"""
|
||||
|
||||
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.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
|
||||
AND b.billingTime BETWEEN CAST('2017-10-01' AS timestamp) AND CAST('2017-11-01' AS timestamp)
|
||||
AND c.id IS NULL
|
||||
AND cr.id IS NULL
|
||||
""");
|
||||
}
|
||||
|
||||
/** Returns the text contents of a file under the beamBucket/results directory. */
|
||||
@@ -604,31 +603,33 @@ class InvoicingPipelineTest {
|
||||
PCollection<google.registry.beam.billing.BillingEvent>,
|
||||
PCollection<google.registry.beam.billing.BillingEvent>> {
|
||||
|
||||
private static final long serialVersionUID = 2695033474967615250L;
|
||||
@Serial private static final long serialVersionUID = 2695033474967615250L;
|
||||
|
||||
@Override
|
||||
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(google.registry.beam.billing.BillingEvent.class))
|
||||
.via(
|
||||
billingEvent ->
|
||||
google.registry.beam.billing.BillingEvent.create(
|
||||
billingEvent.id(),
|
||||
billingEvent.billingTime(),
|
||||
billingEvent.eventTime(),
|
||||
billingEvent.registrarId(),
|
||||
billingEvent.billingId(),
|
||||
billingEvent.poNumber(),
|
||||
billingEvent.tld(),
|
||||
billingEvent.action(),
|
||||
billingEvent.domain(),
|
||||
"REPO-ID",
|
||||
billingEvent.years(),
|
||||
billingEvent.currency(),
|
||||
billingEvent.amount(),
|
||||
billingEvent.flags())));
|
||||
return input
|
||||
.apply(
|
||||
"Map to invoicing key",
|
||||
MapElements.into(TypeDescriptor.of(google.registry.beam.billing.BillingEvent.class))
|
||||
.via(
|
||||
billingEvent ->
|
||||
google.registry.beam.billing.BillingEvent.create(
|
||||
billingEvent.id(),
|
||||
billingEvent.billingTime(),
|
||||
billingEvent.eventTime(),
|
||||
billingEvent.registrarId(),
|
||||
billingEvent.billingId(),
|
||||
billingEvent.poNumber(),
|
||||
billingEvent.tld(),
|
||||
billingEvent.action(),
|
||||
billingEvent.domain(),
|
||||
"REPO-ID",
|
||||
billingEvent.years(),
|
||||
billingEvent.currency(),
|
||||
billingEvent.amount(),
|
||||
billingEvent.flags())))
|
||||
.setCoder(BillingEventCoder.ofNullable());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user