Add GenerateSpec11Action and SafeBrowsing evaluation

This adds actual subdomain verification via the SafeBrowsing API to the Spec11
pipeline, as well as on-the-fly KMS decryption via the GenerateSpec11Action to
securely store our API key in source code.

Testing the interaction becomes difficult due to serialization requirements, and will be significantly expanded in the next cl. For now, it verifies basic end-to-end pipeline behavior.

-------------
Created by MOE: https://github.com/google/moe
MOE_MIGRATED_REVID=208092942
This commit is contained in:
larryruili 2018-08-09 12:32:30 -07:00 committed by jianglai
parent f554ace51b
commit 33ee7de457
29 changed files with 767 additions and 26 deletions

View file

@ -17,10 +17,14 @@ java_library(
"@com_google_flogger_system_backend", "@com_google_flogger_system_backend",
"@com_google_guava", "@com_google_guava",
"@javax_inject", "@javax_inject",
"@joda_time",
"@org_apache_avro", "@org_apache_avro",
"@org_apache_beam_runners_direct_java", "@org_apache_beam_runners_direct_java",
"@org_apache_beam_runners_google_cloud_dataflow_java", "@org_apache_beam_runners_google_cloud_dataflow_java",
"@org_apache_beam_sdks_java_core", "@org_apache_beam_sdks_java_core",
"@org_apache_beam_sdks_java_io_google_cloud_platform", "@org_apache_beam_sdks_java_io_google_cloud_platform",
"@org_apache_httpcomponents_httpclient",
"@org_apache_httpcomponents_httpcore",
"@org_json",
], ],
) )

View file

@ -0,0 +1,239 @@
// 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.spec11;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.http.HttpStatus.SC_OK;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import com.google.common.flogger.FluentLogger;
import com.google.common.io.CharStreams;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.Serializable;
import java.net.URISyntaxException;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.function.Supplier;
import org.apache.beam.sdk.options.ValueProvider;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.values.KV;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.utils.URIBuilder;
import org.apache.http.entity.ByteArrayEntity;
import org.apache.http.entity.ContentType;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import org.apache.http.protocol.HTTP;
import org.joda.time.Instant;
import org.json.JSONArray;
import org.json.JSONException;
import org.json.JSONObject;
/** Utilities and Beam {@code PTransforms} for interacting with the SafeBrowsing API. */
public class SafeBrowsingTransforms {
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
/** The URL to send SafeBrowsing API calls (POSTS) to. */
private static final String SAFE_BROWSING_URL =
"https://safebrowsing.googleapis.com/v4/threatMatches:find";
/**
* {@link DoFn} mapping a {@link Subdomain} to its evaluation report from SafeBrowsing.
*
* <p>Refer to the Lookup API documentation for the request/response format and other details.
*
* @see <a href=https://developers.google.com/safe-browsing/v4/lookup-api>Lookup API</a>
*/
static class EvaluateSafeBrowsingFn extends DoFn<Subdomain, KV<Subdomain, String>> {
/**
* Max number of urls we can check in a single query.
*
* <p>The actual max is 500, but we leave a small gap in case of concurrency errors.
*/
private static final int BATCH_SIZE = 490;
/** Provides the SafeBrowsing API key at runtime. */
private final ValueProvider<String> apiKeyProvider;
/**
* Maps a subdomain's HTTP URL to its corresponding {@link Subdomain} to facilitate batching
* SafeBrowsing API requests.
*/
private final Map<String, Subdomain> subdomainBuffer = new LinkedHashMap<>(BATCH_SIZE);
/**
* Provides the HTTP client we use to interact with the SafeBrowsing API.
*
* <p>This is a supplier to enable mocking out the connection in unit tests while maintaining a
* serializable field.
*/
private final Supplier<CloseableHttpClient> closeableHttpClientSupplier;
/**
* Constructs a {@link EvaluateSafeBrowsingFn} that gets its API key from the given provider.
*
* <p>We need to dual-cast the closeableHttpClientSupplier lambda because all {@code DoFn}
* member variables need to be serializable. The (Supplier & Serializable) dual cast is safe
* because class methods are generally serializable, especially a static function such as {@link
* HttpClients#createDefault()}.
*
* @param apiKeyProvider provides the SafeBrowsing API key from {@code KMS} at runtime
*/
@SuppressWarnings("unchecked")
EvaluateSafeBrowsingFn(ValueProvider<String> apiKeyProvider) {
this.apiKeyProvider = apiKeyProvider;
this.closeableHttpClientSupplier = (Supplier & Serializable) HttpClients::createDefault;
}
/**
* Constructs a {@link EvaluateSafeBrowsingFn}, allowing us to swap out the HTTP client supplier
* for testing.
*
* @param clientSupplier a serializable CloseableHttpClient supplier
*/
@VisibleForTesting
@SuppressWarnings("unchecked")
EvaluateSafeBrowsingFn(
ValueProvider<String> apiKeyProvider, Supplier<CloseableHttpClient> clientSupplier) {
this.apiKeyProvider = apiKeyProvider;
this.closeableHttpClientSupplier = clientSupplier;
}
/** Evaluates any buffered {@link Subdomain} objects upon completing the bundle. */
@FinishBundle
public void finishBundle(FinishBundleContext context) {
if (!subdomainBuffer.isEmpty()) {
ImmutableList<KV<Subdomain, String>> results = evaluateAndFlush();
results.forEach((kv) -> context.output(kv, Instant.now(), GlobalWindow.INSTANCE));
}
}
/**
* Buffers {@link Subdomain} objects until we reach the batch size, then bulk-evaluate the URLs
* with the SafeBrowsing API.
*/
@ProcessElement
public void processElement(ProcessContext context) {
Subdomain subdomain = context.element();
// We put HTTP URLs into the buffer because the API requires specifying the protocol.
subdomainBuffer.put(
String.format("http://%s", subdomain.fullyQualifiedDomainName()), subdomain);
if (subdomainBuffer.size() >= BATCH_SIZE) {
ImmutableList<KV<Subdomain, String>> results = evaluateAndFlush();
results.forEach(context::output);
}
}
/**
* Evaluates all {@link Subdomain} objects in the buffer and returns a list of key-value pairs
* from {@link Subdomain} to its SafeBrowsing report.
*
* <p>If a {@link Subdomain} is safe according to the API, it will not emit a report.
*/
private ImmutableList<KV<Subdomain, String>> evaluateAndFlush() {
ImmutableList.Builder<KV<Subdomain, String>> resultBuilder = new ImmutableList.Builder<>();
try {
URIBuilder uriBuilder = new URIBuilder(SAFE_BROWSING_URL);
// Add the API key param
uriBuilder.addParameter("key", apiKeyProvider.get());
HttpPost httpPost = new HttpPost(uriBuilder.build());
httpPost.addHeader(HTTP.CONTENT_TYPE, ContentType.APPLICATION_JSON.toString());
JSONObject requestBody = createRequestBody();
httpPost.setEntity(new ByteArrayEntity(requestBody.toString().getBytes(UTF_8)));
try (CloseableHttpClient client = closeableHttpClientSupplier.get();
CloseableHttpResponse response = client.execute(httpPost)) {
processResponse(response, resultBuilder);
}
} catch (URISyntaxException | JSONException e) {
// TODO(b/112354588): also send an alert e-mail to indicate the pipeline failed
logger.atSevere().withCause(e).log(
"Caught parsing error during execution, skipping batch.");
} catch (IOException e) {
logger.atSevere().withCause(e).log("Caught IOException during processing, skipping batch.");
} finally {
// Flush the buffer
subdomainBuffer.clear();
}
return resultBuilder.build();
}
/** Creates a JSON object matching the request format for the SafeBrowsing API. */
private JSONObject createRequestBody() throws JSONException {
// Accumulate all domain names to evaluate.
JSONArray threatArray = new JSONArray();
for (String fullyQualifiedDomainName : subdomainBuffer.keySet()) {
threatArray.put(new JSONObject().put("url", fullyQualifiedDomainName));
}
// Construct the JSON request body
return new JSONObject()
.put(
"client",
new JSONObject().put("clientId", "domainregistry").put("clientVersion", "0.0.1"))
.put(
"threatInfo",
new JSONObject()
.put(
"threatTypes",
new JSONArray()
.put("MALWARE")
.put("SOCIAL_ENGINEERING")
.put("UNWANTED_SOFTWARE"))
.put("platformTypes", new JSONArray().put("ANY_PLATFORM"))
.put("threatEntryTypes", new JSONArray().put("URL"))
.put("threatEntries", threatArray));
}
/**
* Iterates through all threat matches in the API response and adds them to the resultBuilder.
*/
private void processResponse(
CloseableHttpResponse response, ImmutableList.Builder<KV<Subdomain, String>> resultBuilder)
throws JSONException, IOException {
int statusCode = response.getStatusLine().getStatusCode();
if (statusCode != SC_OK) {
logger.atWarning().log("Got unexpected status code %s from response", statusCode);
} else {
// Unpack the response body
JSONObject responseBody =
new JSONObject(
CharStreams.toString(
new InputStreamReader(response.getEntity().getContent(), UTF_8)));
logger.atInfo().log("Got response: %s", responseBody.toString());
if (responseBody.length() == 0) {
logger.atInfo().log("Response was empty, no threats detected");
} else {
// Emit all Subdomains with their API results.
JSONArray threatMatches = responseBody.getJSONArray("matches");
for (int i = 0; i < threatMatches.length(); i++) {
JSONObject match = threatMatches.getJSONObject(i);
String url = match.getJSONObject("threat").getString("url");
resultBuilder.add(KV.of(subdomainBuffer.get(url), match.toString()));
}
}
}
}
}
}

View file

@ -14,6 +14,7 @@
package google.registry.beam.spec11; package google.registry.beam.spec11;
import google.registry.beam.spec11.SafeBrowsingTransforms.EvaluateSafeBrowsingFn;
import google.registry.config.RegistryConfig.Config; import google.registry.config.RegistryConfig.Config;
import java.io.Serializable; import java.io.Serializable;
import javax.inject.Inject; import javax.inject.Inject;
@ -26,7 +27,8 @@ 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.transforms.Count; import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Sample;
import org.apache.beam.sdk.transforms.ToString; import org.apache.beam.sdk.transforms.ToString;
import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection;
@ -70,13 +72,25 @@ public class Spec11Pipeline implements Serializable {
/** /**
* Sets the yearMonth we generate invoices for. * Sets the yearMonth we generate invoices for.
* *
* <p>This is implicitly set when executing the Dataflow template, by specifying the 'yearMonth * <p>This is implicitly set when executing the Dataflow template, by specifying the "yearMonth"
* parameter. * parameter.
*/ */
void setYearMonth(ValueProvider<String> value); void setYearMonth(ValueProvider<String> value);
/** Returns the SafeBrowsing API key we use to evaluate subdomain health. */
@Description("The API key we use to access the SafeBrowsing API.")
ValueProvider<String> getSafeBrowsingApiKey();
/**
* Sets the SafeBrowsing API key we use.
*
* <p>This is implicitly set when executing the Dataflow template, by specifying the
* "safeBrowsingApiKey" parameter.
*/
void setSafeBrowsingApiKey(ValueProvider<String> value);
} }
/** Deploys the spec11 pipeline as a template on GCS, for a given projectID and GCS bucket. */ /** Deploys the spec11 pipeline as a template on GCS. */
public void deploy() { public void deploy() {
// We can't store options as a member variable due to serialization concerns. // We can't store options as a member variable due to serialization concerns.
Spec11PipelineOptions options = PipelineOptionsFactory.as(Spec11PipelineOptions.class); Spec11PipelineOptions options = PipelineOptionsFactory.as(Spec11PipelineOptions.class);
@ -85,6 +99,7 @@ public class Spec11Pipeline implements Serializable {
// This causes p.run() to stage the pipeline as a template on GCS, as opposed to running it. // This causes p.run() to stage the pipeline as a template on GCS, as opposed to running it.
options.setTemplateLocation(spec11TemplateUrl); options.setTemplateLocation(spec11TemplateUrl);
options.setStagingLocation(beamStagingUrl); options.setStagingLocation(beamStagingUrl);
Pipeline p = Pipeline.create(options); Pipeline p = Pipeline.create(options);
PCollection<Subdomain> domains = PCollection<Subdomain> domains =
p.apply( p.apply(
@ -97,16 +112,24 @@ public class Spec11Pipeline implements Serializable {
.usingStandardSql() .usingStandardSql()
.withoutValidation() .withoutValidation()
.withTemplateCompatibility()); .withTemplateCompatibility());
countDomainsAndOutputResults(domains); evaluateUrlHealth(domains, new EvaluateSafeBrowsingFn(options.getSafeBrowsingApiKey()));
p.run(); p.run();
} }
/** Globally count the number of elements and output the results to GCS. */ /**
void countDomainsAndOutputResults(PCollection<Subdomain> domains) { * Evaluate each {@link Subdomain} URL via the SafeBrowsing API.
// TODO(b/111545355): Actually process each domain with the SafeBrowsing API *
* <p>This is factored out to facilitate testing.
*/
void evaluateUrlHealth(
PCollection<Subdomain> domains, EvaluateSafeBrowsingFn evaluateSafeBrowsingFn) {
domains domains
.apply("Count number of subdomains", Count.globally()) // TODO(b/111545355): Remove this limiter once we're confident we won't go over quota.
.apply("Convert global count to string", ToString.elements()) .apply(
"Get just a few representative samples for now, don't want to overwhelm our quota",
Sample.any(1000))
.apply("Run through SafeBrowsingAPI", ParDo.of(evaluateSafeBrowsingFn))
.apply("Convert results to string", ToString.elements())
.apply( .apply(
"Output to text file", "Output to text file",
TextIO.write() TextIO.write()
@ -115,4 +138,5 @@ public class Spec11Pipeline implements Serializable {
.withoutSharding() .withoutSharding()
.withHeader("HELLO WORLD")); .withHeader("HELLO WORLD"));
} }
} }

View file

@ -531,6 +531,18 @@ public final class RegistryConfig {
return beamBucketUrl + "/templates/spec11"; return beamBucketUrl + "/templates/spec11";
} }
/**
* Returns the default job zone to run Apache Beam (Cloud Dataflow) jobs in.
*
* @see google.registry.reporting.billing.GenerateInvoicesAction
* @see google.registry.reporting.spec11.GenerateSpec11ReportAction
*/
@Provides
@Config("defaultJobZone")
public static String provideDefaultJobZone(RegistryConfigSettings config) {
return config.beam.defaultJobZone;
}
/** /**
* Returns the URL of the GCS location we store jar dependencies for beam pipelines. * Returns the URL of the GCS location we store jar dependencies for beam pipelines.
* *

View file

@ -32,6 +32,7 @@ public class RegistryConfigSettings {
public RegistrarConsole registrarConsole; public RegistrarConsole registrarConsole;
public Monitoring monitoring; public Monitoring monitoring;
public Misc misc; public Misc misc;
public Beam beam;
public Kms kms; public Kms kms;
public RegistryTool registryTool; public RegistryTool registryTool;
@ -96,6 +97,11 @@ public class RegistryConfigSettings {
public String projectId; public String projectId;
} }
/** Configuration for Apache Beam (Cloud Dataflow). */
public static class Beam {
public String defaultJobZone;
}
/** Configuration for Cloud DNS. */ /** Configuration for Cloud DNS. */
public static class CloudDns { public static class CloudDns {
public String rootUrl; public String rootUrl;

View file

@ -246,6 +246,10 @@ misc:
# hosts from being used on domains. # hosts from being used on domains.
asyncDeleteDelaySeconds: 90 asyncDeleteDelaySeconds: 90
beam:
# The default zone to run Apache Beam (Cloud Dataflow) jobs in.
defaultJobZone: us-east1-c
kms: kms:
# GCP project containing the KMS keyring. Should only be used for KMS in # GCP project containing the KMS keyring. Should only be used for KMS in
# order to keep a simple locked down IAM configuration. # order to keep a simple locked down IAM configuration.

View file

@ -107,6 +107,14 @@
<url-pattern>/_dr/task/icannReportingUpload</url-pattern> <url-pattern>/_dr/task/icannReportingUpload</url-pattern>
</servlet-mapping> </servlet-mapping>
<!--
Generates the Spec11 report for the month, storing it on GCS.
-->
<servlet-mapping>
<servlet-name>backend-servlet</servlet-name>
<url-pattern>/_dr/task/generateSpec11</url-pattern>
</servlet-mapping>
<!-- Trademark Clearinghouse --> <!-- Trademark Clearinghouse -->
<!-- Downloads TMCH DNL data from MarksDB. --> <!-- Downloads TMCH DNL data from MarksDB. -->

View file

@ -106,6 +106,7 @@ public final class DummyKeyringModule {
"not a real key", "not a real key",
"not a real key", "not a real key",
"not a real password", "not a real password",
"not a real API key",
"not a real login", "not a real login",
"not a real password", "not a real password",
"not a real login", "not a real login",

View file

@ -34,6 +34,7 @@ public final class InMemoryKeyring implements Keyring {
private final String rdeSshClientPublicKey; private final String rdeSshClientPublicKey;
private final String rdeSshClientPrivateKey; private final String rdeSshClientPrivateKey;
private final String icannReportingPassword; private final String icannReportingPassword;
private final String safeBrowsingAPIKey;
private final String marksdbDnlLogin; private final String marksdbDnlLogin;
private final String marksdbLordnPassword; private final String marksdbLordnPassword;
private final String marksdbSmdrlLogin; private final String marksdbSmdrlLogin;
@ -48,6 +49,7 @@ public final class InMemoryKeyring implements Keyring {
String rdeSshClientPublicKey, String rdeSshClientPublicKey,
String rdeSshClientPrivateKey, String rdeSshClientPrivateKey,
String icannReportingPassword, String icannReportingPassword,
String safeBrowsingAPIKey,
String marksdbDnlLogin, String marksdbDnlLogin,
String marksdbLordnPassword, String marksdbLordnPassword,
String marksdbSmdrlLogin, String marksdbSmdrlLogin,
@ -70,6 +72,7 @@ public final class InMemoryKeyring implements Keyring {
this.rdeSshClientPublicKey = checkNotNull(rdeSshClientPublicKey, "rdeSshClientPublicKey"); this.rdeSshClientPublicKey = checkNotNull(rdeSshClientPublicKey, "rdeSshClientPublicKey");
this.rdeSshClientPrivateKey = checkNotNull(rdeSshClientPrivateKey, "rdeSshClientPrivateKey"); this.rdeSshClientPrivateKey = checkNotNull(rdeSshClientPrivateKey, "rdeSshClientPrivateKey");
this.icannReportingPassword = checkNotNull(icannReportingPassword, "icannReportingPassword"); this.icannReportingPassword = checkNotNull(icannReportingPassword, "icannReportingPassword");
this.safeBrowsingAPIKey = checkNotNull(safeBrowsingAPIKey, "safeBrowsingAPIKey");
this.marksdbDnlLogin = checkNotNull(marksdbDnlLogin, "marksdbDnlLogin"); this.marksdbDnlLogin = checkNotNull(marksdbDnlLogin, "marksdbDnlLogin");
this.marksdbLordnPassword = checkNotNull(marksdbLordnPassword, "marksdbLordnPassword"); this.marksdbLordnPassword = checkNotNull(marksdbLordnPassword, "marksdbLordnPassword");
this.marksdbSmdrlLogin = checkNotNull(marksdbSmdrlLogin, "marksdbSmdrlLogin"); this.marksdbSmdrlLogin = checkNotNull(marksdbSmdrlLogin, "marksdbSmdrlLogin");
@ -122,6 +125,11 @@ public final class InMemoryKeyring implements Keyring {
} }
@Override @Override
public String getSafeBrowsingAPIKey() {
return safeBrowsingAPIKey;
}
@Override
public String getMarksdbDnlLogin() { public String getMarksdbDnlLogin() {
return marksdbDnlLogin; return marksdbDnlLogin;
} }

View file

@ -115,6 +115,12 @@ public final class KeyModule {
return keyring.getRdeSshClientPublicKey(); return keyring.getRdeSshClientPublicKey();
} }
@Provides
@Key("safeBrowsingAPIKey")
static String provideSafeBrowsingAPIKey(Keyring keyring) {
return keyring.getSafeBrowsingAPIKey();
}
@Provides @Provides
@Key("jsonCredential") @Key("jsonCredential")
static String provideJsonCredential(Keyring keyring) { static String provideJsonCredential(Keyring keyring) {

View file

@ -115,6 +115,13 @@ public interface Keyring extends AutoCloseable {
*/ */
String getRdeSshClientPrivateKey(); String getRdeSshClientPrivateKey();
/**
* Returns the API key for accessing the SafeBrowsing API.
*
* @see google.registry.reporting.spec11.GenerateSpec11ReportAction
*/
String getSafeBrowsingAPIKey();
/** /**
* Returns password to be used when uploading reports to ICANN. * Returns password to be used when uploading reports to ICANN.
* *

View file

@ -64,6 +64,7 @@ public class KmsKeyring implements Keyring {
} }
enum StringKeyLabel { enum StringKeyLabel {
SAFE_BROWSING_API_KEY,
ICANN_REPORTING_PASSWORD_STRING, ICANN_REPORTING_PASSWORD_STRING,
JSON_CREDENTIAL_STRING, JSON_CREDENTIAL_STRING,
MARKSDB_DNL_LOGIN_STRING, MARKSDB_DNL_LOGIN_STRING,
@ -124,6 +125,11 @@ public class KmsKeyring implements Keyring {
return getString(StringKeyLabel.RDE_SSH_CLIENT_PRIVATE_STRING); return getString(StringKeyLabel.RDE_SSH_CLIENT_PRIVATE_STRING);
} }
@Override
public String getSafeBrowsingAPIKey() {
return getString(StringKeyLabel.SAFE_BROWSING_API_KEY);
}
@Override @Override
public String getIcannReportingPassword() { public String getIcannReportingPassword() {
return getString(StringKeyLabel.ICANN_REPORTING_PASSWORD_STRING); return getString(StringKeyLabel.ICANN_REPORTING_PASSWORD_STRING);

View file

@ -31,6 +31,7 @@ import static google.registry.keyring.kms.KmsKeyring.StringKeyLabel.MARKSDB_LORD
import static google.registry.keyring.kms.KmsKeyring.StringKeyLabel.MARKSDB_SMDRL_LOGIN_STRING; import static google.registry.keyring.kms.KmsKeyring.StringKeyLabel.MARKSDB_SMDRL_LOGIN_STRING;
import static google.registry.keyring.kms.KmsKeyring.StringKeyLabel.RDE_SSH_CLIENT_PRIVATE_STRING; import static google.registry.keyring.kms.KmsKeyring.StringKeyLabel.RDE_SSH_CLIENT_PRIVATE_STRING;
import static google.registry.keyring.kms.KmsKeyring.StringKeyLabel.RDE_SSH_CLIENT_PUBLIC_STRING; import static google.registry.keyring.kms.KmsKeyring.StringKeyLabel.RDE_SSH_CLIENT_PUBLIC_STRING;
import static google.registry.keyring.kms.KmsKeyring.StringKeyLabel.SAFE_BROWSING_API_KEY;
import static google.registry.model.ofy.ObjectifyService.ofy; import static google.registry.model.ofy.ObjectifyService.ofy;
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull; import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
@ -95,6 +96,10 @@ public final class KmsUpdater {
return setString(asciiPrivateKey, RDE_SSH_CLIENT_PRIVATE_STRING); return setString(asciiPrivateKey, RDE_SSH_CLIENT_PRIVATE_STRING);
} }
public KmsUpdater setSafeBrowsingAPIKey(String apiKey) {
return setString(apiKey, SAFE_BROWSING_API_KEY);
}
public KmsUpdater setIcannReportingPassword(String password) { public KmsUpdater setIcannReportingPassword(String password) {
return setString(password, ICANN_REPORTING_PASSWORD_STRING); return setString(password, ICANN_REPORTING_PASSWORD_STRING);
} }

View file

@ -32,6 +32,7 @@ java_library(
"//java/google/registry/reporting", "//java/google/registry/reporting",
"//java/google/registry/reporting/billing", "//java/google/registry/reporting/billing",
"//java/google/registry/reporting/icann", "//java/google/registry/reporting/icann",
"//java/google/registry/reporting/spec11",
"//java/google/registry/request", "//java/google/registry/request",
"//java/google/registry/request:modules", "//java/google/registry/request:modules",
"//java/google/registry/request/auth", "//java/google/registry/request/auth",

View file

@ -75,6 +75,7 @@ import google.registry.reporting.billing.PublishInvoicesAction;
import google.registry.reporting.icann.IcannReportingModule; import google.registry.reporting.icann.IcannReportingModule;
import google.registry.reporting.icann.IcannReportingStagingAction; import google.registry.reporting.icann.IcannReportingStagingAction;
import google.registry.reporting.icann.IcannReportingUploadAction; import google.registry.reporting.icann.IcannReportingUploadAction;
import google.registry.reporting.spec11.GenerateSpec11ReportAction;
import google.registry.request.RequestComponentBuilder; import google.registry.request.RequestComponentBuilder;
import google.registry.request.RequestModule; import google.registry.request.RequestModule;
import google.registry.request.RequestScope; import google.registry.request.RequestScope;
@ -128,6 +129,7 @@ interface BackendRequestComponent {
ExportReservedTermsAction exportReservedTermsAction(); ExportReservedTermsAction exportReservedTermsAction();
ExportSnapshotAction exportSnapshotAction(); ExportSnapshotAction exportSnapshotAction();
GenerateInvoicesAction generateInvoicesAction(); GenerateInvoicesAction generateInvoicesAction();
GenerateSpec11ReportAction generateSpec11ReportAction();
IcannReportingStagingAction icannReportingStagingAction(); IcannReportingStagingAction icannReportingStagingAction();
IcannReportingUploadAction icannReportingUploadAction(); IcannReportingUploadAction icannReportingUploadAction();
LoadSnapshotAction loadSnapshotAction(); LoadSnapshotAction loadSnapshotAction();

View file

@ -57,6 +57,7 @@ public class GenerateInvoicesAction implements Runnable {
private final String projectId; private final String projectId;
private final String beamBucketUrl; private final String beamBucketUrl;
private final String invoiceTemplateUrl; private final String invoiceTemplateUrl;
private final String jobZone;
private final boolean shouldPublish; private final boolean shouldPublish;
private final YearMonth yearMonth; private final YearMonth yearMonth;
private final Dataflow dataflow; private final Dataflow dataflow;
@ -68,6 +69,7 @@ public class GenerateInvoicesAction implements Runnable {
@Config("projectId") String projectId, @Config("projectId") String projectId,
@Config("apacheBeamBucketUrl") String beamBucketUrl, @Config("apacheBeamBucketUrl") String beamBucketUrl,
@Config("invoiceTemplateUrl") String invoiceTemplateUrl, @Config("invoiceTemplateUrl") String invoiceTemplateUrl,
@Config("defaultJobZone") String jobZone,
@Parameter(PARAM_SHOULD_PUBLISH) boolean shouldPublish, @Parameter(PARAM_SHOULD_PUBLISH) boolean shouldPublish,
YearMonth yearMonth, YearMonth yearMonth,
Dataflow dataflow, Dataflow dataflow,
@ -76,6 +78,7 @@ public class GenerateInvoicesAction implements Runnable {
this.projectId = projectId; this.projectId = projectId;
this.beamBucketUrl = beamBucketUrl; this.beamBucketUrl = beamBucketUrl;
this.invoiceTemplateUrl = invoiceTemplateUrl; this.invoiceTemplateUrl = invoiceTemplateUrl;
this.jobZone = jobZone;
this.shouldPublish = shouldPublish; this.shouldPublish = shouldPublish;
this.yearMonth = yearMonth; this.yearMonth = yearMonth;
this.dataflow = dataflow; this.dataflow = dataflow;
@ -92,7 +95,7 @@ public class GenerateInvoicesAction implements Runnable {
.setJobName(String.format("invoicing-%s", yearMonth)) .setJobName(String.format("invoicing-%s", yearMonth))
.setEnvironment( .setEnvironment(
new RuntimeEnvironment() new RuntimeEnvironment()
.setZone("us-east1-c") .setZone(jobZone)
.setTempLocation(beamBucketUrl + "/temporary")) .setTempLocation(beamBucketUrl + "/temporary"))
.setParameters(ImmutableMap.of("yearMonth", yearMonth.toString("yyyy-MM"))); .setParameters(ImmutableMap.of("yearMonth", yearMonth.toString("yyyy-MM")));
LaunchTemplateResponse launchResponse = LaunchTemplateResponse launchResponse =

View file

@ -0,0 +1,32 @@
package(
default_visibility = ["//visibility:public"],
)
licenses(["notice"]) # Apache 2.0
java_library(
name = "spec11",
srcs = glob(["*.java"]),
deps = [
"//java/google/registry/config",
"//java/google/registry/keyring/api",
"//java/google/registry/request",
"//java/google/registry/request/auth",
"@com_google_api_client_appengine",
"@com_google_apis_google_api_services_dataflow",
"@com_google_appengine_api_1_0_sdk",
"@com_google_appengine_tools_appengine_gcs_client",
"@com_google_dagger",
"@com_google_flogger",
"@com_google_flogger_system_backend",
"@com_google_guava",
"@com_google_http_client",
"@javax_inject",
"@javax_servlet_api",
"@joda_time",
"@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",
],
)

View file

@ -0,0 +1,106 @@
// 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.reporting.spec11;
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_OK;
import com.google.api.services.dataflow.Dataflow;
import com.google.api.services.dataflow.model.LaunchTemplateParameters;
import com.google.api.services.dataflow.model.LaunchTemplateResponse;
import com.google.api.services.dataflow.model.RuntimeEnvironment;
import com.google.common.collect.ImmutableMap;
import com.google.common.flogger.FluentLogger;
import com.google.common.net.MediaType;
import google.registry.config.RegistryConfig.Config;
import google.registry.keyring.api.KeyModule.Key;
import google.registry.request.Action;
import google.registry.request.Response;
import google.registry.request.auth.Auth;
import java.io.IOException;
import javax.inject.Inject;
/**
* Invokes the {@code Spec11Pipeline} Beam template via the REST api.
*
* <p>This action runs the {@link google.registry.beam.spec11.Spec11Pipeline} template, which
* generates the specified month's Spec11 report and stores it on GCS.
*/
@Action(path = GenerateSpec11ReportAction.PATH, method = POST, auth = Auth.AUTH_INTERNAL_ONLY)
public class GenerateSpec11ReportAction implements Runnable {
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
static final String PATH = "/_dr/task/generateSpec11";
private final String projectId;
private final String beamBucketUrl;
private final String spec11TemplateUrl;
private final String jobZone;
private final String apiKey;
private final Response response;
private final Dataflow dataflow;
@Inject
GenerateSpec11ReportAction(
@Config("projectId") String projectId,
@Config("apacheBeamBucketUrl") String beamBucketUrl,
@Config("spec11TemplateUrl") String spec11TemplateUrl,
@Config("defaultJobZone") String jobZone,
@Key("safeBrowsingAPIKey") String apiKey,
Response response,
Dataflow dataflow) {
this.projectId = projectId;
this.beamBucketUrl = beamBucketUrl;
this.spec11TemplateUrl = spec11TemplateUrl;
this.jobZone = jobZone;
this.apiKey = apiKey;
this.response = response;
this.dataflow = dataflow;
}
@Override
public void run() {
try {
LaunchTemplateParameters params =
new LaunchTemplateParameters()
.setJobName("spec11_action")
.setEnvironment(
new RuntimeEnvironment()
.setZone(jobZone)
.setTempLocation(beamBucketUrl + "/temporary"))
.setParameters(ImmutableMap.of("safeBrowsingApiKey", apiKey));
LaunchTemplateResponse launchResponse =
dataflow
.projects()
.templates()
.launch(projectId, params)
.setGcsPath(spec11TemplateUrl)
.execute();
// TODO(b/111545355): Send an e-mail alert interpreting the results.
logger.atInfo().log("Got response: %s", launchResponse.getJob().toPrettyString());
} catch (IOException e) {
logger.atWarning().withCause(e).log("Template Launch failed");
response.setStatus(SC_INTERNAL_SERVER_ERROR);
response.setContentType(MediaType.PLAIN_TEXT_UTF_8);
response.setPayload(String.format("Template launch failed: %s", e.getMessage()));
return;
}
response.setStatus(SC_OK);
response.setContentType(MediaType.PLAIN_TEXT_UTF_8);
response.setPayload("Launched Spec11 dataflow template.");
}
}

View file

@ -2170,10 +2170,10 @@ def org_apache_ftpserver_core():
def org_apache_httpcomponents_httpclient(): def org_apache_httpcomponents_httpclient():
java_import_external( java_import_external(
name = "org_apache_httpcomponents_httpclient", name = "org_apache_httpcomponents_httpclient",
jar_sha256 = "752596ebdc7c9ae5d9a655de3bb06d078734679a9de23321dbf284ee44563c03", jar_sha256 = "0dffc621400d6c632f55787d996b8aeca36b30746a716e079a985f24d8074057",
jar_urls = [ jar_urls = [
"http://domain-registry-maven.storage.googleapis.com/repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.0.1/httpclient-4.0.1.jar", "http://domain-registry-maven.storage.googleapis.com/repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar",
"http://repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.0.1/httpclient-4.0.1.jar", "http://repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar",
], ],
licenses = ["notice"], # Apache License licenses = ["notice"], # Apache License
deps = [ deps = [
@ -2186,10 +2186,10 @@ def org_apache_httpcomponents_httpclient():
def org_apache_httpcomponents_httpcore(): def org_apache_httpcomponents_httpcore():
java_import_external( java_import_external(
name = "org_apache_httpcomponents_httpcore", name = "org_apache_httpcomponents_httpcore",
jar_sha256 = "3b6bf92affa85d4169a91547ce3c7093ed993b41ad2df80469fc768ad01e6b6b", jar_sha256 = "f7bc09dc8a7003822d109634ffd3845d579d12e725ae54673e323a7ce7f5e325",
jar_urls = [ jar_urls = [
"http://domain-registry-maven.storage.googleapis.com/repo1.maven.org/maven2/org/apache/httpcomponents/httpcore/4.0.1/httpcore-4.0.1.jar", "http://domain-registry-maven.storage.googleapis.com/repo1.maven.org/maven2/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar",
"http://repo1.maven.org/maven2/org/apache/httpcomponents/httpcore/4.0.1/httpcore-4.0.1.jar", "http://repo1.maven.org/maven2/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar",
], ],
licenses = ["notice"], # Apache License licenses = ["notice"], # Apache License
) )

View file

@ -69,6 +69,9 @@ final class GetKeyringSecretCommand implements RemoteApiCommand {
case ICANN_REPORTING_PASSWORD: case ICANN_REPORTING_PASSWORD:
out.write(KeySerializer.serializeString(keyring.getIcannReportingPassword())); out.write(KeySerializer.serializeString(keyring.getIcannReportingPassword()));
break; break;
case SAFE_BROWSING_API_KEY:
out.write(KeySerializer.serializeString(keyring.getSafeBrowsingAPIKey()));
break;
case JSON_CREDENTIAL: case JSON_CREDENTIAL:
out.write(KeySerializer.serializeString(keyring.getJsonCredential())); out.write(KeySerializer.serializeString(keyring.getJsonCredential()));
break; break;

View file

@ -105,6 +105,9 @@ final class UpdateKmsKeyringCommand implements RemoteApiCommand {
case RDE_STAGING_KEY_PAIR: case RDE_STAGING_KEY_PAIR:
kmsUpdater.setRdeStagingKey(deserializeKeyPair(input)); kmsUpdater.setRdeStagingKey(deserializeKeyPair(input));
break; break;
case SAFE_BROWSING_API_KEY:
kmsUpdater.setSafeBrowsingAPIKey(deserializeString(input));
break;
case RDE_STAGING_PUBLIC_KEY: case RDE_STAGING_PUBLIC_KEY:
throw new IllegalArgumentException( throw new IllegalArgumentException(
"Can't update RDE_STAGING_PUBLIC_KEY directly." "Can't update RDE_STAGING_PUBLIC_KEY directly."

View file

@ -36,5 +36,6 @@ public enum KeyringKeyName {
RDE_SSH_CLIENT_PUBLIC_KEY, RDE_SSH_CLIENT_PUBLIC_KEY,
RDE_STAGING_KEY_PAIR, RDE_STAGING_KEY_PAIR,
RDE_STAGING_PUBLIC_KEY, RDE_STAGING_PUBLIC_KEY,
SAFE_BROWSING_API_KEY,
} }

View file

@ -24,6 +24,8 @@ java_library(
"@org_apache_beam_runners_google_cloud_dataflow_java", "@org_apache_beam_runners_google_cloud_dataflow_java",
"@org_apache_beam_sdks_java_core", "@org_apache_beam_sdks_java_core",
"@org_apache_beam_sdks_java_io_google_cloud_platform", "@org_apache_beam_sdks_java_io_google_cloud_platform",
"@org_apache_httpcomponents_httpclient",
"@org_apache_httpcomponents_httpcore",
"@org_mockito_all", "@org_mockito_all",
], ],
) )

View file

@ -15,19 +15,39 @@
package google.registry.beam.spec11; package google.registry.beam.spec11;
import static com.google.common.truth.Truth.assertThat; import static com.google.common.truth.Truth.assertThat;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.mockito.Mockito.withSettings;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.io.CharStreams;
import google.registry.beam.spec11.SafeBrowsingTransforms.EvaluateSafeBrowsingFn;
import google.registry.util.ResourceUtils; import google.registry.util.ResourceUtils;
import java.io.ByteArrayInputStream;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.InputStreamReader;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.time.ZoneId; import java.time.ZoneId;
import java.time.ZonedDateTime; import java.time.ZonedDateTime;
import java.util.function.Supplier;
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;
import org.apache.http.ProtocolVersion;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.entity.BasicHttpEntity;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.message.BasicStatusLine;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Rule; import org.junit.Rule;
@ -35,6 +55,7 @@ import org.junit.Test;
import org.junit.rules.TemporaryFolder; import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.JUnit4; import org.junit.runners.JUnit4;
import org.mockito.stubbing.Answer;
/** Unit tests for {@link Spec11Pipeline}. */ /** Unit tests for {@link Spec11Pipeline}. */
@RunWith(JUnit4.class) @RunWith(JUnit4.class)
@ -64,25 +85,72 @@ public class Spec11PipelineTest {
} }
private ImmutableList<Subdomain> getInputDomains() { private ImmutableList<Subdomain> getInputDomains() {
return ImmutableList.of( ImmutableList.Builder<Subdomain> subdomainsBuilder = new ImmutableList.Builder<>();
Subdomain.create( // Put in 2 batches worth (490 < max < 490*2) to get one positive and one negative example.
"a.com", ZonedDateTime.of(2017, 9, 29, 0, 0, 0, 0, ZoneId.of("UTC")), "OK"), for (int i = 0; i < 510; i++) {
Subdomain.create( subdomainsBuilder.add(
"b.com", ZonedDateTime.of(2017, 9, 29, 0, 0, 0, 0, ZoneId.of("UTC")), "OK"), Subdomain.create(
Subdomain.create( String.format("%s.com", i),
"c.com", ZonedDateTime.of(2017, 9, 29, 0, 0, 0, 0, ZoneId.of("UTC")), "OK")); ZonedDateTime.of(2017, 9, 29, 0, 0, 0, 0, ZoneId.of("UTC")),
"OK"));
}
return subdomainsBuilder.build();
} }
/**
* Tests the end-to-end Spec11 pipeline with mocked out API calls.
*
* <p>We suppress the (Serializable & Supplier) dual-casted lambda warnings because the supplier
* produces an explicitly serializable mock, which is safe to cast.
*/
@Test @Test
@SuppressWarnings("unchecked")
public void testEndToEndPipeline_generatesExpectedFiles() throws Exception { public void testEndToEndPipeline_generatesExpectedFiles() throws Exception {
// Establish mocks for testing
ImmutableList<Subdomain> inputRows = getInputDomains(); ImmutableList<Subdomain> inputRows = getInputDomains();
CloseableHttpClient httpClient = mock(CloseableHttpClient.class, withSettings().serializable());
CloseableHttpResponse negativeResponse =
mock(CloseableHttpResponse.class, withSettings().serializable());
CloseableHttpResponse positiveResponse =
mock(CloseableHttpResponse.class, withSettings().serializable());
// Tailor the fake API's response based on whether or not it contains the "bad url" 111.com
when(httpClient.execute(any(HttpPost.class)))
.thenAnswer(
(Answer & Serializable)
(i) -> {
String request =
CharStreams.toString(
new InputStreamReader(
((HttpPost) i.getArguments()[0]).getEntity().getContent(), UTF_8));
if (request.contains("http://111.com")) {
return positiveResponse;
} else {
return negativeResponse;
}
});
when(negativeResponse.getStatusLine())
.thenReturn(new BasicStatusLine(new ProtocolVersion("HTTP", 1, 1), 200, "Done"));
when(negativeResponse.getEntity()).thenReturn(new FakeHttpEntity("{}"));
when(positiveResponse.getStatusLine())
.thenReturn(new BasicStatusLine(new ProtocolVersion("HTTP", 1, 1), 200, "Done"));
when(positiveResponse.getEntity())
.thenReturn(new FakeHttpEntity(getBadUrlMatch("http://111.com")));
EvaluateSafeBrowsingFn evalFn =
new EvaluateSafeBrowsingFn(
StaticValueProvider.of("apikey"), (Serializable & Supplier) () -> httpClient);
// Apply input and evaluation transforms
PCollection<Subdomain> input = p.apply(Create.of(inputRows)); PCollection<Subdomain> input = p.apply(Create.of(inputRows));
spec11Pipeline.countDomainsAndOutputResults(input); spec11Pipeline.evaluateUrlHealth(input, evalFn);
p.run(); p.run();
// Verify output of text file
ImmutableList<String> generatedReport = resultFileContents(); ImmutableList<String> generatedReport = resultFileContents();
assertThat(generatedReport.get(0)).isEqualTo("HELLO WORLD"); // TODO(b/80524726): Rigorously test this output once the pipeline output is finalized.
assertThat(generatedReport.get(1)).isEqualTo("3"); assertThat(generatedReport).hasSize(2);
assertThat(generatedReport.get(1)).contains("http://111.com");
} }
/** Returns the text contents of a file under the beamBucket/results directory. */ /** Returns the text contents of a file under the beamBucket/results directory. */
@ -91,4 +159,45 @@ public class Spec11PipelineTest {
return ImmutableList.copyOf( return ImmutableList.copyOf(
ResourceUtils.readResourceUtf8(resultFile.toURI().toURL()).split("\n")); ResourceUtils.readResourceUtf8(resultFile.toURI().toURL()).split("\n"));
} }
/** Returns a filled-in template for threat detected at a given url. */
private static String getBadUrlMatch(String url) {
return "{\n"
+ " \"matches\": [{\n"
+ " \"threatType\": \"MALWARE\",\n"
+ " \"platformType\": \"WINDOWS\",\n"
+ " \"threatEntryType\": \"URL\",\n"
+ String.format(" \"threat\": {\"url\": \"%s\"},\n", url)
+ " \"threatEntryMetadata\": {\n"
+ " \"entries\": [{\n"
+ " \"key\": \"malware_threat_type\",\n"
+ " \"value\": \"landing\"\n"
+ " }]\n"
+ " },\n"
+ " \"cacheDuration\": \"300.000s\"\n"
+ " },"
+ "]\n"
+ "}";
}
/** A serializable HttpEntity fake that returns {@link String} content. */
private static class FakeHttpEntity extends BasicHttpEntity implements Serializable {
private static final long serialVersionUID = 105738294571L;
private String content;
private void writeObject(ObjectOutputStream oos) throws IOException {
oos.defaultWriteObject();
}
private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
ois.defaultReadObject();
super.setContent(new ByteArrayInputStream(this.content.getBytes(UTF_8)));
}
FakeHttpEntity(String content) {
this.content = content;
}
}
} }

View file

@ -17,6 +17,7 @@ PATH CLASS METHOD
/_dr/task/exportReservedTerms ExportReservedTermsAction POST n INTERNAL APP IGNORED /_dr/task/exportReservedTerms ExportReservedTermsAction POST n INTERNAL APP IGNORED
/_dr/task/exportSnapshot ExportSnapshotAction POST y INTERNAL APP IGNORED /_dr/task/exportSnapshot ExportSnapshotAction POST y INTERNAL APP IGNORED
/_dr/task/generateInvoices GenerateInvoicesAction POST n INTERNAL APP IGNORED /_dr/task/generateInvoices GenerateInvoicesAction POST n INTERNAL APP IGNORED
/_dr/task/generateSpec11 GenerateSpec11ReportAction POST n INTERNAL APP IGNORED
/_dr/task/icannReportingStaging IcannReportingStagingAction POST n INTERNAL APP IGNORED /_dr/task/icannReportingStaging IcannReportingStagingAction POST n INTERNAL APP IGNORED
/_dr/task/icannReportingUpload IcannReportingUploadAction POST n INTERNAL,API APP ADMIN /_dr/task/icannReportingUpload IcannReportingUploadAction POST n INTERNAL,API APP ADMIN
/_dr/task/importRdeContacts RdeContactImportAction GET n INTERNAL APP IGNORED /_dr/task/importRdeContacts RdeContactImportAction GET n INTERNAL APP IGNORED

View file

@ -83,6 +83,7 @@ public class GenerateInvoicesActionTest {
"test-project", "test-project",
"gs://test-project-beam", "gs://test-project-beam",
"gs://test-project-beam/templates/invoicing", "gs://test-project-beam/templates/invoicing",
"us-east1-c",
true, true,
new YearMonth(2017, 10), new YearMonth(2017, 10),
dataflow, dataflow,
@ -118,6 +119,7 @@ public class GenerateInvoicesActionTest {
"test-project", "test-project",
"gs://test-project-beam", "gs://test-project-beam",
"gs://test-project-beam/templates/invoicing", "gs://test-project-beam/templates/invoicing",
"us-east1-c",
false, false,
new YearMonth(2017, 10), new YearMonth(2017, 10),
dataflow, dataflow,
@ -147,6 +149,7 @@ public class GenerateInvoicesActionTest {
"test-project", "test-project",
"gs://test-project-beam", "gs://test-project-beam",
"gs://test-project-beam/templates/invoicing", "gs://test-project-beam/templates/invoicing",
"us-east1-c",
true, true,
new YearMonth(2017, 10), new YearMonth(2017, 10),
dataflow, dataflow,

View file

@ -0,0 +1,39 @@
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 = "spec11",
srcs = glob(["*.java"]),
deps = [
"//java/google/registry/reporting/spec11",
"//javatests/google/registry/testing",
"@com_google_apis_google_api_services_dataflow",
"@com_google_appengine_api_1_0_sdk",
"@com_google_appengine_tools_appengine_gcs_client",
"@com_google_dagger",
"@com_google_guava",
"@com_google_truth",
"@com_google_truth_extensions_truth_java8_extension",
"@javax_servlet_api",
"@joda_time",
"@junit",
"@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 = [":spec11"],
)

View file

@ -0,0 +1,100 @@
// 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.reporting.spec11;
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.verify;
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.Templates;
import com.google.api.services.dataflow.Dataflow.Projects.Templates.Launch;
import com.google.api.services.dataflow.model.Job;
import com.google.api.services.dataflow.model.LaunchTemplateParameters;
import com.google.api.services.dataflow.model.LaunchTemplateResponse;
import com.google.api.services.dataflow.model.RuntimeEnvironment;
import com.google.common.collect.ImmutableMap;
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;
/** Unit tests for {@link google.registry.reporting.spec11.GenerateSpec11ReportAction}. */
@RunWith(JUnit4.class)
public class GenerateSpec11ReportActionTest {
private FakeResponse response;
private Dataflow dataflow;
private Projects dataflowProjects;
private Templates dataflowTemplates;
private Launch dataflowLaunch;
private GenerateSpec11ReportAction action;
@Before
public void setUp() throws IOException {
response = new FakeResponse();
dataflow = mock(Dataflow.class);
// Establish the Dataflow API call chain
dataflow = mock(Dataflow.class);
dataflowProjects = mock(Dataflow.Projects.class);
dataflowTemplates = mock(Templates.class);
dataflowLaunch = mock(Launch.class);
LaunchTemplateResponse launchTemplateResponse = new LaunchTemplateResponse();
// Ultimately we get back this job response with a given id.
launchTemplateResponse.setJob(new Job().setReplaceJobId("jobid"));
when(dataflow.projects()).thenReturn(dataflowProjects);
when(dataflowProjects.templates()).thenReturn(dataflowTemplates);
when(dataflowTemplates.launch(any(String.class), any(LaunchTemplateParameters.class)))
.thenReturn(dataflowLaunch);
when(dataflowLaunch.setGcsPath(any(String.class))).thenReturn(dataflowLaunch);
when(dataflowLaunch.execute()).thenReturn(launchTemplateResponse);
}
@Test
public void testLaunch_success() throws IOException {
action =
new GenerateSpec11ReportAction(
"test",
"gs://my-bucket-beam",
"gs://template",
"us-east1-c",
"api_key/a",
response,
dataflow);
action.run();
LaunchTemplateParameters expectedLaunchTemplateParameters =
new LaunchTemplateParameters()
.setJobName("spec11_action")
.setEnvironment(
new RuntimeEnvironment()
.setZone("us-east1-c")
.setTempLocation("gs://my-bucket-beam/temporary"))
.setParameters(ImmutableMap.of("safeBrowsingApiKey", "api_key/a"));
verify(dataflowTemplates).launch("test", expectedLaunchTemplateParameters);
verify(dataflowLaunch).setGcsPath("gs://template");
assertThat(response.getStatus()).isEqualTo(200);
assertThat(response.getContentType()).isEqualTo(MediaType.PLAIN_TEXT_UTF_8);
assertThat(response.getPayload()).isEqualTo("Launched Spec11 dataflow template.");
}
}

View file

@ -51,6 +51,7 @@ public final class FakeKeyringModule {
private static final ByteSource PGP_PRIVATE_KEYRING = private static final ByteSource PGP_PRIVATE_KEYRING =
loadBytes(FakeKeyringModule.class, "pgp-private-keyring-registry.asc"); loadBytes(FakeKeyringModule.class, "pgp-private-keyring-registry.asc");
private static final String ICANN_REPORTING_PASSWORD = "yolo"; private static final String ICANN_REPORTING_PASSWORD = "yolo";
private static final String SAFE_BROWSING_API_KEY = "a/b_c";
private static final String MARKSDB_DNL_LOGIN = "dnl:yolo"; private static final String MARKSDB_DNL_LOGIN = "dnl:yolo";
private static final String MARKSDB_LORDN_PASSWORD = "yolo"; private static final String MARKSDB_LORDN_PASSWORD = "yolo";
private static final String MARKSDB_SMDRL_LOGIN = "smdrl:yolo"; private static final String MARKSDB_SMDRL_LOGIN = "smdrl:yolo";
@ -134,6 +135,11 @@ public final class FakeKeyringModule {
return ICANN_REPORTING_PASSWORD; return ICANN_REPORTING_PASSWORD;
} }
@Override
public String getSafeBrowsingAPIKey() {
return SAFE_BROWSING_API_KEY;
}
@Override @Override
public PGPKeyPair getBrdaSigningKey() { public PGPKeyPair getBrdaSigningKey() {
return rdeSigningKey; return rdeSigningKey;