mirror of
https://github.com/google/nomulus.git
synced 2025-07-02 17:23:32 +02:00
Add preliminary spec11 monthly pipeline
This adds the scaffolding for a basic Spec11 pipeline- it gathers all domains from all time for a given project and counts how many there are. I've factored out a few common utilities for beam pipelines to avoid excessive duplication. Future CLs will: - Actually process domains via the SafeBrowsing API - Generate a real spec11 report - Template queries based on the input YearMonth - Abstract more commonalities across beam pipelines to reduce boilerplate when adding new pipelines. TESTED: FOSS test passed, and ran successfully on alpha ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=205997741
This commit is contained in:
parent
ded40851d3
commit
d199b383e5
14 changed files with 252 additions and 38 deletions
|
@ -6,4 +6,15 @@ licenses(["notice"]) # Apache 2.0
|
|||
|
||||
java_library(
|
||||
name = "beam",
|
||||
srcs = glob(["*.java"]),
|
||||
deps = [
|
||||
"@com_google_flogger",
|
||||
"@com_google_flogger_system_backend",
|
||||
"@com_google_guava",
|
||||
"@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",
|
||||
],
|
||||
)
|
||||
|
|
57
java/google/registry/beam/BeamUtils.java
Normal file
57
java/google/registry/beam/BeamUtils.java
Normal file
|
@ -0,0 +1,57 @@
|
|||
// Copyright 2018 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.base.Joiner;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
|
||||
|
||||
/** Static utilities for {@code Beam} pipelines. */
|
||||
public class BeamUtils {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
/** Extracts a string representation of a field in a {@link GenericRecord}. */
|
||||
public static String extractField(GenericRecord record, String fieldName) {
|
||||
return String.valueOf(record.get(fieldName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that no expected fields in the record are missing.
|
||||
*
|
||||
* <p>Note that this simply makes sure the field is not null; it may still generate a parse error
|
||||
* when interpreting the string representation of an object.
|
||||
*
|
||||
* @throws IllegalStateException if the record returns null for any field in {@code fieldNames}
|
||||
*/
|
||||
public static void checkFieldsNotNull(
|
||||
ImmutableList<String> fieldNames, SchemaAndRecord schemaAndRecord) {
|
||||
GenericRecord record = schemaAndRecord.getRecord();
|
||||
ImmutableList<String> nullFields =
|
||||
fieldNames
|
||||
.stream()
|
||||
.filter(fieldName -> record.get(fieldName) == null)
|
||||
.collect(ImmutableList.toImmutableList());
|
||||
String missingFieldList = Joiner.on(", ").join(nullFields);
|
||||
if (!nullFields.isEmpty()) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Read unexpected null value for field(s) %s for record %s",
|
||||
missingFieldList, record));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -9,6 +9,7 @@ java_library(
|
|||
srcs = glob(["*.java"]),
|
||||
resources = glob(["sql/*"]),
|
||||
deps = [
|
||||
"//java/google/registry/beam",
|
||||
"//java/google/registry/config",
|
||||
"//java/google/registry/model",
|
||||
"//java/google/registry/reporting/billing",
|
||||
|
|
|
@ -14,6 +14,9 @@
|
|||
|
||||
package google.registry.beam.invoicing;
|
||||
|
||||
import static google.registry.beam.BeamUtils.checkFieldsNotNull;
|
||||
import static google.registry.beam.BeamUtils.extractField;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Joiner;
|
||||
|
@ -108,7 +111,7 @@ public abstract class BillingEvent implements Serializable {
|
|||
* Apache AVRO GenericRecord</a>
|
||||
*/
|
||||
static BillingEvent parseFromRecord(SchemaAndRecord schemaAndRecord) {
|
||||
checkFieldsNotNull(schemaAndRecord);
|
||||
checkFieldsNotNull(FIELD_NAMES, schemaAndRecord);
|
||||
GenericRecord record = schemaAndRecord.getRecord();
|
||||
String flags = extractField(record, "flags");
|
||||
double amount = getDiscountedAmount(Double.parseDouble(extractField(record, "amount")), flags);
|
||||
|
@ -337,30 +340,4 @@ public abstract class BillingEvent implements Serializable {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Extracts a string representation of a field in a {@code GenericRecord}. */
|
||||
private static String extractField(GenericRecord record, String fieldName) {
|
||||
return String.valueOf(record.get(fieldName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that no expected fields in the record are missing.
|
||||
*
|
||||
* <p>Note that this simply makes sure the field is not null; it may still generate a parse error
|
||||
* in {@code parseFromRecord}.
|
||||
*/
|
||||
private static void checkFieldsNotNull(SchemaAndRecord schemaAndRecord) {
|
||||
GenericRecord record = schemaAndRecord.getRecord();
|
||||
ImmutableList<String> nullFields =
|
||||
FIELD_NAMES
|
||||
.stream()
|
||||
.filter(fieldName -> record.get(fieldName) == null)
|
||||
.collect(ImmutableList.toImmutableList());
|
||||
if (!nullFields.isEmpty()) {
|
||||
logger.atSevere().log(
|
||||
"Found unexpected null value(s) in field(s) %s for record %s",
|
||||
Joiner.on(", ").join(nullFields), record);
|
||||
throw new IllegalStateException("Read null value from Bigquery query");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -67,8 +67,8 @@ public class InvoicingPipeline implements Serializable {
|
|||
String invoiceTemplateUrl;
|
||||
|
||||
@Inject
|
||||
@Config("invoiceStagingUrl")
|
||||
String invoiceStagingUrl;
|
||||
@Config("beamStagingUrl")
|
||||
String beamStagingUrl;
|
||||
|
||||
@Inject
|
||||
@Config("billingBucketUrl")
|
||||
|
@ -99,7 +99,7 @@ public class InvoicingPipeline implements Serializable {
|
|||
options.setRunner(DataflowRunner.class);
|
||||
// This causes p.run() to stage the pipeline as a template on GCS, as opposed to running it.
|
||||
options.setTemplateLocation(invoiceTemplateUrl);
|
||||
options.setStagingLocation(invoiceStagingUrl);
|
||||
options.setStagingLocation(beamStagingUrl);
|
||||
Pipeline p = Pipeline.create(options);
|
||||
|
||||
PCollection<BillingEvent> billingEvents =
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue