mirror of
https://github.com/google/nomulus.git
synced 2025-05-14 00:17:20 +02:00
Update input/output of Spec11 pipeline to final format
This changes the BigQuery input to the fields we ultimately want (fqdn, registrarName, registrarEmailAddress) and the output to a structured POJO holding the results from the API. This POJO is then converted to its final text output, i.e.: Map from registrar e-mail to list of threat-detected subdomains: {"registrarEmail": "c@fake.com", "threats": [{"url": "a.com", "threatType": "MALWARE"}]} {"registrarEmail": "d@fake.com", "threats": [{"url": "x.com", "threatType": "MALWARE"}, {"url": "y.com", "threatType": "MALWARE"}]} This gives us all the data we want in a JSON structured format, to be acted upon downstream by the to-be-constructed PublishSpec11ReportAction. Ideally, we would send an e-mail directly from the beam pipeline, but this is only possible through third-party providers (as opposed to app engine itself). ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=209416880
This commit is contained in:
parent
7dcadaecf6
commit
f7bc17fbe8
11 changed files with 393 additions and 130 deletions
|
@ -8,6 +8,7 @@ java_library(
|
||||||
name = "beam",
|
name = "beam",
|
||||||
srcs = glob(["*.java"]),
|
srcs = glob(["*.java"]),
|
||||||
deps = [
|
deps = [
|
||||||
|
"//java/google/registry/util",
|
||||||
"@com_google_flogger",
|
"@com_google_flogger",
|
||||||
"@com_google_flogger_system_backend",
|
"@com_google_flogger_system_backend",
|
||||||
"@com_google_guava",
|
"@com_google_guava",
|
||||||
|
|
|
@ -17,6 +17,8 @@ package google.registry.beam;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.flogger.FluentLogger;
|
import com.google.common.flogger.FluentLogger;
|
||||||
|
import com.google.common.io.Resources;
|
||||||
|
import google.registry.util.ResourceUtils;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
|
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
|
||||||
|
|
||||||
|
@ -54,4 +56,12 @@ public class BeamUtils {
|
||||||
missingFieldList, record));
|
missingFieldList, record));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the {@link String} contents for a file in the {@code sql/} directory relative to a
|
||||||
|
* class.
|
||||||
|
*/
|
||||||
|
public static String getQueryFromFile(Class<?> clazz, String filename) {
|
||||||
|
return ResourceUtils.readResourceUtf8(Resources.getResource(clazz, "sql/" + filename));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -14,8 +14,8 @@
|
||||||
|
|
||||||
package google.registry.beam.invoicing;
|
package google.registry.beam.invoicing;
|
||||||
|
|
||||||
import com.google.common.io.Resources;
|
import static google.registry.beam.BeamUtils.getQueryFromFile;
|
||||||
import google.registry.util.ResourceUtils;
|
|
||||||
import google.registry.util.SqlTemplate;
|
import google.registry.util.SqlTemplate;
|
||||||
import java.time.LocalDateTime;
|
import java.time.LocalDateTime;
|
||||||
import java.time.LocalTime;
|
import java.time.LocalTime;
|
||||||
|
@ -91,7 +91,7 @@ public class InvoicingUtils {
|
||||||
LocalDateTime firstMoment = reportingMonth.atDay(1).atTime(LocalTime.MIDNIGHT);
|
LocalDateTime firstMoment = reportingMonth.atDay(1).atTime(LocalTime.MIDNIGHT);
|
||||||
LocalDateTime lastMoment = reportingMonth.atEndOfMonth().atTime(LocalTime.MAX);
|
LocalDateTime lastMoment = reportingMonth.atEndOfMonth().atTime(LocalTime.MAX);
|
||||||
// Construct the month's query by filling in the billing_events.sql template
|
// Construct the month's query by filling in the billing_events.sql template
|
||||||
return SqlTemplate.create(getQueryFromFile("billing_events.sql"))
|
return SqlTemplate.create(getQueryFromFile(InvoicingPipeline.class, "billing_events.sql"))
|
||||||
.put("FIRST_TIMESTAMP_OF_MONTH", firstMoment.format(TIMESTAMP_FORMATTER))
|
.put("FIRST_TIMESTAMP_OF_MONTH", firstMoment.format(TIMESTAMP_FORMATTER))
|
||||||
.put("LAST_TIMESTAMP_OF_MONTH", lastMoment.format(TIMESTAMP_FORMATTER))
|
.put("LAST_TIMESTAMP_OF_MONTH", lastMoment.format(TIMESTAMP_FORMATTER))
|
||||||
.put("PROJECT_ID", projectId)
|
.put("PROJECT_ID", projectId)
|
||||||
|
@ -103,10 +103,4 @@ public class InvoicingUtils {
|
||||||
.build();
|
.build();
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns the {@link String} contents for a file in the {@code beam/sql/} directory. */
|
|
||||||
private static String getQueryFromFile(String filename) {
|
|
||||||
return ResourceUtils.readResourceUtf8(
|
|
||||||
Resources.getResource(InvoicingUtils.class, "sql/" + filename));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -11,6 +11,7 @@ java_library(
|
||||||
deps = [
|
deps = [
|
||||||
"//java/google/registry/beam",
|
"//java/google/registry/beam",
|
||||||
"//java/google/registry/config",
|
"//java/google/registry/config",
|
||||||
|
"//java/google/registry/util",
|
||||||
"@com_google_auto_value",
|
"@com_google_auto_value",
|
||||||
"@com_google_dagger",
|
"@com_google_dagger",
|
||||||
"@com_google_flogger",
|
"@com_google_flogger",
|
||||||
|
|
|
@ -19,9 +19,10 @@ import static java.nio.charset.StandardCharsets.UTF_8;
|
||||||
import static org.apache.http.HttpStatus.SC_OK;
|
import static org.apache.http.HttpStatus.SC_OK;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.flogger.FluentLogger;
|
import com.google.common.flogger.FluentLogger;
|
||||||
import com.google.common.io.CharStreams;
|
import com.google.common.io.CharStreams;
|
||||||
|
import google.registry.util.Retrier;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStreamReader;
|
import java.io.InputStreamReader;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
@ -62,7 +63,7 @@ public class SafeBrowsingTransforms {
|
||||||
*
|
*
|
||||||
* @see <a href=https://developers.google.com/safe-browsing/v4/lookup-api>Lookup API</a>
|
* @see <a href=https://developers.google.com/safe-browsing/v4/lookup-api>Lookup API</a>
|
||||||
*/
|
*/
|
||||||
static class EvaluateSafeBrowsingFn extends DoFn<Subdomain, KV<Subdomain, String>> {
|
static class EvaluateSafeBrowsingFn extends DoFn<Subdomain, KV<Subdomain, ThreatMatch>> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Max number of urls we can check in a single query.
|
* Max number of urls we can check in a single query.
|
||||||
|
@ -75,8 +76,8 @@ public class SafeBrowsingTransforms {
|
||||||
private final ValueProvider<String> apiKeyProvider;
|
private final ValueProvider<String> apiKeyProvider;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Maps a subdomain's HTTP URL to its corresponding {@link Subdomain} to facilitate batching
|
* Maps a subdomain's {@code fullyQualifiedDomainName} to its corresponding {@link Subdomain} to
|
||||||
* SafeBrowsing API requests.
|
* facilitate batching SafeBrowsing API requests.
|
||||||
*/
|
*/
|
||||||
private final Map<String, Subdomain> subdomainBuffer = new LinkedHashMap<>(BATCH_SIZE);
|
private final Map<String, Subdomain> subdomainBuffer = new LinkedHashMap<>(BATCH_SIZE);
|
||||||
|
|
||||||
|
@ -88,6 +89,9 @@ public class SafeBrowsingTransforms {
|
||||||
*/
|
*/
|
||||||
private final Supplier<CloseableHttpClient> closeableHttpClientSupplier;
|
private final Supplier<CloseableHttpClient> closeableHttpClientSupplier;
|
||||||
|
|
||||||
|
/** Retries on receiving transient failures such as {@link IOException}. */
|
||||||
|
private final Retrier retrier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs a {@link EvaluateSafeBrowsingFn} that gets its API key from the given provider.
|
* Constructs a {@link EvaluateSafeBrowsingFn} that gets its API key from the given provider.
|
||||||
*
|
*
|
||||||
|
@ -99,8 +103,9 @@ public class SafeBrowsingTransforms {
|
||||||
* @param apiKeyProvider provides the SafeBrowsing API key from {@code KMS} at runtime
|
* @param apiKeyProvider provides the SafeBrowsing API key from {@code KMS} at runtime
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
EvaluateSafeBrowsingFn(ValueProvider<String> apiKeyProvider) {
|
EvaluateSafeBrowsingFn(ValueProvider<String> apiKeyProvider, Retrier retrier) {
|
||||||
this.apiKeyProvider = apiKeyProvider;
|
this.apiKeyProvider = apiKeyProvider;
|
||||||
|
this.retrier = retrier;
|
||||||
this.closeableHttpClientSupplier = (Supplier & Serializable) HttpClients::createDefault;
|
this.closeableHttpClientSupplier = (Supplier & Serializable) HttpClients::createDefault;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -113,8 +118,11 @@ public class SafeBrowsingTransforms {
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
EvaluateSafeBrowsingFn(
|
EvaluateSafeBrowsingFn(
|
||||||
ValueProvider<String> apiKeyProvider, Supplier<CloseableHttpClient> clientSupplier) {
|
ValueProvider<String> apiKeyProvider,
|
||||||
|
Retrier retrier,
|
||||||
|
Supplier<CloseableHttpClient> clientSupplier) {
|
||||||
this.apiKeyProvider = apiKeyProvider;
|
this.apiKeyProvider = apiKeyProvider;
|
||||||
|
this.retrier = retrier;
|
||||||
this.closeableHttpClientSupplier = clientSupplier;
|
this.closeableHttpClientSupplier = clientSupplier;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -122,7 +130,7 @@ public class SafeBrowsingTransforms {
|
||||||
@FinishBundle
|
@FinishBundle
|
||||||
public void finishBundle(FinishBundleContext context) {
|
public void finishBundle(FinishBundleContext context) {
|
||||||
if (!subdomainBuffer.isEmpty()) {
|
if (!subdomainBuffer.isEmpty()) {
|
||||||
ImmutableList<KV<Subdomain, String>> results = evaluateAndFlush();
|
ImmutableSet<KV<Subdomain, ThreatMatch>> results = evaluateAndFlush();
|
||||||
results.forEach((kv) -> context.output(kv, Instant.now(), GlobalWindow.INSTANCE));
|
results.forEach((kv) -> context.output(kv, Instant.now(), GlobalWindow.INSTANCE));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -134,11 +142,9 @@ public class SafeBrowsingTransforms {
|
||||||
@ProcessElement
|
@ProcessElement
|
||||||
public void processElement(ProcessContext context) {
|
public void processElement(ProcessContext context) {
|
||||||
Subdomain subdomain = context.element();
|
Subdomain subdomain = context.element();
|
||||||
// We put HTTP URLs into the buffer because the API requires specifying the protocol.
|
subdomainBuffer.put(subdomain.fullyQualifiedDomainName(), subdomain);
|
||||||
subdomainBuffer.put(
|
|
||||||
String.format("http://%s", subdomain.fullyQualifiedDomainName()), subdomain);
|
|
||||||
if (subdomainBuffer.size() >= BATCH_SIZE) {
|
if (subdomainBuffer.size() >= BATCH_SIZE) {
|
||||||
ImmutableList<KV<Subdomain, String>> results = evaluateAndFlush();
|
ImmutableSet<KV<Subdomain, ThreatMatch>> results = evaluateAndFlush();
|
||||||
results.forEach(context::output);
|
results.forEach(context::output);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -149,8 +155,8 @@ public class SafeBrowsingTransforms {
|
||||||
*
|
*
|
||||||
* <p>If a {@link Subdomain} is safe according to the API, it will not emit a report.
|
* <p>If a {@link Subdomain} is safe according to the API, it will not emit a report.
|
||||||
*/
|
*/
|
||||||
private ImmutableList<KV<Subdomain, String>> evaluateAndFlush() {
|
private ImmutableSet<KV<Subdomain, ThreatMatch>> evaluateAndFlush() {
|
||||||
ImmutableList.Builder<KV<Subdomain, String>> resultBuilder = new ImmutableList.Builder<>();
|
ImmutableSet.Builder<KV<Subdomain, ThreatMatch>> resultBuilder = new ImmutableSet.Builder<>();
|
||||||
try {
|
try {
|
||||||
URIBuilder uriBuilder = new URIBuilder(SAFE_BROWSING_URL);
|
URIBuilder uriBuilder = new URIBuilder(SAFE_BROWSING_URL);
|
||||||
// Add the API key param
|
// Add the API key param
|
||||||
|
@ -161,17 +167,18 @@ public class SafeBrowsingTransforms {
|
||||||
|
|
||||||
JSONObject requestBody = createRequestBody();
|
JSONObject requestBody = createRequestBody();
|
||||||
httpPost.setEntity(new ByteArrayEntity(requestBody.toString().getBytes(UTF_8)));
|
httpPost.setEntity(new ByteArrayEntity(requestBody.toString().getBytes(UTF_8)));
|
||||||
|
// Retry transient exceptions such as IOException
|
||||||
try (CloseableHttpClient client = closeableHttpClientSupplier.get();
|
retrier.callWithRetry(
|
||||||
CloseableHttpResponse response = client.execute(httpPost)) {
|
() -> {
|
||||||
processResponse(response, resultBuilder);
|
try (CloseableHttpClient client = closeableHttpClientSupplier.get();
|
||||||
}
|
CloseableHttpResponse response = client.execute(httpPost)) {
|
||||||
} catch (URISyntaxException | JSONException e) {
|
processResponse(response, resultBuilder);
|
||||||
// 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.");
|
IOException.class);
|
||||||
} catch (IOException e) {
|
} catch (URISyntaxException | JSONException e) {
|
||||||
logger.atSevere().withCause(e).log("Caught IOException during processing, skipping batch.");
|
// Fail the pipeline on a parsing exception- this indicates the API likely changed.
|
||||||
|
throw new RuntimeException("Caught parsing exception, failing pipeline.", e);
|
||||||
} finally {
|
} finally {
|
||||||
// Flush the buffer
|
// Flush the buffer
|
||||||
subdomainBuffer.clear();
|
subdomainBuffer.clear();
|
||||||
|
@ -206,12 +213,13 @@ public class SafeBrowsingTransforms {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Iterates through all threat matches in the API response and adds them to the resultBuilder.
|
* Iterates through all threat matches in the API response and adds them to the {@code
|
||||||
|
* resultBuilder}.
|
||||||
*/
|
*/
|
||||||
private void processResponse(
|
private void processResponse(
|
||||||
CloseableHttpResponse response, ImmutableList.Builder<KV<Subdomain, String>> resultBuilder)
|
CloseableHttpResponse response,
|
||||||
|
ImmutableSet.Builder<KV<Subdomain, ThreatMatch>> resultBuilder)
|
||||||
throws JSONException, IOException {
|
throws JSONException, IOException {
|
||||||
|
|
||||||
int statusCode = response.getStatusLine().getStatusCode();
|
int statusCode = response.getStatusLine().getStatusCode();
|
||||||
if (statusCode != SC_OK) {
|
if (statusCode != SC_OK) {
|
||||||
logger.atWarning().log("Got unexpected status code %s from response", statusCode);
|
logger.atWarning().log("Got unexpected status code %s from response", statusCode);
|
||||||
|
@ -230,7 +238,9 @@ public class SafeBrowsingTransforms {
|
||||||
for (int i = 0; i < threatMatches.length(); i++) {
|
for (int i = 0; i < threatMatches.length(); i++) {
|
||||||
JSONObject match = threatMatches.getJSONObject(i);
|
JSONObject match = threatMatches.getJSONObject(i);
|
||||||
String url = match.getJSONObject("threat").getString("url");
|
String url = match.getJSONObject("threat").getString("url");
|
||||||
resultBuilder.add(KV.of(subdomainBuffer.get(url), match.toString()));
|
Subdomain subdomain = subdomainBuffer.get(url);
|
||||||
|
resultBuilder.add(
|
||||||
|
KV.of(subdomain, ThreatMatch.create(match, subdomain.fullyQualifiedDomainName())));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -14,8 +14,12 @@
|
||||||
|
|
||||||
package google.registry.beam.spec11;
|
package google.registry.beam.spec11;
|
||||||
|
|
||||||
|
import static google.registry.beam.BeamUtils.getQueryFromFile;
|
||||||
|
|
||||||
import google.registry.beam.spec11.SafeBrowsingTransforms.EvaluateSafeBrowsingFn;
|
import google.registry.beam.spec11.SafeBrowsingTransforms.EvaluateSafeBrowsingFn;
|
||||||
import google.registry.config.RegistryConfig.Config;
|
import google.registry.config.RegistryConfig.Config;
|
||||||
|
import google.registry.util.Retrier;
|
||||||
|
import google.registry.util.SqlTemplate;
|
||||||
import java.io.Serializable;
|
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;
|
||||||
|
@ -27,10 +31,17 @@ 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.GroupByKey;
|
||||||
|
import org.apache.beam.sdk.transforms.MapElements;
|
||||||
import org.apache.beam.sdk.transforms.ParDo;
|
import org.apache.beam.sdk.transforms.ParDo;
|
||||||
import org.apache.beam.sdk.transforms.Sample;
|
import org.apache.beam.sdk.values.KV;
|
||||||
import org.apache.beam.sdk.transforms.ToString;
|
|
||||||
import org.apache.beam.sdk.values.PCollection;
|
import org.apache.beam.sdk.values.PCollection;
|
||||||
|
import org.apache.beam.sdk.values.TypeDescriptor;
|
||||||
|
import org.apache.beam.sdk.values.TypeDescriptors;
|
||||||
|
import org.json.JSONArray;
|
||||||
|
import org.json.JSONException;
|
||||||
|
import org.json.JSONObject;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Definition of a Dataflow pipeline template, which generates a given month's spec11 report.
|
* Definition of a Dataflow pipeline template, which generates a given month's spec11 report.
|
||||||
|
@ -60,6 +71,9 @@ public class Spec11Pipeline implements Serializable {
|
||||||
@Config("spec11BucketUrl")
|
@Config("spec11BucketUrl")
|
||||||
String spec11BucketUrl;
|
String spec11BucketUrl;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
Retrier retrier;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
Spec11Pipeline() {}
|
Spec11Pipeline() {}
|
||||||
|
|
||||||
|
@ -106,13 +120,21 @@ public class Spec11Pipeline implements Serializable {
|
||||||
"Read active domains from BigQuery",
|
"Read active domains from BigQuery",
|
||||||
BigQueryIO.read(Subdomain::parseFromRecord)
|
BigQueryIO.read(Subdomain::parseFromRecord)
|
||||||
.fromQuery(
|
.fromQuery(
|
||||||
// This query must be customized for your own use.
|
SqlTemplate.create(getQueryFromFile(Spec11Pipeline.class, "subdomains.sql"))
|
||||||
"SELECT * FROM YOUR_TABLE_HERE")
|
.put("PROJECT_ID", projectId)
|
||||||
|
.put("DATASTORE_EXPORT_DATASET", "latest_datastore_export")
|
||||||
|
.put("REGISTRAR_TABLE", "Registrar")
|
||||||
|
.put("DOMAIN_BASE_TABLE", "DomainBase")
|
||||||
|
.build())
|
||||||
.withCoder(SerializableCoder.of(Subdomain.class))
|
.withCoder(SerializableCoder.of(Subdomain.class))
|
||||||
.usingStandardSql()
|
.usingStandardSql()
|
||||||
.withoutValidation()
|
.withoutValidation()
|
||||||
.withTemplateCompatibility());
|
.withTemplateCompatibility());
|
||||||
evaluateUrlHealth(domains, new EvaluateSafeBrowsingFn(options.getSafeBrowsingApiKey()));
|
|
||||||
|
evaluateUrlHealth(
|
||||||
|
domains,
|
||||||
|
new EvaluateSafeBrowsingFn(options.getSafeBrowsingApiKey(), retrier),
|
||||||
|
options.getYearMonth());
|
||||||
p.run();
|
p.run();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -122,21 +144,51 @@ public class Spec11Pipeline implements Serializable {
|
||||||
* <p>This is factored out to facilitate testing.
|
* <p>This is factored out to facilitate testing.
|
||||||
*/
|
*/
|
||||||
void evaluateUrlHealth(
|
void evaluateUrlHealth(
|
||||||
PCollection<Subdomain> domains, EvaluateSafeBrowsingFn evaluateSafeBrowsingFn) {
|
PCollection<Subdomain> domains,
|
||||||
|
EvaluateSafeBrowsingFn evaluateSafeBrowsingFn,
|
||||||
|
ValueProvider<String> yearMonthProvider) {
|
||||||
domains
|
domains
|
||||||
// TODO(b/111545355): Remove this limiter once we're confident we won't go over quota.
|
|
||||||
.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("Run through SafeBrowsingAPI", ParDo.of(evaluateSafeBrowsingFn))
|
||||||
.apply("Convert results to string", ToString.elements())
|
.apply(
|
||||||
|
"Map registrar e-mail to ThreatMatch",
|
||||||
|
MapElements.into(
|
||||||
|
TypeDescriptors.kvs(
|
||||||
|
TypeDescriptors.strings(), TypeDescriptor.of(ThreatMatch.class)))
|
||||||
|
.via(
|
||||||
|
(KV<Subdomain, ThreatMatch> kv) ->
|
||||||
|
KV.of(kv.getKey().registrarEmailAddress(), kv.getValue())))
|
||||||
|
.apply("Group by registrar email address", GroupByKey.create())
|
||||||
|
.apply(
|
||||||
|
"Convert results to JSON format",
|
||||||
|
MapElements.into(TypeDescriptors.strings())
|
||||||
|
.via(
|
||||||
|
(KV<String, Iterable<ThreatMatch>> kv) -> {
|
||||||
|
JSONObject output = new JSONObject();
|
||||||
|
try {
|
||||||
|
output.put("registrarEmailAddress", kv.getKey());
|
||||||
|
JSONArray threatMatches = new JSONArray();
|
||||||
|
for (ThreatMatch match : kv.getValue()) {
|
||||||
|
threatMatches.put(match.toJSON());
|
||||||
|
}
|
||||||
|
output.put("threatMatches", threatMatches);
|
||||||
|
return output.toString();
|
||||||
|
} catch (JSONException e) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
String.format(
|
||||||
|
"Encountered an error constructing the JSON for %s", kv.toString()),
|
||||||
|
e);
|
||||||
|
}
|
||||||
|
}))
|
||||||
.apply(
|
.apply(
|
||||||
"Output to text file",
|
"Output to text file",
|
||||||
TextIO.write()
|
TextIO.write()
|
||||||
// TODO(b/111545355): Replace this with a templated directory based on yearMonth
|
.to(
|
||||||
.to(spec11BucketUrl)
|
NestedValueProvider.of(
|
||||||
|
yearMonthProvider,
|
||||||
|
yearMonth ->
|
||||||
|
String.format(
|
||||||
|
"%s/%s/%s-monthly-report", spec11BucketUrl, yearMonth, yearMonth)))
|
||||||
.withoutSharding()
|
.withoutSharding()
|
||||||
.withHeader("HELLO WORLD"));
|
.withHeader("Map from registrar email to detected subdomain threats:"));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,9 +22,6 @@ import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.flogger.FluentLogger;
|
import com.google.common.flogger.FluentLogger;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.time.Instant;
|
|
||||||
import java.time.ZoneId;
|
|
||||||
import java.time.ZonedDateTime;
|
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
|
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
|
||||||
|
|
||||||
|
@ -42,14 +39,14 @@ public abstract class Subdomain implements Serializable {
|
||||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||||
|
|
||||||
private static final ImmutableList<String> FIELD_NAMES =
|
private static final ImmutableList<String> FIELD_NAMES =
|
||||||
ImmutableList.of("fullyQualifiedDomainName", "statuses", "creationTime");
|
ImmutableList.of("fullyQualifiedDomainName", "registrarName", "registrarEmailAddress");
|
||||||
|
|
||||||
/** Returns the fully qualified domain name. */
|
/** Returns the fully qualified domain name. */
|
||||||
abstract String fullyQualifiedDomainName();
|
abstract String fullyQualifiedDomainName();
|
||||||
/** Returns the UTC DateTime this domain was created. */
|
/** Returns the name of the associated registrar for this domain. */
|
||||||
abstract ZonedDateTime creationTime();
|
abstract String registrarName();
|
||||||
/** Returns the space-delimited list of statuses on this domain. */
|
/** Returns the email address of the registrar associated with this domain. */
|
||||||
abstract String statuses();
|
abstract String registrarEmailAddress();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs a {@link Subdomain} from an Apache Avro {@code SchemaAndRecord}.
|
* Constructs a {@link Subdomain} from an Apache Avro {@code SchemaAndRecord}.
|
||||||
|
@ -63,10 +60,8 @@ public abstract class Subdomain implements Serializable {
|
||||||
GenericRecord record = schemaAndRecord.getRecord();
|
GenericRecord record = schemaAndRecord.getRecord();
|
||||||
return create(
|
return create(
|
||||||
extractField(record, "fullyQualifiedDomainName"),
|
extractField(record, "fullyQualifiedDomainName"),
|
||||||
// Bigquery provides UNIX timestamps with microsecond precision.
|
extractField(record, "registrarName"),
|
||||||
Instant.ofEpochMilli(Long.parseLong(extractField(record, "creationTime")) / 1000)
|
extractField(record, "registrarEmailAddress"));
|
||||||
.atZone(ZoneId.of("UTC")),
|
|
||||||
extractField(record, "statuses"));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -77,8 +72,8 @@ public abstract class Subdomain implements Serializable {
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
static Subdomain create(
|
static Subdomain create(
|
||||||
String fullyQualifiedDomainName, ZonedDateTime creationTime, String statuses) {
|
String fullyQualifiedDomainName, String registrarName, String registrarEmailAddress) {
|
||||||
return new AutoValue_Subdomain(fullyQualifiedDomainName, creationTime, statuses);
|
return new AutoValue_Subdomain(fullyQualifiedDomainName, registrarName, registrarEmailAddress);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
72
java/google/registry/beam/spec11/ThreatMatch.java
Normal file
72
java/google/registry/beam/spec11/ThreatMatch.java
Normal file
|
@ -0,0 +1,72 @@
|
||||||
|
// 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 com.google.auto.value.AutoValue;
|
||||||
|
import java.io.Serializable;
|
||||||
|
import org.json.JSONException;
|
||||||
|
import org.json.JSONObject;
|
||||||
|
|
||||||
|
/** A POJO representing a threat match response from the {@code SafeBrowsing API}. */
|
||||||
|
@AutoValue
|
||||||
|
public abstract class ThreatMatch implements Serializable {
|
||||||
|
|
||||||
|
private static final String THREAT_TYPE_FIELD = "threatType";
|
||||||
|
private static final String PLATFORM_TYPE_FIELD = "platformType";
|
||||||
|
private static final String METADATA_FIELD = "threatEntryMetadata";
|
||||||
|
|
||||||
|
/** Returns what kind of threat it is (malware, phishing etc.) */
|
||||||
|
abstract String threatType();
|
||||||
|
/** Returns what platforms it affects (Windows, Linux etc.) */
|
||||||
|
abstract String platformType();
|
||||||
|
/**
|
||||||
|
* Returns a String representing a JSON Object containing arbitrary metadata associated with this
|
||||||
|
* threat, or "NONE" if there is no metadata to retrieve.
|
||||||
|
*
|
||||||
|
* <p>This ideally would be a {@link JSONObject} type, but can't be due to serialization
|
||||||
|
* requirements.
|
||||||
|
*/
|
||||||
|
abstract String metadata();
|
||||||
|
/** Returns the fully qualified domain name [SLD].[TLD] of the matched threat. */
|
||||||
|
abstract String fullyQualifiedDomainName();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs a {@link ThreatMatch} by parsing a {@code SafeBrowsing API} response {@link
|
||||||
|
* JSONObject}.
|
||||||
|
*
|
||||||
|
* @throws JSONException when encountering parse errors in the response format
|
||||||
|
*/
|
||||||
|
static ThreatMatch create(JSONObject threatMatchJSON, String fullyQualifiedDomainName)
|
||||||
|
throws JSONException {
|
||||||
|
return new AutoValue_ThreatMatch(
|
||||||
|
threatMatchJSON.getString(THREAT_TYPE_FIELD),
|
||||||
|
threatMatchJSON.getString(PLATFORM_TYPE_FIELD),
|
||||||
|
threatMatchJSON.has(METADATA_FIELD)
|
||||||
|
? threatMatchJSON.getJSONObject(METADATA_FIELD).toString()
|
||||||
|
: "NONE",
|
||||||
|
fullyQualifiedDomainName);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Returns a {@link String} containing the simplest details about this threat. */
|
||||||
|
String getSimpleDetails() {
|
||||||
|
return String.format("%s;%s", this.fullyQualifiedDomainName(), this.threatType());
|
||||||
|
}
|
||||||
|
/** Returns a {@link JSONObject} representing a subset of this object's data. */
|
||||||
|
JSONObject toJSON() throws JSONException {
|
||||||
|
return new JSONObject()
|
||||||
|
.put("fullyQualifiedDomainName", fullyQualifiedDomainName())
|
||||||
|
.put("threatType", threatType());
|
||||||
|
}
|
||||||
|
}
|
49
java/google/registry/beam/spec11/sql/subdomains.sql
Normal file
49
java/google/registry/beam/spec11/sql/subdomains.sql
Normal file
|
@ -0,0 +1,49 @@
|
||||||
|
#standardSQL
|
||||||
|
-- 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.
|
||||||
|
|
||||||
|
-- This query gathers all Subdomains active within a given yearMonth
|
||||||
|
-- and emits a row containing its fully qualified domain name
|
||||||
|
-- [SLD].[TLD], the current registrar's name, and the current registrar's
|
||||||
|
-- email address.
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
domain.fullyQualifiedDomainName AS fullyQualifiedDomainName,
|
||||||
|
registrar.name AS registrarName,
|
||||||
|
registrar.emailAddress AS registrarEmailAddress
|
||||||
|
FROM ( (
|
||||||
|
SELECT
|
||||||
|
fullyQualifiedDomainName,
|
||||||
|
currentSponsorClientId,
|
||||||
|
creationTime
|
||||||
|
FROM
|
||||||
|
`%PROJECT_ID%.%DATASTORE_EXPORT_DATASET%.%DOMAIN_BASE_TABLE%`
|
||||||
|
WHERE
|
||||||
|
-- Only include active registrations
|
||||||
|
-- Registrations that are active (not deleted) will have null deletionTime
|
||||||
|
-- because END_OF_TIME is an invalid timestamp in standardSQL
|
||||||
|
(SAFE_CAST(deletionTime AS STRING) IS NULL
|
||||||
|
OR deletionTime > CURRENT_TIMESTAMP)) AS domain
|
||||||
|
JOIN (
|
||||||
|
SELECT
|
||||||
|
__key__.name AS name,
|
||||||
|
emailAddress
|
||||||
|
FROM
|
||||||
|
`%PROJECT_ID%.%DATASTORE_EXPORT_DATASET%.%REGISTRAR_TABLE%`
|
||||||
|
WHERE
|
||||||
|
type = 'REAL') AS registrar
|
||||||
|
ON
|
||||||
|
domain.currentSponsorClientId = registrar.name)
|
||||||
|
ORDER BY
|
||||||
|
creationTime DESC
|
|
@ -26,6 +26,7 @@ java_library(
|
||||||
"@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_httpclient",
|
||||||
"@org_apache_httpcomponents_httpcore",
|
"@org_apache_httpcomponents_httpcore",
|
||||||
|
"@org_json",
|
||||||
"@org_mockito_all",
|
"@org_mockito_all",
|
||||||
],
|
],
|
||||||
)
|
)
|
||||||
|
|
|
@ -24,7 +24,10 @@ 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 com.google.common.io.CharStreams;
|
||||||
import google.registry.beam.spec11.SafeBrowsingTransforms.EvaluateSafeBrowsingFn;
|
import google.registry.beam.spec11.SafeBrowsingTransforms.EvaluateSafeBrowsingFn;
|
||||||
|
import google.registry.testing.FakeClock;
|
||||||
|
import google.registry.testing.FakeSleeper;
|
||||||
import google.registry.util.ResourceUtils;
|
import google.registry.util.ResourceUtils;
|
||||||
|
import google.registry.util.Retrier;
|
||||||
import java.io.ByteArrayInputStream;
|
import java.io.ByteArrayInputStream;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -32,8 +35,7 @@ import java.io.InputStreamReader;
|
||||||
import java.io.ObjectInputStream;
|
import java.io.ObjectInputStream;
|
||||||
import java.io.ObjectOutputStream;
|
import java.io.ObjectOutputStream;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.time.ZoneId;
|
import java.util.Comparator;
|
||||||
import java.time.ZonedDateTime;
|
|
||||||
import java.util.function.Supplier;
|
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;
|
||||||
|
@ -48,6 +50,9 @@ import org.apache.http.client.methods.HttpPost;
|
||||||
import org.apache.http.entity.BasicHttpEntity;
|
import org.apache.http.entity.BasicHttpEntity;
|
||||||
import org.apache.http.impl.client.CloseableHttpClient;
|
import org.apache.http.impl.client.CloseableHttpClient;
|
||||||
import org.apache.http.message.BasicStatusLine;
|
import org.apache.http.message.BasicStatusLine;
|
||||||
|
import org.json.JSONArray;
|
||||||
|
import org.json.JSONException;
|
||||||
|
import org.json.JSONObject;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
|
@ -55,6 +60,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.invocation.InvocationOnMock;
|
||||||
import org.mockito.stubbing.Answer;
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
/** Unit tests for {@link Spec11Pipeline}. */
|
/** Unit tests for {@link Spec11Pipeline}. */
|
||||||
|
@ -78,21 +84,26 @@ public class Spec11PipelineTest {
|
||||||
public void initializePipeline() throws IOException {
|
public void initializePipeline() throws IOException {
|
||||||
spec11Pipeline = new Spec11Pipeline();
|
spec11Pipeline = new Spec11Pipeline();
|
||||||
spec11Pipeline.projectId = "test-project";
|
spec11Pipeline.projectId = "test-project";
|
||||||
spec11Pipeline.spec11BucketUrl = tempFolder.getRoot().getAbsolutePath() + "/results";
|
spec11Pipeline.spec11BucketUrl = tempFolder.getRoot().getAbsolutePath();
|
||||||
File beamTempFolder = tempFolder.newFolder();
|
File beamTempFolder = tempFolder.newFolder();
|
||||||
spec11Pipeline.beamStagingUrl = beamTempFolder.getAbsolutePath() + "/staging";
|
spec11Pipeline.beamStagingUrl = beamTempFolder.getAbsolutePath() + "/staging";
|
||||||
spec11Pipeline.spec11TemplateUrl = beamTempFolder.getAbsolutePath() + "/templates/invoicing";
|
spec11Pipeline.spec11TemplateUrl = beamTempFolder.getAbsolutePath() + "/templates/invoicing";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static final ImmutableList<String> BAD_DOMAINS =
|
||||||
|
ImmutableList.of("111.com", "222.com", "444.com");
|
||||||
|
|
||||||
private ImmutableList<Subdomain> getInputDomains() {
|
private ImmutableList<Subdomain> getInputDomains() {
|
||||||
ImmutableList.Builder<Subdomain> subdomainsBuilder = new ImmutableList.Builder<>();
|
ImmutableList.Builder<Subdomain> subdomainsBuilder = new ImmutableList.Builder<>();
|
||||||
// Put in 2 batches worth (490 < max < 490*2) to get one positive and one negative example.
|
// Put in at least 2 batches worth (x > 490) to guarantee multiple executions.
|
||||||
for (int i = 0; i < 510; i++) {
|
// Put in half for theRegistrar and half for someRegistrar
|
||||||
|
for (int i = 0; i < 255; i++) {
|
||||||
subdomainsBuilder.add(
|
subdomainsBuilder.add(
|
||||||
Subdomain.create(
|
Subdomain.create(String.format("%s.com", i), "theRegistrar", "fake@theRegistrar.com"));
|
||||||
String.format("%s.com", i),
|
}
|
||||||
ZonedDateTime.of(2017, 9, 29, 0, 0, 0, 0, ZoneId.of("UTC")),
|
for (int i = 255; i < 510; i++) {
|
||||||
"OK"));
|
subdomainsBuilder.add(
|
||||||
|
Subdomain.create(String.format("%s.com", i), "someRegistrar", "fake@someRegistrar.com"));
|
||||||
}
|
}
|
||||||
return subdomainsBuilder.build();
|
return subdomainsBuilder.build();
|
||||||
}
|
}
|
||||||
|
@ -109,75 +120,124 @@ public class Spec11PipelineTest {
|
||||||
// Establish mocks for testing
|
// Establish mocks for testing
|
||||||
ImmutableList<Subdomain> inputRows = getInputDomains();
|
ImmutableList<Subdomain> inputRows = getInputDomains();
|
||||||
CloseableHttpClient httpClient = mock(CloseableHttpClient.class, withSettings().serializable());
|
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
|
// Return a mock HttpResponse that returns a JSON response based on the request.
|
||||||
when(httpClient.execute(any(HttpPost.class)))
|
when(httpClient.execute(any(HttpPost.class))).thenAnswer(new HttpResponder());
|
||||||
.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 =
|
EvaluateSafeBrowsingFn evalFn =
|
||||||
new EvaluateSafeBrowsingFn(
|
new EvaluateSafeBrowsingFn(
|
||||||
StaticValueProvider.of("apikey"), (Serializable & Supplier) () -> httpClient);
|
StaticValueProvider.of("apikey"),
|
||||||
|
new Retrier(new FakeSleeper(new FakeClock()), 3),
|
||||||
|
(Serializable & Supplier) () -> httpClient);
|
||||||
|
|
||||||
// Apply input and evaluation transforms
|
// Apply input and evaluation transforms
|
||||||
PCollection<Subdomain> input = p.apply(Create.of(inputRows));
|
PCollection<Subdomain> input = p.apply(Create.of(inputRows));
|
||||||
spec11Pipeline.evaluateUrlHealth(input, evalFn);
|
spec11Pipeline.evaluateUrlHealth(input, evalFn, StaticValueProvider.of("2018-06"));
|
||||||
p.run();
|
p.run();
|
||||||
|
|
||||||
// Verify output of text file
|
// Verify header and 3 threat matches for 2 registrars are found
|
||||||
ImmutableList<String> generatedReport = resultFileContents();
|
ImmutableList<String> generatedReport = resultFileContents();
|
||||||
// TODO(b/80524726): Rigorously test this output once the pipeline output is finalized.
|
assertThat(generatedReport).hasSize(3);
|
||||||
assertThat(generatedReport).hasSize(2);
|
assertThat(generatedReport.get(0))
|
||||||
assertThat(generatedReport.get(1)).contains("http://111.com");
|
.isEqualTo("Map from registrar email to detected subdomain threats:");
|
||||||
|
|
||||||
|
// The output file can put the registrar emails and bad URLs in any order.
|
||||||
|
// So we sort by length (sorry) to put the shorter JSON first.
|
||||||
|
ImmutableList<String> sortedLines =
|
||||||
|
generatedReport
|
||||||
|
.subList(1, 3)
|
||||||
|
.stream()
|
||||||
|
.sorted(Comparator.comparingInt(String::length))
|
||||||
|
.collect(ImmutableList.toImmutableList());
|
||||||
|
|
||||||
|
JSONObject someRegistrarJSON = new JSONObject(sortedLines.get(0));
|
||||||
|
assertThat(someRegistrarJSON.get("registrarEmailAddress")).isEqualTo("fake@someRegistrar.com");
|
||||||
|
assertThat(someRegistrarJSON.has("threatMatches")).isTrue();
|
||||||
|
JSONArray someThreatMatch = someRegistrarJSON.getJSONArray("threatMatches");
|
||||||
|
assertThat(someThreatMatch.length()).isEqualTo(1);
|
||||||
|
assertThat(someThreatMatch.getJSONObject(0).get("fullyQualifiedDomainName"))
|
||||||
|
.isEqualTo("444.com");
|
||||||
|
assertThat(someThreatMatch.getJSONObject(0).get("threatType"))
|
||||||
|
.isEqualTo("MALWARE");
|
||||||
|
|
||||||
|
// theRegistrar has two ThreatMatches, we have to parse it explicitly
|
||||||
|
JSONObject theRegistrarJSON = new JSONObject(sortedLines.get(1));
|
||||||
|
assertThat(theRegistrarJSON.get("registrarEmailAddress")).isEqualTo("fake@theRegistrar.com");
|
||||||
|
assertThat(theRegistrarJSON.has("threatMatches")).isTrue();
|
||||||
|
JSONArray theThreatMatches = theRegistrarJSON.getJSONArray("threatMatches");
|
||||||
|
assertThat(theThreatMatches.length()).isEqualTo(2);
|
||||||
|
ImmutableList<String> threatMatchStrings =
|
||||||
|
ImmutableList.of(
|
||||||
|
theThreatMatches.getJSONObject(0).toString(),
|
||||||
|
theThreatMatches.getJSONObject(1).toString());
|
||||||
|
assertThat(threatMatchStrings)
|
||||||
|
.containsExactly(
|
||||||
|
new JSONObject()
|
||||||
|
.put("fullyQualifiedDomainName", "111.com")
|
||||||
|
.put("threatType", "MALWARE")
|
||||||
|
.toString(),
|
||||||
|
new JSONObject()
|
||||||
|
.put("fullyQualifiedDomainName", "222.com")
|
||||||
|
.put("threatType", "MALWARE")
|
||||||
|
.toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns the text contents of a file under the beamBucket/results directory. */
|
/**
|
||||||
private ImmutableList<String> resultFileContents() throws Exception {
|
* A serializable {@link Answer} that returns a mock HTTP response based on the HTTP request's
|
||||||
File resultFile = new File(String.format("%s/results", tempFolder.getRoot().getAbsolutePath()));
|
* content.
|
||||||
return ImmutableList.copyOf(
|
*/
|
||||||
ResourceUtils.readResourceUtf8(resultFile.toURI().toURL()).split("\n"));
|
private static class HttpResponder implements Answer<CloseableHttpResponse>, Serializable {
|
||||||
|
@Override
|
||||||
|
public CloseableHttpResponse answer(InvocationOnMock invocation) throws Throwable {
|
||||||
|
return getMockResponse(
|
||||||
|
CharStreams.toString(
|
||||||
|
new InputStreamReader(
|
||||||
|
((HttpPost) invocation.getArguments()[0]).getEntity().getContent(), UTF_8)));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns a filled-in template for threat detected at a given url. */
|
/**
|
||||||
private static String getBadUrlMatch(String url) {
|
* Returns a {@link CloseableHttpResponse} containing either positive (threat found) or negative
|
||||||
return "{\n"
|
* (no threat) API examples based on the request data.
|
||||||
+ " \"matches\": [{\n"
|
*/
|
||||||
+ " \"threatType\": \"MALWARE\",\n"
|
private static CloseableHttpResponse getMockResponse(String request) throws JSONException {
|
||||||
+ " \"platformType\": \"WINDOWS\",\n"
|
// Determine which bad URLs are in the request (if any)
|
||||||
+ " \"threatEntryType\": \"URL\",\n"
|
ImmutableList<String> badUrls =
|
||||||
+ String.format(" \"threat\": {\"url\": \"%s\"},\n", url)
|
BAD_DOMAINS.stream().filter(request::contains).collect(ImmutableList.toImmutableList());
|
||||||
+ " \"threatEntryMetadata\": {\n"
|
|
||||||
+ " \"entries\": [{\n"
|
CloseableHttpResponse httpResponse =
|
||||||
+ " \"key\": \"malware_threat_type\",\n"
|
mock(CloseableHttpResponse.class, withSettings().serializable());
|
||||||
+ " \"value\": \"landing\"\n"
|
when(httpResponse.getStatusLine())
|
||||||
+ " }]\n"
|
.thenReturn(
|
||||||
+ " },\n"
|
new BasicStatusLine(new ProtocolVersion("HTTP", 1, 1), 200, "Done"));
|
||||||
+ " \"cacheDuration\": \"300.000s\"\n"
|
when(httpResponse.getEntity())
|
||||||
+ " },"
|
.thenReturn(new FakeHttpEntity(getAPIResponse(badUrls)));
|
||||||
+ "]\n"
|
return httpResponse;
|
||||||
+ "}";
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the expected API response for a list of bad URLs.
|
||||||
|
*
|
||||||
|
* <p>If there are no badUrls in the list, this returns the empty JSON string "{}".
|
||||||
|
*/
|
||||||
|
private static String getAPIResponse(ImmutableList<String> badUrls) throws JSONException {
|
||||||
|
JSONObject response = new JSONObject();
|
||||||
|
if (badUrls.isEmpty()) {
|
||||||
|
return response.toString();
|
||||||
|
}
|
||||||
|
// Create a threatMatch for each badUrl
|
||||||
|
JSONArray matches = new JSONArray();
|
||||||
|
for (String badUrl : badUrls) {
|
||||||
|
matches.put(
|
||||||
|
new JSONObject()
|
||||||
|
.put("threatType", "MALWARE")
|
||||||
|
.put("platformType", "WINDOWS")
|
||||||
|
.put("threatEntryType", "URL")
|
||||||
|
.put("threat", new JSONObject().put("url", badUrl))
|
||||||
|
.put("cacheDuration", "300.000s"));
|
||||||
|
}
|
||||||
|
response.put("matches", matches);
|
||||||
|
return response.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** A serializable HttpEntity fake that returns {@link String} content. */
|
/** A serializable HttpEntity fake that returns {@link String} content. */
|
||||||
|
@ -191,6 +251,12 @@ public class Spec11PipelineTest {
|
||||||
oos.defaultWriteObject();
|
oos.defaultWriteObject();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the {@link FakeHttpEntity} content upon deserialization.
|
||||||
|
*
|
||||||
|
* <p>This allows us to use {@link #getContent()} as-is, fully emulating the behavior of {@link
|
||||||
|
* BasicHttpEntity} regardless of serialization.
|
||||||
|
*/
|
||||||
private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
|
private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
|
||||||
ois.defaultReadObject();
|
ois.defaultReadObject();
|
||||||
super.setContent(new ByteArrayInputStream(this.content.getBytes(UTF_8)));
|
super.setContent(new ByteArrayInputStream(this.content.getBytes(UTF_8)));
|
||||||
|
@ -198,6 +264,18 @@ public class Spec11PipelineTest {
|
||||||
|
|
||||||
FakeHttpEntity(String content) {
|
FakeHttpEntity(String content) {
|
||||||
this.content = content;
|
this.content = content;
|
||||||
|
super.setContent(new ByteArrayInputStream(this.content.getBytes(UTF_8)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Returns the text contents of a file under the beamBucket/results directory. */
|
||||||
|
private ImmutableList<String> resultFileContents() throws Exception {
|
||||||
|
File resultFile =
|
||||||
|
new File(
|
||||||
|
String.format(
|
||||||
|
"%s/2018-06/2018-06-monthly-report", tempFolder.getRoot().getAbsolutePath()));
|
||||||
|
return ImmutableList.copyOf(
|
||||||
|
ResourceUtils.readResourceUtf8(resultFile.toURI().toURL()).split("\n"));
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue