mirror of
https://github.com/google/nomulus.git
synced 2025-07-13 14:35:16 +02:00
Add beam package to open source build
------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=178833972
This commit is contained in:
parent
52ce49a02c
commit
36ad38e5df
13 changed files with 1398 additions and 3 deletions
38
javatests/google/registry/beam/BUILD
Normal file
38
javatests/google/registry/beam/BUILD
Normal file
|
@ -0,0 +1,38 @@
|
|||
package(
|
||||
default_testonly = 1,
|
||||
default_visibility = ["//java/google/registry:registry_project"],
|
||||
)
|
||||
|
||||
licenses(["notice"]) # Apache 2.0
|
||||
|
||||
load("//java/com/google/testing/builddefs:GenTestRules.bzl", "GenTestRules")
|
||||
|
||||
java_library(
|
||||
name = "beam",
|
||||
srcs = glob(["*.java"]),
|
||||
resources = glob(["testdata/*"]),
|
||||
deps = [
|
||||
"//java/google/registry/beam",
|
||||
"//javatests/google/registry/testing",
|
||||
"@com_google_apis_google_api_services_bigquery",
|
||||
"@com_google_dagger",
|
||||
"@com_google_guava",
|
||||
"@com_google_truth",
|
||||
"@com_google_truth_extensions_truth_java8_extension",
|
||||
"@joda_time",
|
||||
"@junit",
|
||||
"@org_apache_avro",
|
||||
"@org_apache_beam_runners_direct_java",
|
||||
"@org_apache_beam_runners_google_cloud_dataflow_java",
|
||||
"@org_apache_beam_sdks_java_core",
|
||||
"@org_apache_beam_sdks_java_io_google_cloud_platform",
|
||||
"@org_mockito_all",
|
||||
],
|
||||
)
|
||||
|
||||
GenTestRules(
|
||||
name = "GeneratedTestRules",
|
||||
default_test_size = "small",
|
||||
test_files = glob(["*Test.java"]),
|
||||
deps = [":beam"],
|
||||
)
|
|
@ -0,0 +1,84 @@
|
|||
// 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.beam;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
|
||||
import com.google.api.services.bigquery.model.TableRow;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import google.registry.beam.BigqueryTemplatePipeline.CountRequestPaths;
|
||||
import google.registry.beam.BigqueryTemplatePipeline.ExtractRequestPathFn;
|
||||
import java.util.List;
|
||||
import org.apache.beam.runners.direct.DirectRunner;
|
||||
import org.apache.beam.sdk.options.PipelineOptions;
|
||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||
import org.apache.beam.sdk.testing.PAssert;
|
||||
import org.apache.beam.sdk.testing.TestPipeline;
|
||||
import org.apache.beam.sdk.transforms.Create;
|
||||
import org.apache.beam.sdk.transforms.DoFnTester;
|
||||
import org.apache.beam.sdk.values.PCollection;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.JUnit4;
|
||||
|
||||
/** Unit tests for {@link BigqueryTemplatePipeline}*/
|
||||
@RunWith(JUnit4.class)
|
||||
public class BigqueryTemplatePipelineTest {
|
||||
|
||||
private static PipelineOptions pipelineOptions;
|
||||
|
||||
@BeforeClass
|
||||
public static void initializePipelineOptions() {
|
||||
pipelineOptions = PipelineOptionsFactory.create();
|
||||
pipelineOptions.setRunner(DirectRunner.class);
|
||||
}
|
||||
|
||||
@Rule public final transient TestPipeline p = TestPipeline.fromOptions(pipelineOptions);
|
||||
|
||||
@Test
|
||||
public void testExtractRequestPathFn() throws Exception {
|
||||
ExtractRequestPathFn extractRequestPathFn = new ExtractRequestPathFn();
|
||||
DoFnTester<TableRow, String> fnTester = DoFnTester.of(extractRequestPathFn);
|
||||
TableRow emptyRow = new TableRow();
|
||||
TableRow hasRequestPathRow = new TableRow().set("requestPath", "a/path");
|
||||
TableRow hasOtherValueRow = new TableRow().set("anotherValue", "b/lah");
|
||||
List<String> outputs = fnTester.processBundle(emptyRow, hasRequestPathRow, hasOtherValueRow);
|
||||
assertThat(outputs).containsExactly("null", "a/path", "null");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEndToEndPipeline() throws Exception {
|
||||
ImmutableList<TableRow> inputRows =
|
||||
ImmutableList.of(
|
||||
new TableRow(),
|
||||
new TableRow().set("requestPath", "a/path"),
|
||||
new TableRow().set("requestPath", "b/path"),
|
||||
new TableRow().set("requestPath", "b/path"),
|
||||
new TableRow().set("anotherValue", "b/path"));
|
||||
|
||||
PCollection<TableRow> input = p.apply(Create.of(inputRows));
|
||||
PCollection<String> output = input.apply(new CountRequestPaths());
|
||||
|
||||
ImmutableList<String> outputStrings = new ImmutableList.Builder<String>()
|
||||
.add("a/path: 1")
|
||||
.add("b/path: 2")
|
||||
.add("null: 2")
|
||||
.build();
|
||||
PAssert.that(output).containsInAnyOrder(outputStrings);
|
||||
p.run();
|
||||
}
|
||||
}
|
172
javatests/google/registry/beam/BillingEventTest.java
Normal file
172
javatests/google/registry/beam/BillingEventTest.java
Normal file
|
@ -0,0 +1,172 @@
|
|||
// 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.beam;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.testing.JUnitBackports.assertThrows;
|
||||
|
||||
import google.registry.beam.BillingEvent.InvoiceGroupingKey;
|
||||
import google.registry.beam.BillingEvent.InvoiceGroupingKey.InvoiceGroupingKeyCoder;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.JUnit4;
|
||||
|
||||
/** Unit tests for {@link BillingEvent} */
|
||||
@RunWith(JUnit4.class)
|
||||
public class BillingEventTest {
|
||||
|
||||
private static final String BILLING_EVENT_SCHEMA =
|
||||
"{\"name\": \"BillingEvent\", "
|
||||
+ "\"type\": \"record\", "
|
||||
+ "\"fields\": ["
|
||||
+ "{\"name\": \"id\", \"type\": \"long\"},"
|
||||
+ "{\"name\": \"billingTime\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"eventTime\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"registrarId\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"billingId\", \"type\": \"long\"},"
|
||||
+ "{\"name\": \"tld\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"action\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"domain\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"repositoryId\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"years\", \"type\": \"int\"},"
|
||||
+ "{\"name\": \"currency\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"amount\", \"type\": \"float\"},"
|
||||
+ "{\"name\": \"flags\", \"type\": \"string\"}"
|
||||
+ "]}";
|
||||
|
||||
private SchemaAndRecord schemaAndRecord;
|
||||
|
||||
@Before
|
||||
public void initializeRecord() {
|
||||
// Create a record with a given JSON schema.
|
||||
GenericRecord record = new GenericData.Record(new Schema.Parser().parse(BILLING_EVENT_SCHEMA));
|
||||
record.put("id", "1");
|
||||
record.put("billingTime", 1508835963000000L);
|
||||
record.put("eventTime", 1484870383000000L);
|
||||
record.put("registrarId", "myRegistrar");
|
||||
record.put("billingId", "12345-CRRHELLO");
|
||||
record.put("tld", "test");
|
||||
record.put("action", "RENEW");
|
||||
record.put("domain", "example.test");
|
||||
record.put("repositoryId", "123456");
|
||||
record.put("years", 5);
|
||||
record.put("currency", "USD");
|
||||
record.put("amount", 20.5);
|
||||
record.put("flags", "AUTO_RENEW SYNTHETIC");
|
||||
schemaAndRecord = new SchemaAndRecord(record, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseBillingEventFromRecord_success() {
|
||||
BillingEvent event = BillingEvent.parseFromRecord(schemaAndRecord);
|
||||
assertThat(event.id()).isEqualTo(1);
|
||||
assertThat(event.billingTime())
|
||||
.isEqualTo(ZonedDateTime.of(2017, 10, 24, 9, 6, 3, 0, ZoneId.of("UTC")));
|
||||
assertThat(event.eventTime())
|
||||
.isEqualTo(ZonedDateTime.of(2017, 1, 19, 23, 59, 43, 0, ZoneId.of("UTC")));
|
||||
assertThat(event.registrarId()).isEqualTo("myRegistrar");
|
||||
assertThat(event.billingId()).isEqualTo("12345-CRRHELLO");
|
||||
assertThat(event.tld()).isEqualTo("test");
|
||||
assertThat(event.action()).isEqualTo("RENEW");
|
||||
assertThat(event.domain()).isEqualTo("example.test");
|
||||
assertThat(event.repositoryId()).isEqualTo("123456");
|
||||
assertThat(event.years()).isEqualTo(5);
|
||||
assertThat(event.currency()).isEqualTo("USD");
|
||||
assertThat(event.amount()).isEqualTo(20.5);
|
||||
assertThat(event.flags()).isEqualTo("AUTO_RENEW SYNTHETIC");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseBillingEventFromRecord_nullValue_throwsException() {
|
||||
schemaAndRecord.getRecord().put("tld", null);
|
||||
assertThrows(IllegalStateException.class, () -> BillingEvent.parseFromRecord(schemaAndRecord));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertBillingEvent_toCsv() {
|
||||
BillingEvent event = BillingEvent.parseFromRecord(schemaAndRecord);
|
||||
assertThat(event.toCsv())
|
||||
.isEqualTo("1,2017-10-24 09:06:03 UTC,2017-01-19 23:59:43 UTC,myRegistrar,"
|
||||
+ "12345-CRRHELLO,test,RENEW,example.test,123456,5,USD,20.50,AUTO_RENEW");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGenerateBillingEventFilename() {
|
||||
BillingEvent event = BillingEvent.parseFromRecord(schemaAndRecord);
|
||||
assertThat(event.toFilename()).isEqualTo("myRegistrar_test");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetInvoiceGroupingKey_fromBillingEvent() {
|
||||
BillingEvent event = BillingEvent.parseFromRecord(schemaAndRecord);
|
||||
InvoiceGroupingKey invoiceKey = event.getInvoiceGroupingKey();
|
||||
assertThat(invoiceKey.startDate()).isEqualTo("2017-10-01");
|
||||
assertThat(invoiceKey.endDate()).isEqualTo("2022-09-30");
|
||||
assertThat(invoiceKey.productAccountKey()).isEqualTo("12345-CRRHELLO");
|
||||
assertThat(invoiceKey.usageGroupingKey()).isEqualTo("myRegistrar - test");
|
||||
assertThat(invoiceKey.description()).isEqualTo("RENEW | TLD: test | TERM: 5-year");
|
||||
assertThat(invoiceKey.unitPrice()).isEqualTo(20.5);
|
||||
assertThat(invoiceKey.unitPriceCurrency()).isEqualTo("USD");
|
||||
assertThat(invoiceKey.poNumber()).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertInvoiceGroupingKey_toCsv() {
|
||||
BillingEvent event = BillingEvent.parseFromRecord(schemaAndRecord);
|
||||
InvoiceGroupingKey invoiceKey = event.getInvoiceGroupingKey();
|
||||
assertThat(invoiceKey.toCsv(3L))
|
||||
.isEqualTo(
|
||||
"2017-10-01,2022-09-30,12345-CRRHELLO,61.50,USD,10125,1,PURCHASE,"
|
||||
+ "myRegistrar - test,3,RENEW | TLD: test | TERM: 5-year,20.50,USD,");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvoiceGroupingKeyCoder_deterministicSerialization() throws IOException {
|
||||
InvoiceGroupingKey invoiceKey =
|
||||
BillingEvent.parseFromRecord(schemaAndRecord).getInvoiceGroupingKey();
|
||||
InvoiceGroupingKeyCoder coder = new InvoiceGroupingKeyCoder();
|
||||
ByteArrayOutputStream outStream = new ByteArrayOutputStream();
|
||||
coder.encode(invoiceKey, outStream);
|
||||
InputStream inStream = new ByteArrayInputStream(outStream.toByteArray());
|
||||
assertThat(coder.decode(inStream)).isEqualTo(invoiceKey);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetDetailReportHeader() {
|
||||
assertThat(BillingEvent.getHeader())
|
||||
.isEqualTo(
|
||||
"id,billingTime,eventTime,registrarId,billingId,tld,action,"
|
||||
+ "domain,repositoryId,years,currency,amount,flags");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetOverallInvoiceHeader() {
|
||||
assertThat(InvoiceGroupingKey.invoiceHeader())
|
||||
.isEqualTo("StartDate,EndDate,ProductAccountKey,Amount,AmountCurrency,BillingProductCode,"
|
||||
+ "SalesChannel,LineItemType,UsageGroupingKey,Quantity,Description,UnitPrice,"
|
||||
+ "UnitPriceCurrency,PONumber");
|
||||
}
|
||||
}
|
122
javatests/google/registry/beam/InvoicingPipelineTest.java
Normal file
122
javatests/google/registry/beam/InvoicingPipelineTest.java
Normal file
|
@ -0,0 +1,122 @@
|
|||
// 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.beam;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import google.registry.beam.InvoicingPipeline.GenerateInvoiceRows;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import org.apache.beam.runners.direct.DirectRunner;
|
||||
import org.apache.beam.sdk.options.PipelineOptions;
|
||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||
import org.apache.beam.sdk.testing.PAssert;
|
||||
import org.apache.beam.sdk.testing.TestPipeline;
|
||||
import org.apache.beam.sdk.transforms.Create;
|
||||
import org.apache.beam.sdk.values.PCollection;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.JUnit4;
|
||||
|
||||
/** Unit tests for {@link InvoicingPipeline}. */
|
||||
@RunWith(JUnit4.class)
|
||||
public class InvoicingPipelineTest {
|
||||
|
||||
private static PipelineOptions pipelineOptions;
|
||||
|
||||
@BeforeClass
|
||||
public static void initializePipelineOptions() {
|
||||
pipelineOptions = PipelineOptionsFactory.create();
|
||||
pipelineOptions.setRunner(DirectRunner.class);
|
||||
}
|
||||
|
||||
@Rule public final transient TestPipeline p = TestPipeline.fromOptions(pipelineOptions);
|
||||
|
||||
@Test
|
||||
public void testGenerateInvoiceRowsFn() throws Exception {
|
||||
ImmutableList<BillingEvent> inputRows =
|
||||
ImmutableList.of(
|
||||
BillingEvent.create(
|
||||
1,
|
||||
ZonedDateTime.of(2017, 10, 4, 0, 0, 0, 0, ZoneId.of("UTC")),
|
||||
ZonedDateTime.of(2017, 10, 4, 0, 0, 0, 0, ZoneId.of("UTC")),
|
||||
"theRegistrar",
|
||||
"234",
|
||||
"test",
|
||||
"RENEW",
|
||||
"mydomain.test",
|
||||
"REPO-ID",
|
||||
3,
|
||||
"USD",
|
||||
20.5,
|
||||
""),
|
||||
BillingEvent.create(
|
||||
1,
|
||||
ZonedDateTime.of(2017, 10, 4, 0, 0, 0, 0, ZoneId.of("UTC")),
|
||||
ZonedDateTime.of(2017, 10, 4, 0, 0, 0, 0, ZoneId.of("UTC")),
|
||||
"theRegistrar",
|
||||
"234",
|
||||
"test",
|
||||
"RENEW",
|
||||
"mydomain2.test",
|
||||
"REPO-ID",
|
||||
3,
|
||||
"USD",
|
||||
20.5,
|
||||
""),
|
||||
BillingEvent.create(
|
||||
1,
|
||||
ZonedDateTime.of(2017, 10, 2, 0, 0, 0, 0, ZoneId.of("UTC")),
|
||||
ZonedDateTime.of(2017, 9, 29, 0, 0, 0, 0, ZoneId.of("UTC")),
|
||||
"theRegistrar",
|
||||
"234",
|
||||
"hello",
|
||||
"CREATE",
|
||||
"mydomain3.hello",
|
||||
"REPO-ID",
|
||||
5,
|
||||
"JPY",
|
||||
70.75,
|
||||
""),
|
||||
BillingEvent.create(
|
||||
1,
|
||||
ZonedDateTime.of(2017, 10, 4, 0, 0, 0, 0, ZoneId.of("UTC")),
|
||||
ZonedDateTime.of(2017, 10, 4, 0, 0, 0, 0, ZoneId.of("UTC")),
|
||||
"googledomains",
|
||||
"456",
|
||||
"test",
|
||||
"RENEW",
|
||||
"mydomain4.test",
|
||||
"REPO-ID",
|
||||
1,
|
||||
"USD",
|
||||
20.5,
|
||||
""));
|
||||
|
||||
PCollection<BillingEvent> input = p.apply(Create.of(inputRows));
|
||||
PCollection<String> output = input.apply(new GenerateInvoiceRows());
|
||||
|
||||
ImmutableList<String> outputStrings = ImmutableList.of(
|
||||
"2017-10-01,2020-09-30,234,41.00,USD,10125,1,PURCHASE,theRegistrar - test,2,"
|
||||
+ "RENEW | TLD: test | TERM: 3-year,20.50,USD,",
|
||||
"2017-10-01,2022-09-30,234,70.75,JPY,10125,1,PURCHASE,theRegistrar - hello,1,"
|
||||
+ "CREATE | TLD: hello | TERM: 5-year,70.75,JPY,",
|
||||
"2017-10-01,2018-09-30,456,20.50,USD,10125,1,PURCHASE,googledomains - test,1,"
|
||||
+ "RENEW | TLD: test | TERM: 1-year,20.50,USD,");
|
||||
PAssert.that(output).containsInAnyOrder(outputStrings);
|
||||
p.run();
|
||||
}
|
||||
}
|
74
javatests/google/registry/beam/InvoicingUtilsTest.java
Normal file
74
javatests/google/registry/beam/InvoicingUtilsTest.java
Normal file
|
@ -0,0 +1,74 @@
|
|||
// 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.beam;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import google.registry.testing.TestDataHelper;
|
||||
import org.apache.beam.sdk.io.DefaultFilenamePolicy.Params;
|
||||
import org.apache.beam.sdk.io.FileBasedSink;
|
||||
import org.apache.beam.sdk.options.ValueProvider;
|
||||
import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
|
||||
import org.apache.beam.sdk.transforms.SerializableFunction;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.JUnit4;
|
||||
|
||||
/** Unit tests for {@link InvoicingUtils}. */
|
||||
@RunWith(JUnit4.class)
|
||||
public class InvoicingUtilsTest {
|
||||
|
||||
@Test
|
||||
public void testDestinationFunction_generatesProperFileParams() {
|
||||
SerializableFunction<BillingEvent, Params> destinationFunction =
|
||||
InvoicingUtils.makeDestinationFunction("my/directory");
|
||||
|
||||
BillingEvent billingEvent = mock(BillingEvent.class);
|
||||
// We mock BillingEvent to make the test independent of the implementation of toFilename()
|
||||
when(billingEvent.toFilename()).thenReturn("registrar_tld");
|
||||
|
||||
assertThat(destinationFunction.apply(billingEvent))
|
||||
.isEqualTo(
|
||||
new Params()
|
||||
.withShardTemplate("")
|
||||
.withSuffix(".csv")
|
||||
.withBaseFilename(
|
||||
FileBasedSink.convertToFileResourceIfPossible("my/directory/registrar_tld")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyDestinationParams() {
|
||||
assertThat(InvoicingUtils.makeEmptyDestinationParams("my/directory"))
|
||||
.isEqualTo(
|
||||
new Params()
|
||||
.withBaseFilename(
|
||||
FileBasedSink.convertToFileResourceIfPossible("my/directory/failed")));
|
||||
}
|
||||
|
||||
/** Asserts that the instantiated sql template matches a golden expected file. */
|
||||
@Test
|
||||
public void testMakeQueryProvider() {
|
||||
ValueProvider<String> queryProvider =
|
||||
InvoicingUtils.makeQueryProvider(StaticValueProvider.of("2017-10"), "my-project-id");
|
||||
assertThat(queryProvider.get()).isEqualTo(loadFile("billing_events_test.sql"));
|
||||
}
|
||||
|
||||
/** Returns a {@link String} from a file in the {@code billing/testdata/} directory. */
|
||||
private static String loadFile(String filename) {
|
||||
return TestDataHelper.loadFile(InvoicingUtilsTest.class, filename);
|
||||
}
|
||||
}
|
99
javatests/google/registry/beam/testdata/billing_events_test.sql
vendored
Normal file
99
javatests/google/registry/beam/testdata/billing_events_test.sql
vendored
Normal file
|
@ -0,0 +1,99 @@
|
|||
#standardSQL
|
||||
-- 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.
|
||||
|
||||
-- This query gathers all non-canceled billing events for a given
|
||||
-- YEAR_MONTH in yyyy-MM format.
|
||||
|
||||
SELECT
|
||||
__key__.id AS id,
|
||||
billingTime,
|
||||
eventTime,
|
||||
BillingEvent.clientId AS registrarId,
|
||||
RegistrarData.accountId AS billingId,
|
||||
tld,
|
||||
reason as action,
|
||||
targetId as domain,
|
||||
BillingEvent.domainRepoId as repositoryId,
|
||||
periodYears as years,
|
||||
BillingEvent.currency AS currency,
|
||||
BillingEvent.amount as amount,
|
||||
-- We'll strip out non-useful flags downstream
|
||||
ARRAY_TO_STRING(flags, " ") AS flags
|
||||
FROM (
|
||||
SELECT
|
||||
*,
|
||||
-- We store cost as "CURRENCY AMOUNT" such as "JPY 800" or "USD 20.00"
|
||||
SPLIT(cost, ' ')[OFFSET(0)] AS currency,
|
||||
SPLIT(cost, ' ')[OFFSET(1)] AS amount,
|
||||
-- Extract everything after the first dot in the domain as the TLD
|
||||
REGEXP_EXTRACT(targetId, r'[.](.+)') AS tld,
|
||||
-- __key__.path looks like '"DomainBase", "<repoId>", ...'
|
||||
REGEXP_REPLACE(SPLIT(__key__.path, ', ')[OFFSET(1)], '"', '')
|
||||
AS domainRepoId,
|
||||
COALESCE(cancellationMatchingBillingEvent.path,
|
||||
__key__.path) AS cancellationMatchingPath
|
||||
FROM
|
||||
`my-project-id.latest_datastore_export.OneTime`
|
||||
-- Only include real TLDs (filter prober data)
|
||||
WHERE
|
||||
REGEXP_EXTRACT(targetId, r'[.](.+)') IN (
|
||||
SELECT
|
||||
tldStr
|
||||
FROM
|
||||
`my-project-id.latest_datastore_export.Registry`
|
||||
WHERE
|
||||
tldType = 'REAL') ) AS BillingEvent
|
||||
-- Gather billing ID from registrar table
|
||||
-- This is a 'JOIN' as opposed to 'LEFT JOIN' to filter out
|
||||
-- non-billable registrars
|
||||
JOIN (
|
||||
SELECT
|
||||
__key__.name AS clientId,
|
||||
billingIdentifier,
|
||||
r.billingAccountMap.currency[SAFE_OFFSET(index)] AS currency,
|
||||
r.billingAccountMap.accountId[SAFE_OFFSET(index)] AS accountId
|
||||
FROM
|
||||
`my-project-id.latest_datastore_export.Registrar` AS r,
|
||||
UNNEST(GENERATE_ARRAY(0, ARRAY_LENGTH(r.billingAccountMap.currency) - 1))
|
||||
AS index
|
||||
WHERE billingAccountMap IS NOT NULL
|
||||
AND type = 'REAL') AS RegistrarData
|
||||
ON
|
||||
BillingEvent.clientId = RegistrarData.clientId
|
||||
AND BillingEvent.currency = RegistrarData.currency
|
||||
-- Gather cancellations
|
||||
LEFT JOIN (
|
||||
SELECT __key__.id AS cancellationId,
|
||||
COALESCE(refOneTime.path, refRecurring.path) AS cancelledEventPath,
|
||||
eventTime as cancellationTime,
|
||||
billingTime as cancellationBillingTime
|
||||
FROM
|
||||
(SELECT
|
||||
*,
|
||||
-- Count everything after first dot as TLD (to support multi-part TLDs).
|
||||
REGEXP_EXTRACT(targetId, r'[.](.+)') AS tld
|
||||
FROM
|
||||
`my-project-id.latest_datastore_export.Cancellation`)
|
||||
) AS Cancellation
|
||||
ON BillingEvent.cancellationMatchingPath = Cancellation.cancelledEventPath
|
||||
AND BillingEvent.billingTime = Cancellation.cancellationBillingTime
|
||||
WHERE billingTime BETWEEN TIMESTAMP('2017-10-01 00:00:00.000000')
|
||||
AND TIMESTAMP('2017-10-31 23:59:59.999999')
|
||||
-- Filter out canceled events
|
||||
AND Cancellation.cancellationId IS NULL
|
||||
ORDER BY
|
||||
billingTime DESC,
|
||||
id,
|
||||
tld
|
Loading…
Add table
Add a link
Reference in a new issue