1
0
mirror of https://github.com/google/nomulus synced 2026-04-25 18:51:40 +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:
Lai Jiang
2024-02-16 15:43:40 -05:00
committed by GitHub
parent 7d2330c943
commit 08bcf579a5
4 changed files with 132 additions and 71 deletions

View File

@@ -21,26 +21,21 @@ import google.registry.reporting.billing.BillingModule;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.Serializable;
import java.util.regex.Pattern;
import org.apache.beam.sdk.coders.AtomicCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.DoubleCoder;
import org.apache.beam.sdk.coders.NullableCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.coders.VarLongCoder;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
/**
* A POJO representing a single billable event, parsed from a {@code SchemaAndRecord}.
*
* <p>This is a trivially serializable class that allows Beam to transform the results of a Cloud
* SQL query into a standard Java representation, giving us the type guarantees and ease of
* manipulation Cloud SQL lacks.
*/
/** A POJO representing a single billable event, parsed from a {@code SchemaAndRecord}. */
@AutoValue
public abstract class BillingEvent implements Serializable {
private static final long serialVersionUID = -3593088371541450077L;
public abstract class BillingEvent {
private static final DateTimeFormatter DATE_TIME_FORMATTER =
DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss zzz");
@@ -85,7 +80,7 @@ public abstract class BillingEvent implements Serializable {
/** Returns the tld this event was generated for. */
abstract String tld();
/** Returns the billable action this event was generated for (i.e. RENEW, CREATE, TRANSFER...) */
/** Returns the billable action this event was generated for (i.e., RENEW, CREATE, TRANSFER...) */
abstract String action();
/** Returns the fully qualified domain name this event was generated for. */
@@ -97,7 +92,7 @@ public abstract class BillingEvent implements Serializable {
/** Returns the number of years this billing event is made out for. */
abstract int years();
/** Returns the 3-letter currency code for the billing event (i.e. USD or JPY.) */
/** Returns the 3-letter currency code for the billing event (i.e., USD or JPY.) */
abstract String currency();
/** Returns the cost associated with this billing event. */
@@ -203,9 +198,7 @@ public abstract class BillingEvent implements Serializable {
/** Key for each {@code BillingEvent}, when aggregating for the overall invoice. */
@AutoValue
abstract static class InvoiceGroupingKey implements Serializable {
private static final long serialVersionUID = -151561764235256205L;
abstract static class InvoiceGroupingKey {
private static final ImmutableList<String> INVOICE_HEADERS =
ImmutableList.of(
@@ -277,8 +270,14 @@ public abstract class BillingEvent implements Serializable {
/** Coder that provides deterministic (de)serialization for {@code InvoiceGroupingKey}. */
static class InvoiceGroupingKeyCoder extends AtomicCoder<InvoiceGroupingKey> {
private static final Coder<String> stringCoder = StringUtf8Coder.of();
private static final InvoiceGroupingKeyCoder INSTANCE = new InvoiceGroupingKeyCoder();
private static final long serialVersionUID = 6680701524304107547L;
public static InvoiceGroupingKeyCoder of() {
return INSTANCE;
}
private InvoiceGroupingKeyCoder() {}
@Override
public void encode(InvoiceGroupingKey value, OutputStream outStream) throws IOException {
@@ -295,7 +294,6 @@ public abstract class BillingEvent implements Serializable {
@Override
public InvoiceGroupingKey decode(InputStream inStream) throws IOException {
Coder<String> stringCoder = StringUtf8Coder.of();
return new AutoValue_BillingEvent_InvoiceGroupingKey(
stringCoder.decode(inStream),
stringCoder.decode(inStream),
@@ -308,4 +306,55 @@ public abstract class BillingEvent implements Serializable {
}
}
}
static class BillingEventCoder extends AtomicCoder<BillingEvent> {
private static final Coder<String> stringCoder = StringUtf8Coder.of();
private static final Coder<Integer> integerCoder = VarIntCoder.of();
private static final Coder<Long> longCoder = VarLongCoder.of();
private static final Coder<Double> doubleCoder = DoubleCoder.of();
private static final BillingEventCoder INSTANCE = new BillingEventCoder();
static NullableCoder<BillingEvent> ofNullable() {
return NullableCoder.of(INSTANCE);
}
private BillingEventCoder() {}
@Override
public void encode(BillingEvent value, OutputStream outStream) throws IOException {
longCoder.encode(value.id(), outStream);
stringCoder.encode(DATE_TIME_FORMATTER.print(value.billingTime()), outStream);
stringCoder.encode(DATE_TIME_FORMATTER.print(value.eventTime()), outStream);
stringCoder.encode(value.registrarId(), outStream);
stringCoder.encode(value.billingId(), outStream);
stringCoder.encode(value.poNumber(), outStream);
stringCoder.encode(value.tld(), outStream);
stringCoder.encode(value.action(), outStream);
stringCoder.encode(value.domain(), outStream);
stringCoder.encode(value.repositoryId(), outStream);
integerCoder.encode(value.years(), outStream);
stringCoder.encode(value.currency(), outStream);
doubleCoder.encode(value.amount(), outStream);
stringCoder.encode(value.flags(), outStream);
}
@Override
public BillingEvent decode(InputStream inStream) throws IOException {
return new AutoValue_BillingEvent(
longCoder.decode(inStream),
DATE_TIME_FORMATTER.parseDateTime(stringCoder.decode(inStream)),
DATE_TIME_FORMATTER.parseDateTime(stringCoder.decode(inStream)),
stringCoder.decode(inStream),
stringCoder.decode(inStream),
stringCoder.decode(inStream),
stringCoder.decode(inStream),
stringCoder.decode(inStream),
stringCoder.decode(inStream),
stringCoder.decode(inStream),
integerCoder.decode(inStream),
stringCoder.decode(inStream),
doubleCoder.decode(inStream),
stringCoder.decode(inStream));
}
}
}

View File

@@ -18,6 +18,7 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static org.apache.beam.sdk.values.TypeDescriptors.strings;
import com.google.common.flogger.FluentLogger;
import google.registry.beam.billing.BillingEvent.BillingEventCoder;
import google.registry.beam.billing.BillingEvent.InvoiceGroupingKey;
import google.registry.beam.billing.BillingEvent.InvoiceGroupingKey.InvoiceGroupingKeyCoder;
import google.registry.beam.common.RegistryJpaIO;
@@ -30,6 +31,7 @@ import google.registry.reporting.billing.BillingModule;
import google.registry.util.DomainNameUtils;
import google.registry.util.ResourceUtils;
import google.registry.util.SqlTemplate;
import java.io.Serial;
import java.io.Serializable;
import java.time.YearMonth;
import java.util.Objects;
@@ -37,13 +39,13 @@ import java.util.Optional;
import java.util.regex.Pattern;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.coders.SerializableCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.io.FileIO;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.transforms.Contextful;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.Distinct;
import org.apache.beam.sdk.transforms.Filter;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
@@ -65,7 +67,7 @@ import org.joda.money.CurrencyUnit;
*/
public class InvoicingPipeline implements Serializable {
private static final long serialVersionUID = 5386330443625580081L;
@Serial private static final long serialVersionUID = 5386330443625580081L;
private static final Pattern SQL_COMMENT_REGEX =
Pattern.compile("^\\s*--.*\\n", Pattern.MULTILINE);
@@ -97,13 +99,11 @@ public class InvoicingPipeline implements Serializable {
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(google.registry.beam.billing.BillingEvent.class));
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));
.withCoder(BillingEventCoder.ofNullable());
return pipeline
.apply("Read BillingEvents from Cloud SQL", read)
.apply("Remove null elements", Filter.by(Objects::nonNull))
.apply("Remove duplicates", Distinct.create());
}
private static Optional<google.registry.beam.billing.BillingEvent> parseRow(Object[] row) {
@@ -142,7 +142,7 @@ public class InvoicingPipeline implements Serializable {
extends PTransform<
PCollection<google.registry.beam.billing.BillingEvent>, PCollection<String>> {
private static final long serialVersionUID = -8090619008258393728L;
@Serial private static final long serialVersionUID = -8090619008258393728L;
@Override
public PCollection<String> expand(
@@ -152,9 +152,9 @@ public class InvoicingPipeline implements Serializable {
"Map to invoicing key",
MapElements.into(TypeDescriptor.of(InvoiceGroupingKey.class))
.via(google.registry.beam.billing.BillingEvent::getInvoiceGroupingKey))
.setCoder(InvoiceGroupingKeyCoder.of())
.apply(
"Filter out free events", Filter.by((InvoiceGroupingKey key) -> key.unitPrice() != 0))
.setCoder(new InvoiceGroupingKeyCoder())
.apply("Count occurrences", Count.perElement())
.apply(
"Format as CSVs",