mirror of
https://github.com/google/nomulus.git
synced 2025-05-14 16:37:13 +02:00
Prepare billing pipeline for production
This makes a few cosmetic changes that prepares the pipeline for production. Namely: - Converts file names to include the input yearMonth, mostly mirroring the original invoicing pipeline. - Factors out the yearMonth logic from the reporting module to the more common backend module. We will likely use the default yearMonth logic in other backend tasks (such as spec11 reporting). - Adds the "withTemplateCompatability" flag to the Bigquery read, which allows multiple uses of the same template. - Adds the 'billing' task queue, which retries up to 5 times every 3 minutes, which is about the rate we desire for checking if the pipeline is complete. - Adds a shell 'invoicing upload' class, which tests the retry semantics we want for post-generation work (e-mailing the invoice to crr-tech, and publishing detail reports) While this cl may look big, it's mostly just a refactor and setting up boilerplate needed to frame the upload logic. ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=179849586
This commit is contained in:
parent
53ed6035c4
commit
552ab12314
24 changed files with 436 additions and 115 deletions
|
@ -171,8 +171,8 @@ public abstract class BillingEvent implements Serializable {
|
||||||
* <p>When modifying this function, take care to ensure that there's no way to generate an illegal
|
* <p>When modifying this function, take care to ensure that there's no way to generate an illegal
|
||||||
* filepath with the arguments, such as "../sensitive_info".
|
* filepath with the arguments, such as "../sensitive_info".
|
||||||
*/
|
*/
|
||||||
String toFilename() {
|
String toFilename(String yearMonth) {
|
||||||
return String.format("%s_%s", registrarId(), tld());
|
return String.format("invoice_details_%s_%s_%s", yearMonth, registrarId(), tld());
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Generates a CSV representation of this {@code BillingEvent}. */
|
/** Generates a CSV representation of this {@code BillingEvent}. */
|
||||||
|
|
|
@ -17,6 +17,7 @@ package google.registry.beam;
|
||||||
import google.registry.beam.BillingEvent.InvoiceGroupingKey;
|
import google.registry.beam.BillingEvent.InvoiceGroupingKey;
|
||||||
import google.registry.beam.BillingEvent.InvoiceGroupingKey.InvoiceGroupingKeyCoder;
|
import google.registry.beam.BillingEvent.InvoiceGroupingKey.InvoiceGroupingKeyCoder;
|
||||||
import google.registry.config.RegistryConfig.Config;
|
import google.registry.config.RegistryConfig.Config;
|
||||||
|
import java.io.Serializable;
|
||||||
import javax.inject.Inject;
|
import javax.inject.Inject;
|
||||||
import org.apache.beam.runners.dataflow.DataflowRunner;
|
import org.apache.beam.runners.dataflow.DataflowRunner;
|
||||||
import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
|
import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
|
||||||
|
@ -29,6 +30,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
|
||||||
import org.apache.beam.sdk.options.Description;
|
import org.apache.beam.sdk.options.Description;
|
||||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||||
import org.apache.beam.sdk.options.ValueProvider;
|
import org.apache.beam.sdk.options.ValueProvider;
|
||||||
|
import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
|
||||||
import org.apache.beam.sdk.transforms.Count;
|
import org.apache.beam.sdk.transforms.Count;
|
||||||
import org.apache.beam.sdk.transforms.MapElements;
|
import org.apache.beam.sdk.transforms.MapElements;
|
||||||
import org.apache.beam.sdk.transforms.PTransform;
|
import org.apache.beam.sdk.transforms.PTransform;
|
||||||
|
@ -49,7 +51,7 @@ import org.apache.beam.sdk.values.TypeDescriptors;
|
||||||
*
|
*
|
||||||
* @see <a href="https://cloud.google.com/dataflow/docs/templates/overview">Dataflow Templates</a>
|
* @see <a href="https://cloud.google.com/dataflow/docs/templates/overview">Dataflow Templates</a>
|
||||||
*/
|
*/
|
||||||
public class InvoicingPipeline {
|
public class InvoicingPipeline implements Serializable {
|
||||||
|
|
||||||
@Inject @Config("projectId") String projectId;
|
@Inject @Config("projectId") String projectId;
|
||||||
@Inject @Config("apacheBeamBucketUrl") String beamBucket;
|
@Inject @Config("apacheBeamBucketUrl") String beamBucket;
|
||||||
|
@ -66,12 +68,12 @@ public class InvoicingPipeline {
|
||||||
|
|
||||||
/** Deploys the invoicing pipeline as a template on GCS, for a given projectID and GCS bucket. */
|
/** Deploys the invoicing pipeline as a template on GCS, for a given projectID and GCS bucket. */
|
||||||
public void deploy() {
|
public void deploy() {
|
||||||
|
// We can't store options as a member variable due to serialization concerns.
|
||||||
InvoicingPipelineOptions options = PipelineOptionsFactory.as(InvoicingPipelineOptions.class);
|
InvoicingPipelineOptions options = PipelineOptionsFactory.as(InvoicingPipelineOptions.class);
|
||||||
options.setProject(projectId);
|
options.setProject(projectId);
|
||||||
options.setRunner(DataflowRunner.class);
|
options.setRunner(DataflowRunner.class);
|
||||||
options.setStagingLocation(beamBucket + "/staging");
|
options.setStagingLocation(beamBucket + "/staging");
|
||||||
options.setTemplateLocation(beamBucket + "/templates/invoicing");
|
options.setTemplateLocation(beamBucket + "/templates/invoicing");
|
||||||
|
|
||||||
Pipeline p = Pipeline.create(options);
|
Pipeline p = Pipeline.create(options);
|
||||||
|
|
||||||
PCollection<BillingEvent> billingEvents =
|
PCollection<BillingEvent> billingEvents =
|
||||||
|
@ -81,9 +83,9 @@ public class InvoicingPipeline {
|
||||||
.fromQuery(InvoicingUtils.makeQueryProvider(options.getYearMonth(), projectId))
|
.fromQuery(InvoicingUtils.makeQueryProvider(options.getYearMonth(), projectId))
|
||||||
.withCoder(SerializableCoder.of(BillingEvent.class))
|
.withCoder(SerializableCoder.of(BillingEvent.class))
|
||||||
.usingStandardSql()
|
.usingStandardSql()
|
||||||
.withoutValidation());
|
.withoutValidation()
|
||||||
|
.withTemplateCompatibility());
|
||||||
applyTerminalTransforms(billingEvents);
|
applyTerminalTransforms(billingEvents, options.getYearMonth());
|
||||||
p.run();
|
p.run();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -92,13 +94,15 @@ public class InvoicingPipeline {
|
||||||
*
|
*
|
||||||
* <p>This is factored out purely to facilitate testing.
|
* <p>This is factored out purely to facilitate testing.
|
||||||
*/
|
*/
|
||||||
void applyTerminalTransforms(PCollection<BillingEvent> billingEvents) {
|
void applyTerminalTransforms(
|
||||||
|
PCollection<BillingEvent> billingEvents, ValueProvider<String> yearMonthProvider) {
|
||||||
billingEvents.apply(
|
billingEvents.apply(
|
||||||
"Write events to separate CSVs keyed by registrarId_tld pair", writeDetailReports());
|
"Write events to separate CSVs keyed by registrarId_tld pair",
|
||||||
|
writeDetailReports(yearMonthProvider));
|
||||||
|
|
||||||
billingEvents
|
billingEvents
|
||||||
.apply("Generate overall invoice rows", new GenerateInvoiceRows())
|
.apply("Generate overall invoice rows", new GenerateInvoiceRows())
|
||||||
.apply("Write overall invoice to CSV", writeInvoice());
|
.apply("Write overall invoice to CSV", writeInvoice(yearMonthProvider));
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Transform that converts a {@code BillingEvent} into an invoice CSV row. */
|
/** Transform that converts a {@code BillingEvent} into an invoice CSV row. */
|
||||||
|
@ -121,19 +125,23 @@ public class InvoicingPipeline {
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns an IO transform that writes the overall invoice to a single CSV file. */
|
/** Returns an IO transform that writes the overall invoice to a single CSV file. */
|
||||||
private TextIO.Write writeInvoice() {
|
private TextIO.Write writeInvoice(ValueProvider<String> yearMonthProvider) {
|
||||||
return TextIO.write()
|
return TextIO.write()
|
||||||
.to(beamBucket + "/results/overall_invoice")
|
.to(
|
||||||
|
NestedValueProvider.of(
|
||||||
|
yearMonthProvider,
|
||||||
|
yearMonth -> String.format("%s/results/CRR-INV-%s", beamBucket, yearMonth)))
|
||||||
.withHeader(InvoiceGroupingKey.invoiceHeader())
|
.withHeader(InvoiceGroupingKey.invoiceHeader())
|
||||||
.withoutSharding()
|
.withoutSharding()
|
||||||
.withSuffix(".csv");
|
.withSuffix(".csv");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns an IO transform that writes detail reports to registrar-tld keyed CSV files. */
|
/** Returns an IO transform that writes detail reports to registrar-tld keyed CSV files. */
|
||||||
private TextIO.TypedWrite<BillingEvent, Params> writeDetailReports() {
|
private TextIO.TypedWrite<BillingEvent, Params> writeDetailReports(
|
||||||
|
ValueProvider<String> yearMonthProvider) {
|
||||||
return TextIO.<BillingEvent>writeCustomType()
|
return TextIO.<BillingEvent>writeCustomType()
|
||||||
.to(
|
.to(
|
||||||
InvoicingUtils.makeDestinationFunction(beamBucket + "/results"),
|
InvoicingUtils.makeDestinationFunction(beamBucket + "/results", yearMonthProvider),
|
||||||
InvoicingUtils.makeEmptyDestinationParams(beamBucket + "/results"))
|
InvoicingUtils.makeEmptyDestinationParams(beamBucket + "/results"))
|
||||||
.withFormatFunction(BillingEvent::toCsv)
|
.withFormatFunction(BillingEvent::toCsv)
|
||||||
.withoutSharding()
|
.withoutSharding()
|
||||||
|
|
|
@ -39,15 +39,23 @@ public class InvoicingUtils {
|
||||||
* Returns a function mapping from {@code BillingEvent} to filename {@code Params}.
|
* Returns a function mapping from {@code BillingEvent} to filename {@code Params}.
|
||||||
*
|
*
|
||||||
* <p>Beam uses this to determine which file a given {@code BillingEvent} should get placed into.
|
* <p>Beam uses this to determine which file a given {@code BillingEvent} should get placed into.
|
||||||
|
*
|
||||||
|
* @param outputBucket the GCS bucket we're outputting reports to
|
||||||
|
* @param yearMonthProvider a runtime provider for the yyyy-MM we're generating the invoice for
|
||||||
*/
|
*/
|
||||||
static SerializableFunction<BillingEvent, Params> makeDestinationFunction(String outputBucket) {
|
static SerializableFunction<BillingEvent, Params> makeDestinationFunction(
|
||||||
|
String outputBucket, ValueProvider<String> yearMonthProvider) {
|
||||||
return billingEvent ->
|
return billingEvent ->
|
||||||
new Params()
|
new Params()
|
||||||
.withShardTemplate("")
|
.withShardTemplate("")
|
||||||
.withSuffix(".csv")
|
.withSuffix(".csv")
|
||||||
.withBaseFilename(
|
.withBaseFilename(
|
||||||
FileBasedSink.convertToFileResourceIfPossible(
|
NestedValueProvider.of(
|
||||||
String.format("%s/%s", outputBucket, billingEvent.toFilename())));
|
yearMonthProvider,
|
||||||
|
yearMonth ->
|
||||||
|
FileBasedSink.convertToFileResourceIfPossible(
|
||||||
|
String.format(
|
||||||
|
"%s/%s", outputBucket, billingEvent.toFilename(yearMonth)))));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -17,6 +17,7 @@ java_library(
|
||||||
"//java/google/registry/util",
|
"//java/google/registry/util",
|
||||||
"@com_google_api_client_appengine",
|
"@com_google_api_client_appengine",
|
||||||
"@com_google_apis_google_api_services_dataflow",
|
"@com_google_apis_google_api_services_dataflow",
|
||||||
|
"@com_google_appengine_api_1_0_sdk",
|
||||||
"@com_google_dagger",
|
"@com_google_dagger",
|
||||||
"@com_google_guava",
|
"@com_google_guava",
|
||||||
"@com_google_http_client",
|
"@com_google_http_client",
|
||||||
|
|
|
@ -14,6 +14,8 @@
|
||||||
|
|
||||||
package google.registry.billing;
|
package google.registry.billing;
|
||||||
|
|
||||||
|
import static google.registry.request.RequestParameters.extractRequiredParameter;
|
||||||
|
|
||||||
import com.google.api.client.googleapis.extensions.appengine.auth.oauth2.AppIdentityCredential;
|
import com.google.api.client.googleapis.extensions.appengine.auth.oauth2.AppIdentityCredential;
|
||||||
import com.google.api.client.http.HttpTransport;
|
import com.google.api.client.http.HttpTransport;
|
||||||
import com.google.api.client.json.JsonFactory;
|
import com.google.api.client.json.JsonFactory;
|
||||||
|
@ -22,8 +24,10 @@ import com.google.common.collect.ImmutableSet;
|
||||||
import dagger.Module;
|
import dagger.Module;
|
||||||
import dagger.Provides;
|
import dagger.Provides;
|
||||||
import google.registry.config.RegistryConfig.Config;
|
import google.registry.config.RegistryConfig.Config;
|
||||||
|
import google.registry.request.Parameter;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
|
import javax.servlet.http.HttpServletRequest;
|
||||||
|
|
||||||
/** Module for dependencies required by monthly billing actions. */
|
/** Module for dependencies required by monthly billing actions. */
|
||||||
@Module
|
@Module
|
||||||
|
@ -31,6 +35,15 @@ public final class BillingModule {
|
||||||
|
|
||||||
private static final String CLOUD_PLATFORM_SCOPE =
|
private static final String CLOUD_PLATFORM_SCOPE =
|
||||||
"https://www.googleapis.com/auth/cloud-platform";
|
"https://www.googleapis.com/auth/cloud-platform";
|
||||||
|
static final String BILLING_QUEUE = "billing";
|
||||||
|
static final String PARAM_JOB_ID = "jobId";
|
||||||
|
|
||||||
|
/** Provides the invoicing Dataflow jobId enqueued by {@link GenerateInvoicesAction}. */
|
||||||
|
@Provides
|
||||||
|
@Parameter(PARAM_JOB_ID)
|
||||||
|
static String provideJobId(HttpServletRequest req) {
|
||||||
|
return extractRequiredParameter(req, PARAM_JOB_ID);
|
||||||
|
}
|
||||||
|
|
||||||
/** Constructs a {@link Dataflow} API client with default settings. */
|
/** Constructs a {@link Dataflow} API client with default settings. */
|
||||||
@Provides
|
@Provides
|
||||||
|
|
|
@ -22,6 +22,9 @@ import com.google.api.services.dataflow.Dataflow;
|
||||||
import com.google.api.services.dataflow.model.LaunchTemplateParameters;
|
import com.google.api.services.dataflow.model.LaunchTemplateParameters;
|
||||||
import com.google.api.services.dataflow.model.LaunchTemplateResponse;
|
import com.google.api.services.dataflow.model.LaunchTemplateResponse;
|
||||||
import com.google.api.services.dataflow.model.RuntimeEnvironment;
|
import com.google.api.services.dataflow.model.RuntimeEnvironment;
|
||||||
|
import com.google.appengine.api.taskqueue.QueueFactory;
|
||||||
|
import com.google.appengine.api.taskqueue.TaskOptions;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.net.MediaType;
|
import com.google.common.net.MediaType;
|
||||||
import google.registry.config.RegistryConfig.Config;
|
import google.registry.config.RegistryConfig.Config;
|
||||||
import google.registry.request.Action;
|
import google.registry.request.Action;
|
||||||
|
@ -30,9 +33,12 @@ import google.registry.request.auth.Auth;
|
||||||
import google.registry.util.FormattingLogger;
|
import google.registry.util.FormattingLogger;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import javax.inject.Inject;
|
import javax.inject.Inject;
|
||||||
|
import org.joda.time.Duration;
|
||||||
|
import org.joda.time.YearMonth;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Invokes the {@code InvoicingPipeline} beam template via the REST api.
|
* Invokes the {@code InvoicingPipeline} beam template via the REST api, and enqueues the {@link
|
||||||
|
* PublishInvoicesAction} to publish the subsequent output.
|
||||||
*
|
*
|
||||||
* <p>This action runs the {@link google.registry.beam.InvoicingPipeline} beam template, staged at
|
* <p>This action runs the {@link google.registry.beam.InvoicingPipeline} beam template, staged at
|
||||||
* gs://<projectId>-beam/templates/invoicing. The pipeline then generates invoices for the month and
|
* gs://<projectId>-beam/templates/invoicing. The pipeline then generates invoices for the month and
|
||||||
|
@ -43,25 +49,35 @@ public class GenerateInvoicesAction implements Runnable {
|
||||||
|
|
||||||
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
||||||
|
|
||||||
@Inject @Config("projectId") String projectId;
|
@Inject
|
||||||
@Inject @Config("apacheBeamBucketUrl") String beamBucketUrl;
|
@Config("projectId")
|
||||||
|
String projectId;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
@Config("apacheBeamBucketUrl")
|
||||||
|
String beamBucketUrl;
|
||||||
|
|
||||||
|
@Inject YearMonth yearMonth;
|
||||||
@Inject Dataflow dataflow;
|
@Inject Dataflow dataflow;
|
||||||
@Inject Response response;
|
@Inject Response response;
|
||||||
@Inject GenerateInvoicesAction() {}
|
|
||||||
|
@Inject
|
||||||
|
GenerateInvoicesAction() {}
|
||||||
|
|
||||||
static final String PATH = "/_dr/task/generateInvoices";
|
static final String PATH = "/_dr/task/generateInvoices";
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
logger.info("Launching dataflow job");
|
logger.infofmt("Launching invoicing pipeline for %s", yearMonth);
|
||||||
try {
|
try {
|
||||||
LaunchTemplateParameters params =
|
LaunchTemplateParameters params =
|
||||||
new LaunchTemplateParameters()
|
new LaunchTemplateParameters()
|
||||||
.setJobName("test-invoicing")
|
.setJobName(String.format("invoicing-%s", yearMonth))
|
||||||
.setEnvironment(
|
.setEnvironment(
|
||||||
new RuntimeEnvironment()
|
new RuntimeEnvironment()
|
||||||
.setZone("us-east1-c")
|
.setZone("us-east1-c")
|
||||||
.setTempLocation(beamBucketUrl + "/temp"));
|
.setTempLocation(beamBucketUrl + "/temporary"))
|
||||||
|
.setParameters(ImmutableMap.of("yearMonth", yearMonth.toString("yyyy-MM")));
|
||||||
LaunchTemplateResponse launchResponse =
|
LaunchTemplateResponse launchResponse =
|
||||||
dataflow
|
dataflow
|
||||||
.projects()
|
.projects()
|
||||||
|
@ -70,6 +86,14 @@ public class GenerateInvoicesAction implements Runnable {
|
||||||
.setGcsPath(beamBucketUrl + "/templates/invoicing")
|
.setGcsPath(beamBucketUrl + "/templates/invoicing")
|
||||||
.execute();
|
.execute();
|
||||||
logger.infofmt("Got response: %s", launchResponse.getJob().toPrettyString());
|
logger.infofmt("Got response: %s", launchResponse.getJob().toPrettyString());
|
||||||
|
String jobId = launchResponse.getJob().getId();
|
||||||
|
TaskOptions uploadTask =
|
||||||
|
TaskOptions.Builder.withUrl(PublishInvoicesAction.PATH)
|
||||||
|
.method(TaskOptions.Method.POST)
|
||||||
|
// Dataflow jobs tend to take about 10 minutes to complete.
|
||||||
|
.countdownMillis(Duration.standardMinutes(10).getMillis())
|
||||||
|
.param(BillingModule.PARAM_JOB_ID, jobId);
|
||||||
|
QueueFactory.getQueue(BillingModule.BILLING_QUEUE).add(uploadTask);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
logger.warningfmt("Template Launch failed due to: %s", e.getMessage());
|
logger.warningfmt("Template Launch failed due to: %s", e.getMessage());
|
||||||
response.setStatus(SC_INTERNAL_SERVER_ERROR);
|
response.setStatus(SC_INTERNAL_SERVER_ERROR);
|
||||||
|
|
89
java/google/registry/billing/PublishInvoicesAction.java
Normal file
89
java/google/registry/billing/PublishInvoicesAction.java
Normal file
|
@ -0,0 +1,89 @@
|
||||||
|
// 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.billing;
|
||||||
|
|
||||||
|
import static google.registry.request.Action.Method.POST;
|
||||||
|
import static javax.servlet.http.HttpServletResponse.SC_INTERNAL_SERVER_ERROR;
|
||||||
|
import static javax.servlet.http.HttpServletResponse.SC_NOT_MODIFIED;
|
||||||
|
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
|
||||||
|
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||||
|
|
||||||
|
import com.google.api.services.dataflow.Dataflow;
|
||||||
|
import com.google.api.services.dataflow.model.Job;
|
||||||
|
import com.google.common.net.MediaType;
|
||||||
|
import google.registry.config.RegistryConfig.Config;
|
||||||
|
import google.registry.request.Action;
|
||||||
|
import google.registry.request.Parameter;
|
||||||
|
import google.registry.request.Response;
|
||||||
|
import google.registry.request.auth.Auth;
|
||||||
|
import google.registry.util.FormattingLogger;
|
||||||
|
import java.io.IOException;
|
||||||
|
import javax.inject.Inject;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Uploads the results of the {@link google.registry.beam.InvoicingPipeline}.
|
||||||
|
*
|
||||||
|
* <p>This relies on the retry semantics in {@code queue.xml} to ensure proper upload, in spite of
|
||||||
|
* fluctuations in generation timing.
|
||||||
|
*
|
||||||
|
* @see <a href=https://cloud.google.com/dataflow/docs/reference/rest/v1b3/projects.jobs#Job.JobState>
|
||||||
|
* Job States</a>
|
||||||
|
*/
|
||||||
|
@Action(path = PublishInvoicesAction.PATH, method = POST, auth = Auth.AUTH_INTERNAL_OR_ADMIN)
|
||||||
|
public class PublishInvoicesAction implements Runnable {
|
||||||
|
|
||||||
|
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
||||||
|
private static final String JOB_DONE = "JOB_STATE_DONE";
|
||||||
|
private static final String JOB_FAILED = "JOB_STATE_FAILED";
|
||||||
|
|
||||||
|
@Inject @Config("projectId") String projectId;
|
||||||
|
@Inject @Parameter(BillingModule.PARAM_JOB_ID) String jobId;
|
||||||
|
@Inject Dataflow dataflow;
|
||||||
|
@Inject Response response;
|
||||||
|
@Inject PublishInvoicesAction() {}
|
||||||
|
|
||||||
|
static final String PATH = "/_dr/task/publishInvoices";
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
logger.info("Starting publish job.");
|
||||||
|
try {
|
||||||
|
Job job = dataflow.projects().jobs().get(projectId, jobId).execute();
|
||||||
|
String state = job.getCurrentState();
|
||||||
|
switch (state) {
|
||||||
|
case JOB_DONE:
|
||||||
|
logger.infofmt("Dataflow job %s finished successfully.", jobId);
|
||||||
|
response.setStatus(SC_OK);
|
||||||
|
// TODO(larryruili): Implement upload logic.
|
||||||
|
break;
|
||||||
|
case JOB_FAILED:
|
||||||
|
logger.severefmt("Dataflow job %s finished unsuccessfully.", jobId);
|
||||||
|
// Return a 'success' code to stop task queue retry.
|
||||||
|
response.setStatus(SC_NO_CONTENT);
|
||||||
|
// TODO(larryruili): Implement failure response.
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
logger.infofmt("Job in non-terminal state %s, retrying:", state);
|
||||||
|
response.setStatus(SC_NOT_MODIFIED);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
logger.warningfmt("Template Launch failed due to: %s", e.getMessage());
|
||||||
|
response.setStatus(SC_INTERNAL_SERVER_ERROR);
|
||||||
|
response.setContentType(MediaType.PLAIN_TEXT_UTF_8);
|
||||||
|
response.setPayload(String.format("Template launch failed: %s", e.getMessage()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -68,6 +68,15 @@
|
||||||
<url-pattern>/_dr/task/generateInvoices</url-pattern>
|
<url-pattern>/_dr/task/generateInvoices</url-pattern>
|
||||||
</servlet-mapping>
|
</servlet-mapping>
|
||||||
|
|
||||||
|
<!--
|
||||||
|
Emails the month's invoice CSV to the internal billing team, and publishes
|
||||||
|
the detail reports to the individual registrars' drive accounts.
|
||||||
|
-->
|
||||||
|
<servlet-mapping>
|
||||||
|
<servlet-name>backend-servlet</servlet-name>
|
||||||
|
<url-pattern>/_dr/task/publishInvoices</url-pattern>
|
||||||
|
</servlet-mapping>
|
||||||
|
|
||||||
<!-- ICANN Monthly Reporting -->
|
<!-- ICANN Monthly Reporting -->
|
||||||
|
|
||||||
<!--
|
<!--
|
||||||
|
|
|
@ -128,6 +128,18 @@
|
||||||
</retry-parameters>
|
</retry-parameters>
|
||||||
</queue>
|
</queue>
|
||||||
|
|
||||||
|
<!-- Queue for tasks to generate or upload monthly invoices. -->
|
||||||
|
<queue>
|
||||||
|
<name>billing</name>
|
||||||
|
<rate>1/m</rate>
|
||||||
|
<max-concurrent-requests>1</max-concurrent-requests>
|
||||||
|
<retry-parameters>
|
||||||
|
<task-retry-limit>5</task-retry-limit>
|
||||||
|
<min-backoff-seconds>180</min-backoff-seconds>
|
||||||
|
<max-backoff-seconds>180</max-backoff-seconds>
|
||||||
|
</retry-parameters>
|
||||||
|
</queue>
|
||||||
|
|
||||||
<!-- Queue for tasks that communicate with TMCH MarksDB webserver. -->
|
<!-- Queue for tasks that communicate with TMCH MarksDB webserver. -->
|
||||||
<!-- TODO(b/17623181): Delete this once the queue implementation is live and working. -->
|
<!-- TODO(b/17623181): Delete this once the queue implementation is live and working. -->
|
||||||
<queue>
|
<queue>
|
||||||
|
|
|
@ -17,6 +17,7 @@ package google.registry.module.backend;
|
||||||
import static google.registry.model.registry.Registries.assertTldExists;
|
import static google.registry.model.registry.Registries.assertTldExists;
|
||||||
import static google.registry.model.registry.Registries.assertTldsExist;
|
import static google.registry.model.registry.Registries.assertTldsExist;
|
||||||
import static google.registry.request.RequestParameters.extractOptionalDatetimeParameter;
|
import static google.registry.request.RequestParameters.extractOptionalDatetimeParameter;
|
||||||
|
import static google.registry.request.RequestParameters.extractOptionalParameter;
|
||||||
import static google.registry.request.RequestParameters.extractRequiredParameter;
|
import static google.registry.request.RequestParameters.extractRequiredParameter;
|
||||||
import static google.registry.request.RequestParameters.extractSetOfParameters;
|
import static google.registry.request.RequestParameters.extractSetOfParameters;
|
||||||
|
|
||||||
|
@ -24,11 +25,16 @@ import com.google.common.collect.ImmutableSet;
|
||||||
import dagger.Module;
|
import dagger.Module;
|
||||||
import dagger.Provides;
|
import dagger.Provides;
|
||||||
import google.registry.batch.ExpandRecurringBillingEventsAction;
|
import google.registry.batch.ExpandRecurringBillingEventsAction;
|
||||||
|
import google.registry.request.HttpException.BadRequestException;
|
||||||
import google.registry.request.Parameter;
|
import google.registry.request.Parameter;
|
||||||
import google.registry.request.RequestParameters;
|
import google.registry.request.RequestParameters;
|
||||||
|
import google.registry.util.Clock;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import javax.servlet.http.HttpServletRequest;
|
import javax.servlet.http.HttpServletRequest;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.YearMonth;
|
||||||
|
import org.joda.time.format.DateTimeFormat;
|
||||||
|
import org.joda.time.format.DateTimeFormatter;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Dagger module for injecting common settings for all Backend tasks.
|
* Dagger module for injecting common settings for all Backend tasks.
|
||||||
|
@ -36,6 +42,8 @@ import org.joda.time.DateTime;
|
||||||
@Module
|
@Module
|
||||||
public class BackendModule {
|
public class BackendModule {
|
||||||
|
|
||||||
|
public static final String PARAM_YEAR_MONTH = "yearMonth";
|
||||||
|
|
||||||
@Provides
|
@Provides
|
||||||
@Parameter(RequestParameters.PARAM_TLD)
|
@Parameter(RequestParameters.PARAM_TLD)
|
||||||
static String provideTld(HttpServletRequest req) {
|
static String provideTld(HttpServletRequest req) {
|
||||||
|
@ -56,4 +64,30 @@ public class BackendModule {
|
||||||
return extractOptionalDatetimeParameter(
|
return extractOptionalDatetimeParameter(
|
||||||
req, ExpandRecurringBillingEventsAction.PARAM_CURSOR_TIME);
|
req, ExpandRecurringBillingEventsAction.PARAM_CURSOR_TIME);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Extracts an optional YearMonth in yyyy-MM format from the request. */
|
||||||
|
@Provides
|
||||||
|
@Parameter(PARAM_YEAR_MONTH)
|
||||||
|
static Optional<YearMonth> provideYearMonthOptional(HttpServletRequest req) {
|
||||||
|
DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy-MM");
|
||||||
|
Optional<String> optionalYearMonthStr = extractOptionalParameter(req, PARAM_YEAR_MONTH);
|
||||||
|
try {
|
||||||
|
return optionalYearMonthStr.map(s -> YearMonth.parse(s, formatter));
|
||||||
|
} catch (IllegalArgumentException e) {
|
||||||
|
throw new BadRequestException(
|
||||||
|
String.format(
|
||||||
|
"yearMonth must be in yyyy-MM format, got %s instead",
|
||||||
|
optionalYearMonthStr.orElse("UNSPECIFIED YEARMONTH")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides the yearMonth in yyyy-MM format, if not specified in the request, defaults to one
|
||||||
|
* month prior to run time.
|
||||||
|
*/
|
||||||
|
@Provides
|
||||||
|
static YearMonth provideYearMonth(
|
||||||
|
@Parameter(PARAM_YEAR_MONTH) Optional<YearMonth> yearMonthOptional, Clock clock) {
|
||||||
|
return yearMonthOptional.orElseGet(() -> new YearMonth(clock.nowUtc().minusMonths(1)));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,6 +30,7 @@ import google.registry.batch.ResaveAllEppResourcesAction;
|
||||||
import google.registry.batch.VerifyEntityIntegrityAction;
|
import google.registry.batch.VerifyEntityIntegrityAction;
|
||||||
import google.registry.billing.BillingModule;
|
import google.registry.billing.BillingModule;
|
||||||
import google.registry.billing.GenerateInvoicesAction;
|
import google.registry.billing.GenerateInvoicesAction;
|
||||||
|
import google.registry.billing.PublishInvoicesAction;
|
||||||
import google.registry.cron.CommitLogFanoutAction;
|
import google.registry.cron.CommitLogFanoutAction;
|
||||||
import google.registry.cron.CronModule;
|
import google.registry.cron.CronModule;
|
||||||
import google.registry.cron.TldFanoutAction;
|
import google.registry.cron.TldFanoutAction;
|
||||||
|
@ -147,6 +148,7 @@ interface BackendRequestComponent {
|
||||||
TmchDnlAction tmchDnlAction();
|
TmchDnlAction tmchDnlAction();
|
||||||
TmchSmdrlAction tmchSmdrlAction();
|
TmchSmdrlAction tmchSmdrlAction();
|
||||||
UpdateSnapshotViewAction updateSnapshotViewAction();
|
UpdateSnapshotViewAction updateSnapshotViewAction();
|
||||||
|
PublishInvoicesAction uploadInvoicesAction();
|
||||||
VerifyEntityIntegrityAction verifyEntityIntegrityAction();
|
VerifyEntityIntegrityAction verifyEntityIntegrityAction();
|
||||||
|
|
||||||
@Subcomponent.Builder
|
@Subcomponent.Builder
|
||||||
|
|
|
@ -28,7 +28,6 @@ import dagger.Provides;
|
||||||
import google.registry.bigquery.BigqueryConnection;
|
import google.registry.bigquery.BigqueryConnection;
|
||||||
import google.registry.request.HttpException.BadRequestException;
|
import google.registry.request.HttpException.BadRequestException;
|
||||||
import google.registry.request.Parameter;
|
import google.registry.request.Parameter;
|
||||||
import google.registry.util.Clock;
|
|
||||||
import google.registry.util.SendEmailService;
|
import google.registry.util.SendEmailService;
|
||||||
import java.lang.annotation.Documented;
|
import java.lang.annotation.Documented;
|
||||||
import java.lang.annotation.Retention;
|
import java.lang.annotation.Retention;
|
||||||
|
@ -38,7 +37,6 @@ import javax.servlet.http.HttpServletRequest;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.joda.time.YearMonth;
|
import org.joda.time.YearMonth;
|
||||||
import org.joda.time.format.DateTimeFormat;
|
import org.joda.time.format.DateTimeFormat;
|
||||||
import org.joda.time.format.DateTimeFormatter;
|
|
||||||
|
|
||||||
/** Module for dependencies required by ICANN monthly transactions/activity reporting. */
|
/** Module for dependencies required by ICANN monthly transactions/activity reporting. */
|
||||||
@Module
|
@Module
|
||||||
|
@ -50,7 +48,6 @@ public final class IcannReportingModule {
|
||||||
ACTIVITY
|
ACTIVITY
|
||||||
}
|
}
|
||||||
|
|
||||||
static final String PARAM_YEAR_MONTH = "yearMonth";
|
|
||||||
static final String PARAM_SUBDIR = "subdir";
|
static final String PARAM_SUBDIR = "subdir";
|
||||||
static final String PARAM_REPORT_TYPE = "reportType";
|
static final String PARAM_REPORT_TYPE = "reportType";
|
||||||
static final String ICANN_REPORTING_DATA_SET = "icann_reporting";
|
static final String ICANN_REPORTING_DATA_SET = "icann_reporting";
|
||||||
|
@ -59,29 +56,6 @@ public final class IcannReportingModule {
|
||||||
private static final String DEFAULT_SUBDIR = "icann/monthly";
|
private static final String DEFAULT_SUBDIR = "icann/monthly";
|
||||||
private static final String BIGQUERY_SCOPE = "https://www.googleapis.com/auth/cloud-platform";
|
private static final String BIGQUERY_SCOPE = "https://www.googleapis.com/auth/cloud-platform";
|
||||||
|
|
||||||
/** Extracts an optional YearMonth in yyyy-MM format from the request. */
|
|
||||||
@Provides
|
|
||||||
@Parameter(PARAM_YEAR_MONTH)
|
|
||||||
static Optional<YearMonth> provideYearMonthOptional(HttpServletRequest req) {
|
|
||||||
DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy-MM");
|
|
||||||
Optional<String> optionalYearMonthStr = extractOptionalParameter(req, PARAM_YEAR_MONTH);
|
|
||||||
try {
|
|
||||||
return optionalYearMonthStr.map(s -> YearMonth.parse(s, formatter));
|
|
||||||
} catch (IllegalArgumentException e) {
|
|
||||||
throw new BadRequestException(
|
|
||||||
String.format(
|
|
||||||
"yearMonth must be in yyyy-MM format, got %s instead",
|
|
||||||
optionalYearMonthStr.orElse("UNSPECIFIED YEARMONTH")));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Provides the yearMonth in yyyy-MM format, defaults to one month prior to run time. */
|
|
||||||
@Provides
|
|
||||||
static YearMonth provideYearMonth(
|
|
||||||
@Parameter(PARAM_YEAR_MONTH) Optional<YearMonth> yearMonthOptional, Clock clock) {
|
|
||||||
return yearMonthOptional.orElseGet(() -> new YearMonth(clock.nowUtc().minusMonths(1)));
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Provides an optional subdirectory to store/upload reports to, extracted from the request. */
|
/** Provides an optional subdirectory to store/upload reports to, extracted from the request. */
|
||||||
@Provides
|
@Provides
|
||||||
@Parameter(PARAM_SUBDIR)
|
@Parameter(PARAM_SUBDIR)
|
||||||
|
|
|
@ -35,7 +35,6 @@ import google.registry.util.Retrier;
|
||||||
import javax.inject.Inject;
|
import javax.inject.Inject;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.joda.time.YearMonth;
|
import org.joda.time.YearMonth;
|
||||||
import org.joda.time.format.DateTimeFormat;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Action that generates monthly ICANN activity and transactions reports.
|
* Action that generates monthly ICANN activity and transactions reports.
|
||||||
|
@ -98,9 +97,6 @@ public final class IcannReportingStagingAction implements Runnable {
|
||||||
TaskOptions uploadTask = TaskOptions.Builder.withUrl(IcannReportingUploadAction.PATH)
|
TaskOptions uploadTask = TaskOptions.Builder.withUrl(IcannReportingUploadAction.PATH)
|
||||||
.method(Method.POST)
|
.method(Method.POST)
|
||||||
.countdownMillis(Duration.standardMinutes(2).getMillis())
|
.countdownMillis(Duration.standardMinutes(2).getMillis())
|
||||||
.param(
|
|
||||||
IcannReportingModule.PARAM_YEAR_MONTH,
|
|
||||||
DateTimeFormat.forPattern("yyyy-MM").print(yearMonth))
|
|
||||||
.param(IcannReportingModule.PARAM_SUBDIR, subdir);
|
.param(IcannReportingModule.PARAM_SUBDIR, subdir);
|
||||||
QueueFactory.getQueue(CRON_QUEUE).add(uploadTask);
|
QueueFactory.getQueue(CRON_QUEUE).add(uploadTask);
|
||||||
return null;
|
return null;
|
||||||
|
|
|
@ -116,7 +116,7 @@ public class BillingEventTest {
|
||||||
@Test
|
@Test
|
||||||
public void testGenerateBillingEventFilename() {
|
public void testGenerateBillingEventFilename() {
|
||||||
BillingEvent event = BillingEvent.parseFromRecord(schemaAndRecord);
|
BillingEvent event = BillingEvent.parseFromRecord(schemaAndRecord);
|
||||||
assertThat(event.toFilename()).isEqualTo("myRegistrar_test");
|
assertThat(event.toFilename("2017-10")).isEqualTo("invoice_details_2017-10_myRegistrar_test");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Map.Entry;
|
||||||
import org.apache.beam.runners.direct.DirectRunner;
|
import org.apache.beam.runners.direct.DirectRunner;
|
||||||
import org.apache.beam.sdk.options.PipelineOptions;
|
import org.apache.beam.sdk.options.PipelineOptions;
|
||||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||||
|
import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
|
||||||
import org.apache.beam.sdk.testing.TestPipeline;
|
import org.apache.beam.sdk.testing.TestPipeline;
|
||||||
import org.apache.beam.sdk.transforms.Create;
|
import org.apache.beam.sdk.transforms.Create;
|
||||||
import org.apache.beam.sdk.values.PCollection;
|
import org.apache.beam.sdk.values.PCollection;
|
||||||
|
@ -124,17 +125,17 @@ public class InvoicingPipelineTest {
|
||||||
/** Returns a map from filename to expected contents for detail reports. */
|
/** Returns a map from filename to expected contents for detail reports. */
|
||||||
private ImmutableMap<String, ImmutableList<String>> getExpectedDetailReportMap() {
|
private ImmutableMap<String, ImmutableList<String>> getExpectedDetailReportMap() {
|
||||||
return ImmutableMap.of(
|
return ImmutableMap.of(
|
||||||
"theRegistrar_test.csv",
|
"invoice_details_2017-10_theRegistrar_test.csv",
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
"1,2017-10-04 00:00:00 UTC,2017-10-04 00:00:00 UTC,theRegistrar,234,"
|
"1,2017-10-04 00:00:00 UTC,2017-10-04 00:00:00 UTC,theRegistrar,234,"
|
||||||
+ "test,RENEW,mydomain2.test,REPO-ID,3,USD,20.50,",
|
+ "test,RENEW,mydomain2.test,REPO-ID,3,USD,20.50,",
|
||||||
"1,2017-10-04 00:00:00 UTC,2017-10-04 00:00:00 UTC,theRegistrar,234,"
|
"1,2017-10-04 00:00:00 UTC,2017-10-04 00:00:00 UTC,theRegistrar,234,"
|
||||||
+ "test,RENEW,mydomain.test,REPO-ID,3,USD,20.50,"),
|
+ "test,RENEW,mydomain.test,REPO-ID,3,USD,20.50,"),
|
||||||
"theRegistrar_hello.csv",
|
"invoice_details_2017-10_theRegistrar_hello.csv",
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
"1,2017-10-02 00:00:00 UTC,2017-09-29 00:00:00 UTC,theRegistrar,234,"
|
"1,2017-10-02 00:00:00 UTC,2017-09-29 00:00:00 UTC,theRegistrar,234,"
|
||||||
+ "hello,CREATE,mydomain3.hello,REPO-ID,5,JPY,70.75,"),
|
+ "hello,CREATE,mydomain3.hello,REPO-ID,5,JPY,70.75,"),
|
||||||
"googledomains_test.csv",
|
"invoice_details_2017-10_googledomains_test.csv",
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
"1,2017-10-04 00:00:00 UTC,2017-10-04 00:00:00 UTC,googledomains,456,"
|
"1,2017-10-04 00:00:00 UTC,2017-10-04 00:00:00 UTC,googledomains,456,"
|
||||||
+ "test,RENEW,mydomain4.test,REPO-ID,1,USD,20.50,"));
|
+ "test,RENEW,mydomain4.test,REPO-ID,1,USD,20.50,"));
|
||||||
|
@ -154,7 +155,7 @@ public class InvoicingPipelineTest {
|
||||||
public void testEndToEndPipeline_generatesExpectedFiles() throws Exception {
|
public void testEndToEndPipeline_generatesExpectedFiles() throws Exception {
|
||||||
ImmutableList<BillingEvent> inputRows = getInputEvents();
|
ImmutableList<BillingEvent> inputRows = getInputEvents();
|
||||||
PCollection<BillingEvent> input = p.apply(Create.of(inputRows));
|
PCollection<BillingEvent> input = p.apply(Create.of(inputRows));
|
||||||
invoicingPipeline.applyTerminalTransforms(input);
|
invoicingPipeline.applyTerminalTransforms(input, StaticValueProvider.of("2017-10"));
|
||||||
p.run();
|
p.run();
|
||||||
|
|
||||||
for (Entry<String, ImmutableList<String>> entry : getExpectedDetailReportMap().entrySet()) {
|
for (Entry<String, ImmutableList<String>> entry : getExpectedDetailReportMap().entrySet()) {
|
||||||
|
@ -166,7 +167,7 @@ public class InvoicingPipelineTest {
|
||||||
.containsExactlyElementsIn(entry.getValue());
|
.containsExactlyElementsIn(entry.getValue());
|
||||||
}
|
}
|
||||||
|
|
||||||
ImmutableList<String> overallInvoice = resultFileContents("overall_invoice.csv");
|
ImmutableList<String> overallInvoice = resultFileContents("CRR-INV-2017-10.csv");
|
||||||
assertThat(overallInvoice.get(0))
|
assertThat(overallInvoice.get(0))
|
||||||
.isEqualTo(
|
.isEqualTo(
|
||||||
"StartDate,EndDate,ProductAccountKey,Amount,AmountCurrency,BillingProductCode,"
|
"StartDate,EndDate,ProductAccountKey,Amount,AmountCurrency,BillingProductCode,"
|
||||||
|
@ -179,7 +180,7 @@ public class InvoicingPipelineTest {
|
||||||
/** Returns the text contents of a file under the beamBucket/results directory. */
|
/** Returns the text contents of a file under the beamBucket/results directory. */
|
||||||
private ImmutableList<String> resultFileContents(String filename) throws Exception {
|
private ImmutableList<String> resultFileContents(String filename) throws Exception {
|
||||||
File resultFile =
|
File resultFile =
|
||||||
new File(String.format("%s/results/%s", invoicingPipeline.beamBucket, filename));
|
new File(String.format("%s/results/%s", tempFolder.getRoot().getAbsolutePath(), filename));
|
||||||
return ImmutableList.copyOf(
|
return ImmutableList.copyOf(
|
||||||
ResourceUtils.readResourceUtf8(resultFile.toURI().toURL()).split("\n"));
|
ResourceUtils.readResourceUtf8(resultFile.toURI().toURL()).split("\n"));
|
||||||
}
|
}
|
||||||
|
|
|
@ -15,6 +15,7 @@
|
||||||
package google.registry.beam;
|
package google.registry.beam;
|
||||||
|
|
||||||
import static com.google.common.truth.Truth.assertThat;
|
import static com.google.common.truth.Truth.assertThat;
|
||||||
|
import static org.mockito.Matchers.any;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
@ -35,11 +36,11 @@ public class InvoicingUtilsTest {
|
||||||
@Test
|
@Test
|
||||||
public void testDestinationFunction_generatesProperFileParams() {
|
public void testDestinationFunction_generatesProperFileParams() {
|
||||||
SerializableFunction<BillingEvent, Params> destinationFunction =
|
SerializableFunction<BillingEvent, Params> destinationFunction =
|
||||||
InvoicingUtils.makeDestinationFunction("my/directory");
|
InvoicingUtils.makeDestinationFunction("my/directory", StaticValueProvider.of("2017-10"));
|
||||||
|
|
||||||
BillingEvent billingEvent = mock(BillingEvent.class);
|
BillingEvent billingEvent = mock(BillingEvent.class);
|
||||||
// We mock BillingEvent to make the test independent of the implementation of toFilename()
|
// We mock BillingEvent to make the test independent of the implementation of toFilename()
|
||||||
when(billingEvent.toFilename()).thenReturn("registrar_tld");
|
when(billingEvent.toFilename(any())).thenReturn("invoice_details_2017-10_registrar_tld");
|
||||||
|
|
||||||
assertThat(destinationFunction.apply(billingEvent))
|
assertThat(destinationFunction.apply(billingEvent))
|
||||||
.isEqualTo(
|
.isEqualTo(
|
||||||
|
@ -47,7 +48,8 @@ public class InvoicingUtilsTest {
|
||||||
.withShardTemplate("")
|
.withShardTemplate("")
|
||||||
.withSuffix(".csv")
|
.withSuffix(".csv")
|
||||||
.withBaseFilename(
|
.withBaseFilename(
|
||||||
FileBasedSink.convertToFileResourceIfPossible("my/directory/registrar_tld")));
|
FileBasedSink.convertToFileResourceIfPossible(
|
||||||
|
"my/directory/invoice_details_2017-10_registrar_tld")));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -18,6 +18,7 @@ java_library(
|
||||||
"@com_google_guava",
|
"@com_google_guava",
|
||||||
"@com_google_truth",
|
"@com_google_truth",
|
||||||
"@com_google_truth_extensions_truth_java8_extension",
|
"@com_google_truth_extensions_truth_java8_extension",
|
||||||
|
"@javax_servlet_api",
|
||||||
"@joda_time",
|
"@joda_time",
|
||||||
"@junit",
|
"@junit",
|
||||||
"@org_apache_beam_runners_direct_java",
|
"@org_apache_beam_runners_direct_java",
|
||||||
|
|
|
@ -15,6 +15,7 @@
|
||||||
package google.registry.billing;
|
package google.registry.billing;
|
||||||
|
|
||||||
import static com.google.common.truth.Truth.assertThat;
|
import static com.google.common.truth.Truth.assertThat;
|
||||||
|
import static google.registry.testing.TaskQueueHelper.assertTasksEnqueued;
|
||||||
import static org.mockito.Matchers.any;
|
import static org.mockito.Matchers.any;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.verify;
|
import static org.mockito.Mockito.verify;
|
||||||
|
@ -28,9 +29,14 @@ import com.google.api.services.dataflow.model.Job;
|
||||||
import com.google.api.services.dataflow.model.LaunchTemplateParameters;
|
import com.google.api.services.dataflow.model.LaunchTemplateParameters;
|
||||||
import com.google.api.services.dataflow.model.LaunchTemplateResponse;
|
import com.google.api.services.dataflow.model.LaunchTemplateResponse;
|
||||||
import com.google.api.services.dataflow.model.RuntimeEnvironment;
|
import com.google.api.services.dataflow.model.RuntimeEnvironment;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import google.registry.testing.AppEngineRule;
|
||||||
import google.registry.testing.FakeResponse;
|
import google.registry.testing.FakeResponse;
|
||||||
|
import google.registry.testing.TaskQueueHelper.TaskMatcher;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import org.joda.time.YearMonth;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
import org.junit.runners.JUnit4;
|
import org.junit.runners.JUnit4;
|
||||||
|
@ -46,6 +52,9 @@ public class GenerateInvoicesActionTest {
|
||||||
GenerateInvoicesAction action;
|
GenerateInvoicesAction action;
|
||||||
FakeResponse response = new FakeResponse();
|
FakeResponse response = new FakeResponse();
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public final AppEngineRule appEngine = AppEngineRule.builder().withTaskQueue().build();
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void initializeObjects() throws Exception {
|
public void initializeObjects() throws Exception {
|
||||||
when(dataflow.projects()).thenReturn(projects);
|
when(dataflow.projects()).thenReturn(projects);
|
||||||
|
@ -53,13 +62,16 @@ public class GenerateInvoicesActionTest {
|
||||||
when(templates.launch(any(String.class), any(LaunchTemplateParameters.class)))
|
when(templates.launch(any(String.class), any(LaunchTemplateParameters.class)))
|
||||||
.thenReturn(launch);
|
.thenReturn(launch);
|
||||||
when(launch.setGcsPath(any(String.class))).thenReturn(launch);
|
when(launch.setGcsPath(any(String.class))).thenReturn(launch);
|
||||||
when(launch.execute()).thenReturn(new LaunchTemplateResponse().setJob(new Job()));
|
Job job = new Job();
|
||||||
|
job.setId("12345");
|
||||||
|
when(launch.execute()).thenReturn(new LaunchTemplateResponse().setJob(job));
|
||||||
|
|
||||||
action = new GenerateInvoicesAction();
|
action = new GenerateInvoicesAction();
|
||||||
action.dataflow = dataflow;
|
action.dataflow = dataflow;
|
||||||
action.response = response;
|
action.response = response;
|
||||||
action.projectId = "test-project";
|
action.projectId = "test-project";
|
||||||
action.beamBucketUrl = "gs://test-project-beam";
|
action.beamBucketUrl = "gs://test-project-beam";
|
||||||
|
action.yearMonth = new YearMonth(2017, 10);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -67,15 +79,23 @@ public class GenerateInvoicesActionTest {
|
||||||
action.run();
|
action.run();
|
||||||
LaunchTemplateParameters expectedParams =
|
LaunchTemplateParameters expectedParams =
|
||||||
new LaunchTemplateParameters()
|
new LaunchTemplateParameters()
|
||||||
.setJobName("test-invoicing")
|
.setJobName("invoicing-2017-10")
|
||||||
.setEnvironment(
|
.setEnvironment(
|
||||||
new RuntimeEnvironment()
|
new RuntimeEnvironment()
|
||||||
.setZone("us-east1-c")
|
.setZone("us-east1-c")
|
||||||
.setTempLocation("gs://test-project-beam/temp"));
|
.setTempLocation("gs://test-project-beam/temporary"))
|
||||||
|
.setParameters(ImmutableMap.of("yearMonth", "2017-10"));
|
||||||
verify(templates).launch("test-project", expectedParams);
|
verify(templates).launch("test-project", expectedParams);
|
||||||
verify(launch).setGcsPath("gs://test-project-beam/templates/invoicing");
|
verify(launch).setGcsPath("gs://test-project-beam/templates/invoicing");
|
||||||
assertThat(response.getStatus()).isEqualTo(200);
|
assertThat(response.getStatus()).isEqualTo(200);
|
||||||
assertThat(response.getPayload()).isEqualTo("Launched dataflow template.");
|
assertThat(response.getPayload()).isEqualTo("Launched dataflow template.");
|
||||||
|
|
||||||
|
TaskMatcher matcher =
|
||||||
|
new TaskMatcher()
|
||||||
|
.url("/_dr/task/publishInvoices")
|
||||||
|
.method("POST")
|
||||||
|
.param("jobId", "12345");
|
||||||
|
assertTasksEnqueued("billing", matcher);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -0,0 +1,95 @@
|
||||||
|
// 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.billing;
|
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat;
|
||||||
|
import static javax.servlet.http.HttpServletResponse.SC_INTERNAL_SERVER_ERROR;
|
||||||
|
import static javax.servlet.http.HttpServletResponse.SC_NOT_MODIFIED;
|
||||||
|
import static javax.servlet.http.HttpServletResponse.SC_NO_CONTENT;
|
||||||
|
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
import com.google.api.services.dataflow.Dataflow;
|
||||||
|
import com.google.api.services.dataflow.Dataflow.Projects;
|
||||||
|
import com.google.api.services.dataflow.Dataflow.Projects.Jobs;
|
||||||
|
import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get;
|
||||||
|
import com.google.api.services.dataflow.model.Job;
|
||||||
|
import com.google.common.net.MediaType;
|
||||||
|
import google.registry.testing.FakeResponse;
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.JUnit4;
|
||||||
|
|
||||||
|
@RunWith(JUnit4.class)
|
||||||
|
public class PublishInvoicesActionTest {
|
||||||
|
|
||||||
|
private final Dataflow dataflow = mock(Dataflow.class);
|
||||||
|
private final Projects projects = mock(Projects.class);
|
||||||
|
private final Jobs jobs = mock(Jobs.class);
|
||||||
|
private final Get get = mock(Get.class);
|
||||||
|
|
||||||
|
private Job expectedJob;
|
||||||
|
private FakeResponse response;
|
||||||
|
private PublishInvoicesAction uploadAction;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void initializeObjects() throws Exception {
|
||||||
|
when(dataflow.projects()).thenReturn(projects);
|
||||||
|
when(projects.jobs()).thenReturn(jobs);
|
||||||
|
when(jobs.get("test-project", "12345")).thenReturn(get);
|
||||||
|
expectedJob = new Job();
|
||||||
|
when(get.execute()).thenReturn(expectedJob);
|
||||||
|
|
||||||
|
uploadAction = new PublishInvoicesAction();
|
||||||
|
uploadAction.projectId = "test-project";
|
||||||
|
uploadAction.jobId = "12345";
|
||||||
|
uploadAction.dataflow = dataflow;
|
||||||
|
response = new FakeResponse();
|
||||||
|
uploadAction.response = response;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testJobDone_returnsSuccess() {
|
||||||
|
expectedJob.setCurrentState("JOB_STATE_DONE");
|
||||||
|
uploadAction.run();
|
||||||
|
assertThat(response.getStatus()).isEqualTo(SC_OK);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testJobFailed_returnsNonRetriableResponse() {
|
||||||
|
expectedJob.setCurrentState("JOB_STATE_FAILED");
|
||||||
|
uploadAction.run();
|
||||||
|
assertThat(response.getStatus()).isEqualTo(SC_NO_CONTENT);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testJobIndeterminate_returnsRetriableResponse() {
|
||||||
|
expectedJob.setCurrentState("JOB_STATE_RUNNING");
|
||||||
|
uploadAction.run();
|
||||||
|
assertThat(response.getStatus()).isEqualTo(SC_NOT_MODIFIED);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIOException_returnsFailureMessage() throws Exception {
|
||||||
|
when(get.execute()).thenThrow(new IOException("expected"));
|
||||||
|
uploadAction.run();
|
||||||
|
assertThat(response.getStatus()).isEqualTo(SC_INTERNAL_SERVER_ERROR);
|
||||||
|
assertThat(response.getContentType()).isEqualTo(MediaType.PLAIN_TEXT_UTF_8);
|
||||||
|
assertThat(response.getPayload()).isEqualTo("Template launch failed: expected");
|
||||||
|
}
|
||||||
|
}
|
|
@ -22,6 +22,7 @@ java_library(
|
||||||
"@com_google_truth",
|
"@com_google_truth",
|
||||||
"@com_google_truth_extensions_truth_java8_extension",
|
"@com_google_truth_extensions_truth_java8_extension",
|
||||||
"@javax_servlet_api",
|
"@javax_servlet_api",
|
||||||
|
"@joda_time",
|
||||||
"@junit",
|
"@junit",
|
||||||
"@org_mockito_all",
|
"@org_mockito_all",
|
||||||
],
|
],
|
||||||
|
|
|
@ -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.module.backend;
|
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat;
|
||||||
|
import static google.registry.testing.JUnitBackports.expectThrows;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
import google.registry.request.HttpException.BadRequestException;
|
||||||
|
import google.registry.testing.FakeClock;
|
||||||
|
import google.registry.util.Clock;
|
||||||
|
import java.util.Optional;
|
||||||
|
import javax.servlet.http.HttpServletRequest;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.YearMonth;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.JUnit4;
|
||||||
|
|
||||||
|
/** Unit tests for {@link BackendModule}. */
|
||||||
|
@RunWith(JUnit4.class)
|
||||||
|
public class BackendModuleTest {
|
||||||
|
|
||||||
|
private HttpServletRequest req = mock(HttpServletRequest.class);
|
||||||
|
private Clock clock;
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
clock = new FakeClock(DateTime.parse("2017-07-01TZ"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEmptyYearMonthParameter_returnsEmptyYearMonthOptional() {
|
||||||
|
when(req.getParameter("yearMonth")).thenReturn("");
|
||||||
|
assertThat(BackendModule.provideYearMonthOptional(req)).isEqualTo(Optional.empty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInvalidYearMonthParameter_throwsException() {
|
||||||
|
when(req.getParameter("yearMonth")).thenReturn("201705");
|
||||||
|
BadRequestException thrown =
|
||||||
|
expectThrows(
|
||||||
|
BadRequestException.class, () -> BackendModule.provideYearMonthOptional(req));
|
||||||
|
assertThat(thrown)
|
||||||
|
.hasMessageThat()
|
||||||
|
.contains("yearMonth must be in yyyy-MM format, got 201705 instead");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEmptyYearMonth_returnsLastMonth() {
|
||||||
|
assertThat(BackendModule.provideYearMonth(Optional.empty(), clock))
|
||||||
|
.isEqualTo(new YearMonth(2017, 6));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGivenYearMonth_returnsThatMonth() {
|
||||||
|
assertThat(BackendModule.provideYearMonth(Optional.of(new YearMonth(2017, 5)), clock))
|
||||||
|
.isEqualTo(new YearMonth(2017, 5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -28,6 +28,7 @@ PATH CLASS METHOD
|
||||||
/_dr/task/nordnVerify NordnVerifyAction POST y INTERNAL APP IGNORED
|
/_dr/task/nordnVerify NordnVerifyAction POST y INTERNAL APP IGNORED
|
||||||
/_dr/task/pollBigqueryJob BigqueryPollJobAction GET,POST y INTERNAL APP IGNORED
|
/_dr/task/pollBigqueryJob BigqueryPollJobAction GET,POST y INTERNAL APP IGNORED
|
||||||
/_dr/task/publishDnsUpdates PublishDnsUpdatesAction POST y INTERNAL APP IGNORED
|
/_dr/task/publishDnsUpdates PublishDnsUpdatesAction POST y INTERNAL APP IGNORED
|
||||||
|
/_dr/task/publishInvoices PublishInvoicesAction POST n INTERNAL,API APP ADMIN
|
||||||
/_dr/task/rdeReport RdeReportAction POST n INTERNAL APP IGNORED
|
/_dr/task/rdeReport RdeReportAction POST n INTERNAL APP IGNORED
|
||||||
/_dr/task/rdeStaging RdeStagingAction GET,POST n INTERNAL APP IGNORED
|
/_dr/task/rdeStaging RdeStagingAction GET,POST n INTERNAL APP IGNORED
|
||||||
/_dr/task/rdeUpload RdeUploadAction POST n INTERNAL APP IGNORED
|
/_dr/task/rdeUpload RdeUploadAction POST n INTERNAL APP IGNORED
|
||||||
|
|
|
@ -16,18 +16,11 @@ package google.registry.reporting;
|
||||||
|
|
||||||
import static com.google.common.truth.Truth.assertThat;
|
import static com.google.common.truth.Truth.assertThat;
|
||||||
import static google.registry.testing.JUnitBackports.expectThrows;
|
import static google.registry.testing.JUnitBackports.expectThrows;
|
||||||
import static org.mockito.Mockito.mock;
|
|
||||||
import static org.mockito.Mockito.when;
|
|
||||||
|
|
||||||
import google.registry.reporting.IcannReportingModule.ReportType;
|
import google.registry.reporting.IcannReportingModule.ReportType;
|
||||||
import google.registry.request.HttpException.BadRequestException;
|
import google.registry.request.HttpException.BadRequestException;
|
||||||
import google.registry.testing.FakeClock;
|
|
||||||
import google.registry.util.Clock;
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import javax.servlet.http.HttpServletRequest;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
import org.joda.time.YearMonth;
|
import org.joda.time.YearMonth;
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
import org.junit.runners.JUnit4;
|
import org.junit.runners.JUnit4;
|
||||||
|
@ -36,42 +29,6 @@ import org.junit.runners.JUnit4;
|
||||||
@RunWith(JUnit4.class)
|
@RunWith(JUnit4.class)
|
||||||
public class IcannReportingModuleTest {
|
public class IcannReportingModuleTest {
|
||||||
|
|
||||||
HttpServletRequest req = mock(HttpServletRequest.class);
|
|
||||||
Clock clock;
|
|
||||||
@Before
|
|
||||||
public void setUp() {
|
|
||||||
clock = new FakeClock(DateTime.parse("2017-07-01TZ"));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testEmptyYearMonthParameter_returnsEmptyYearMonthOptional() {
|
|
||||||
when(req.getParameter("yearMonth")).thenReturn("");
|
|
||||||
assertThat(IcannReportingModule.provideYearMonthOptional(req)).isEqualTo(Optional.empty());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testInvalidYearMonthParameter_throwsException() {
|
|
||||||
when(req.getParameter("yearMonth")).thenReturn("201705");
|
|
||||||
BadRequestException thrown =
|
|
||||||
expectThrows(
|
|
||||||
BadRequestException.class, () -> IcannReportingModule.provideYearMonthOptional(req));
|
|
||||||
assertThat(thrown)
|
|
||||||
.hasMessageThat()
|
|
||||||
.contains("yearMonth must be in yyyy-MM format, got 201705 instead");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testEmptyYearMonth_returnsLastMonth() {
|
|
||||||
assertThat(IcannReportingModule.provideYearMonth(Optional.empty(), clock))
|
|
||||||
.isEqualTo(new YearMonth(2017, 6));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testGivenYearMonth_returnsThatMonth() {
|
|
||||||
assertThat(IcannReportingModule.provideYearMonth(Optional.of(new YearMonth(2017, 5)), clock))
|
|
||||||
.isEqualTo(new YearMonth(2017, 5));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testEmptySubDir_returnsDefaultSubdir() {
|
public void testEmptySubDir_returnsDefaultSubdir() {
|
||||||
assertThat(IcannReportingModule.provideSubdir(Optional.empty(), new YearMonth(2017, 6)))
|
assertThat(IcannReportingModule.provideSubdir(Optional.empty(), new YearMonth(2017, 6)))
|
||||||
|
|
|
@ -62,12 +62,11 @@ public class IcannReportingStagingActionTest {
|
||||||
when(stager.stageReports(ReportType.TRANSACTIONS)).thenReturn(ImmutableList.of("c", "d"));
|
when(stager.stageReports(ReportType.TRANSACTIONS)).thenReturn(ImmutableList.of("c", "d"));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void assertUploadTaskEnqueued(String yearMonth, String subDir) throws Exception {
|
private static void assertUploadTaskEnqueued(String subDir) throws Exception {
|
||||||
TaskMatcher matcher =
|
TaskMatcher matcher =
|
||||||
new TaskMatcher()
|
new TaskMatcher()
|
||||||
.url("/_dr/task/icannReportingUpload")
|
.url("/_dr/task/icannReportingUpload")
|
||||||
.method("POST")
|
.method("POST")
|
||||||
.param("yearMonth", yearMonth)
|
|
||||||
.param("subdir", subDir);
|
.param("subdir", subDir);
|
||||||
assertTasksEnqueued("retryable-cron-tasks", matcher);
|
assertTasksEnqueued("retryable-cron-tasks", matcher);
|
||||||
}
|
}
|
||||||
|
@ -94,7 +93,7 @@ public class IcannReportingStagingActionTest {
|
||||||
.emailResults(
|
.emailResults(
|
||||||
"ICANN Monthly report staging summary [SUCCESS]",
|
"ICANN Monthly report staging summary [SUCCESS]",
|
||||||
"Completed staging the following 2 ICANN reports:\na\nb");
|
"Completed staging the following 2 ICANN reports:\na\nb");
|
||||||
assertUploadTaskEnqueued("2017-06", "default/dir");
|
assertUploadTaskEnqueued("default/dir");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -109,7 +108,7 @@ public class IcannReportingStagingActionTest {
|
||||||
.emailResults(
|
.emailResults(
|
||||||
"ICANN Monthly report staging summary [SUCCESS]",
|
"ICANN Monthly report staging summary [SUCCESS]",
|
||||||
"Completed staging the following 4 ICANN reports:\na\nb\nc\nd");
|
"Completed staging the following 4 ICANN reports:\na\nb\nc\nd");
|
||||||
assertUploadTaskEnqueued("2017-06", "default/dir");
|
assertUploadTaskEnqueued("default/dir");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -127,7 +126,7 @@ public class IcannReportingStagingActionTest {
|
||||||
.emailResults(
|
.emailResults(
|
||||||
"ICANN Monthly report staging summary [SUCCESS]",
|
"ICANN Monthly report staging summary [SUCCESS]",
|
||||||
"Completed staging the following 4 ICANN reports:\na\nb\nc\nd");
|
"Completed staging the following 4 ICANN reports:\na\nb\nc\nd");
|
||||||
assertUploadTaskEnqueued("2017-06", "default/dir");
|
assertUploadTaskEnqueued("default/dir");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue