Add a scrap command to backfill Spec11 threats (#897)

This parses through all pre-existing Spec11 files in GCS (starting at
2019-01-01 which is basically when the new format started) and maps them
to the new Spec11ThreatMatch objects.

Because the old format stored domain names only and the new format stores
names + repo IDs, we need to retrieve the DomainBase objects from the
point in time of the scan (failing if they don't exist). Because the
same domains appear multiple times (we estimate a total of 100k+ entries
but only 1-2k unique domains) we cache the DomainBase objects that we
retrieve from Datastore.
This commit is contained in:
gbrodman 2020-12-15 16:18:27 -05:00 committed by GitHub
parent 090eeaa618
commit 1db0c5b018
11 changed files with 4652 additions and 4402 deletions

View file

@ -21,10 +21,10 @@ import javax.persistence.TemporalType;
import org.joda.time.LocalDate;
/**
* Data access object for {@link google.registry.model.reporting.Spec11ThreatMatch}.
* Data access object for {@link Spec11ThreatMatch}.
*
* <p>A JpaTransactionManager is passed into each static method because they are called from a BEAM
* pipeline and we don't know where it's coming from.
* <p>The transaction manager is passed as a parameter because this could be called either from a
* BEAM pipeline or standard non-BEAM code.
*/
public class Spec11ThreatMatchDao {

View file

@ -14,9 +14,11 @@
package google.registry.reporting.spec11;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static java.nio.charset.StandardCharsets.UTF_8;
import com.google.appengine.tools.cloudstorage.GcsFilename;
import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.flogger.FluentLogger;
@ -59,6 +61,25 @@ public class Spec11RegistrarThreatMatchesParser {
return getFromFile(getGcsFilename(date));
}
/** Returns registrar:set-of-threat-match pairings from the file, or empty if it doesn't exist. */
public ImmutableSet<RegistrarThreatMatches> getFromFile(GcsFilename spec11ReportFilename)
throws IOException {
if (!gcsUtils.existsAndNotEmpty(spec11ReportFilename)) {
return ImmutableSet.of();
}
ImmutableSet.Builder<RegistrarThreatMatches> builder = ImmutableSet.builder();
try (InputStream in = gcsUtils.openInputStream(spec11ReportFilename);
InputStreamReader isr = new InputStreamReader(in, UTF_8)) {
// Skip the header at line 0
return Splitter.on("\n")
.omitEmptyStrings()
.splitToStream(CharStreams.toString(isr))
.skip(1)
.map(this::parseRegistrarThreatMatch)
.collect(toImmutableSet());
}
}
public Optional<LocalDate> getPreviousDateWithMatches(LocalDate date) {
LocalDate yesterday = date.minusDays(1);
GcsFilename gcsFilename = getGcsFilename(yesterday);
@ -82,20 +103,6 @@ public class Spec11RegistrarThreatMatchesParser {
return new GcsFilename(reportingBucket, Spec11Pipeline.getSpec11ReportFilePath(localDate));
}
private ImmutableSet<RegistrarThreatMatches> getFromFile(GcsFilename spec11ReportFilename)
throws IOException, JSONException {
ImmutableSet.Builder<RegistrarThreatMatches> builder = ImmutableSet.builder();
try (InputStream in = gcsUtils.openInputStream(spec11ReportFilename)) {
ImmutableList<String> reportLines =
ImmutableList.copyOf(CharStreams.toString(new InputStreamReader(in, UTF_8)).split("\n"));
// Iterate from 1 to size() to skip the header at line 0.
for (int i = 1; i < reportLines.size(); i++) {
builder.add(parseRegistrarThreatMatch(reportLines.get(i)));
}
return builder.build();
}
}
private RegistrarThreatMatches parseRegistrarThreatMatch(String line) throws JSONException {
JSONObject reportJSON = new JSONObject(line);
String clientId = reportJSON.getString(Spec11Pipeline.REGISTRAR_CLIENT_ID_FIELD);

View file

@ -16,6 +16,7 @@ package google.registry.tools;
import com.google.common.collect.ImmutableMap;
import google.registry.tools.javascrap.BackfillRegistryLocksCommand;
import google.registry.tools.javascrap.BackfillSpec11ThreatMatchesCommand;
import google.registry.tools.javascrap.PopulateNullRegistrarFieldsCommand;
import google.registry.tools.javascrap.RemoveIpAddressCommand;
@ -32,6 +33,7 @@ public final class RegistryTool {
new ImmutableMap.Builder<String, Class<? extends Command>>()
.put("ack_poll_messages", AckPollMessagesCommand.class)
.put("backfill_registry_locks", BackfillRegistryLocksCommand.class)
.put("backfill_spec11_threat_matches", BackfillSpec11ThreatMatchesCommand.class)
.put("canonicalize_labels", CanonicalizeLabelsCommand.class)
.put("check_domain", CheckDomainCommand.class)
.put("check_domain_claims", CheckDomainClaimsCommand.class)

View file

@ -0,0 +1,216 @@
// Copyright 2020 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.tools.javascrap;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.collect.ImmutableListMultimap.flatteningToImmutableListMultimap;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static google.registry.model.ofy.ObjectifyService.ofy;
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableListMultimap;
import com.google.common.collect.ImmutableSet;
import google.registry.beam.spec11.ThreatMatch;
import google.registry.model.domain.DomainBase;
import google.registry.model.reporting.Spec11ThreatMatch;
import google.registry.model.reporting.Spec11ThreatMatch.ThreatType;
import google.registry.model.reporting.Spec11ThreatMatchDao;
import google.registry.reporting.spec11.RegistrarThreatMatches;
import google.registry.reporting.spec11.Spec11RegistrarThreatMatchesParser;
import google.registry.tools.CommandWithRemoteApi;
import google.registry.tools.ConfirmingCommand;
import google.registry.util.Clock;
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.function.Function;
import javax.inject.Inject;
import org.joda.time.LocalDate;
/**
* Scrap tool to backfill {@link Spec11ThreatMatch} objects from prior days.
*
* <p>This will load the previously-existing Spec11 files from GCS (looking back to 2019-01-01 (a
* rough estimate of when we started using this format) and convert those RegistrarThreatMatches
* objects into the new Spec11ThreatMatch format. It will then insert these entries into SQL.
*
* <p>Note that the script will attempt to find the corresponding {@link DomainBase} object for each
* domain name on the day of the scan. It will fail if it cannot find a corresponding domain object,
* or if the domain objects were not active at the time of the scan.
*/
@Parameters(
commandDescription =
"Backfills Spec11 threat match entries from the old and deprecated GCS JSON files to the "
+ "Cloud SQL database.")
public class BackfillSpec11ThreatMatchesCommand extends ConfirmingCommand
implements CommandWithRemoteApi {
private static final LocalDate START_DATE = new LocalDate(2019, 1, 1);
@Parameter(
names = {"-o", "--overwrite_existing_dates"},
description =
"Whether the command will overwrite data that already exists for dates that exist in the "
+ "GCS bucket. Defaults to false.")
private boolean overrideExistingDates;
@Inject Spec11RegistrarThreatMatchesParser threatMatchesParser;
// Inject the clock for testing purposes
@Inject Clock clock;
@Override
protected String prompt() {
return String.format("Backfill Spec11 results from %d files?", getDatesToBackfill().size());
}
@Override
protected String execute() {
ImmutableList<LocalDate> dates = getDatesToBackfill();
ImmutableListMultimap.Builder<LocalDate, RegistrarThreatMatches> threatMatchesBuilder =
new ImmutableListMultimap.Builder<>();
for (LocalDate date : dates) {
try {
// It's OK if the file doesn't exist for a particular date; the result will be empty.
threatMatchesBuilder.putAll(date, threatMatchesParser.getRegistrarThreatMatches(date));
} catch (IOException e) {
throw new RuntimeException(
String.format("Error parsing through file with date %s.", date), e);
}
}
ImmutableListMultimap<LocalDate, RegistrarThreatMatches> threatMatches =
threatMatchesBuilder.build();
// Look up all possible DomainBases for these domain names, any of which can be in the past
ImmutableListMultimap<String, DomainBase> domainsByDomainName =
getDomainsByDomainName(threatMatches);
// For each date, convert all threat matches with the proper domain repo ID
int totalNumThreats = 0;
for (LocalDate date : threatMatches.keySet()) {
ImmutableList.Builder<Spec11ThreatMatch> spec11ThreatsBuilder = new ImmutableList.Builder<>();
for (RegistrarThreatMatches rtm : threatMatches.get(date)) {
rtm.threatMatches().stream()
.map(
threatMatch ->
threatMatchToCloudSqlObject(
threatMatch, date, rtm.clientId(), domainsByDomainName))
.forEach(spec11ThreatsBuilder::add);
}
ImmutableList<Spec11ThreatMatch> spec11Threats = spec11ThreatsBuilder.build();
jpaTm()
.transact(
() -> {
Spec11ThreatMatchDao.deleteEntriesByDate(jpaTm(), date);
jpaTm().putAll(spec11Threats);
});
totalNumThreats += spec11Threats.size();
}
return String.format(
"Successfully parsed through %d files with %d threats.", dates.size(), totalNumThreats);
}
/** Returns a per-domain list of possible DomainBase objects, starting with the most recent. */
private ImmutableListMultimap<String, DomainBase> getDomainsByDomainName(
ImmutableListMultimap<LocalDate, RegistrarThreatMatches> threatMatchesByDate) {
return threatMatchesByDate.values().stream()
.map(RegistrarThreatMatches::threatMatches)
.flatMap(ImmutableList::stream)
.map(ThreatMatch::fullyQualifiedDomainName)
.distinct()
.collect(
flatteningToImmutableListMultimap(
Function.identity(),
(domainName) -> {
List<DomainBase> domains =
ofy()
.load()
.type(DomainBase.class)
.filter("fullyQualifiedDomainName", domainName)
.list();
domains.sort(Comparator.comparing(DomainBase::getCreationTime).reversed());
checkState(
!domains.isEmpty(),
"Domain name %s had no associated DomainBase objects.",
domainName);
return domains.stream();
}));
}
/** Converts the previous {@link ThreatMatch} object to {@link Spec11ThreatMatch}. */
private Spec11ThreatMatch threatMatchToCloudSqlObject(
ThreatMatch threatMatch,
LocalDate date,
String registrarId,
ImmutableListMultimap<String, DomainBase> domainsByDomainName) {
DomainBase domain =
findDomainAsOfDateOrThrow(
threatMatch.fullyQualifiedDomainName(), date, domainsByDomainName);
return new Spec11ThreatMatch.Builder()
.setThreatTypes(ImmutableSet.of(ThreatType.valueOf(threatMatch.threatType())))
.setCheckDate(date)
.setRegistrarId(registrarId)
.setDomainName(threatMatch.fullyQualifiedDomainName())
.setDomainRepoId(domain.getRepoId())
.build();
}
/** Returns the DomainBase object as of the particular date, which is likely in the past. */
private DomainBase findDomainAsOfDateOrThrow(
String domainName,
LocalDate date,
ImmutableListMultimap<String, DomainBase> domainsByDomainName) {
ImmutableList<DomainBase> domains = domainsByDomainName.get(domainName);
for (DomainBase domain : domains) {
// We only know the date (not datetime) of the threat scan, so we approximate
LocalDate creationDate = domain.getCreationTime().toLocalDate();
LocalDate deletionDate = domain.getDeletionTime().toLocalDate();
if (!date.isBefore(creationDate) && !date.isAfter(deletionDate)) {
return domain;
}
}
throw new IllegalStateException(
String.format("Could not find a DomainBase valid for %s on day %s.", domainName, date));
}
/** Returns the list of dates between {@link #START_DATE} and now (UTC), inclusive. */
private ImmutableList<LocalDate> getDatesToBackfill() {
ImmutableSet<LocalDate> datesToSkip =
overrideExistingDates ? ImmutableSet.of() : getExistingDates();
ImmutableList.Builder<LocalDate> result = new ImmutableList.Builder<>();
LocalDate endDate = clock.nowUtc().toLocalDate();
for (LocalDate currentDate = START_DATE;
!currentDate.isAfter(endDate);
currentDate = currentDate.plusDays(1)) {
if (!datesToSkip.contains(currentDate)) {
result.add(currentDate);
}
}
return result.build();
}
private ImmutableSet<LocalDate> getExistingDates() {
return jpaTm()
.transact(
() ->
jpaTm()
.getEntityManager()
.createQuery(
"SELECT DISTINCT stm.checkDate FROM Spec11ThreatMatch stm", LocalDate.class)
.getResultStream()
.collect(toImmutableSet()));
}
}

View file

@ -34,7 +34,7 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
/** Unit tests for {@link Spec11RegistrarThreatMatchesParser}. */
class Spec11RegistrarThreatMatchesParserTest {
public class Spec11RegistrarThreatMatchesParserTest {
private static final String TODAY = "2018-07-21";
private static final String YESTERDAY = "2018-07-20";
@ -66,6 +66,12 @@ class Spec11RegistrarThreatMatchesParserTest {
assertThat(parser.getPreviousDateWithMatches(LocalDate.parse(TODAY))).isEmpty();
}
@Test
void testNonexistent_returnsEmpty() throws Exception {
assertThat(parser.getRegistrarThreatMatches(LocalDate.parse(YESTERDAY).minusYears(1)))
.isEmpty();
}
@Test
void testFindPrevious_olderThanYesterdayFound() throws Exception {
setupFile("spec11_fake_report_previous_day", "2018-07-14");
@ -95,7 +101,7 @@ class Spec11RegistrarThreatMatchesParserTest {
}
/** The expected contents of the sample spec11 report file */
static ImmutableSet<RegistrarThreatMatches> sampleThreatMatches() throws Exception {
public static ImmutableSet<RegistrarThreatMatches> sampleThreatMatches() throws Exception {
return ImmutableSet.of(getMatchA(), getMatchB());
}

View file

@ -0,0 +1,271 @@
// Copyright 2020 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.tools.javascrap;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static com.google.common.truth.Truth.assertThat;
import static google.registry.model.ImmutableObjectSubject.assertAboutImmutableObjects;
import static google.registry.model.ImmutableObjectSubject.immutableObjectCorrespondence;
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
import static google.registry.reporting.spec11.Spec11RegistrarThreatMatchesParserTest.sampleThreatMatches;
import static google.registry.testing.DatabaseHelper.createTld;
import static google.registry.testing.DatabaseHelper.deleteResource;
import static google.registry.testing.DatabaseHelper.newDomainBase;
import static google.registry.testing.DatabaseHelper.persistActiveDomain;
import static google.registry.testing.DatabaseHelper.persistResource;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import google.registry.model.domain.DomainBase;
import google.registry.model.reporting.Spec11ThreatMatch;
import google.registry.model.reporting.Spec11ThreatMatch.ThreatType;
import google.registry.reporting.spec11.Spec11RegistrarThreatMatchesParser;
import google.registry.tools.CommandTestCase;
import java.io.IOException;
import org.joda.time.DateTime;
import org.joda.time.LocalDate;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
/** Tests for {@link BackfillSpec11ThreatMatchesCommand}. */
public class BackfillSpec11ThreatMatchesCommandTest
extends CommandTestCase<BackfillSpec11ThreatMatchesCommand> {
private static final LocalDate CURRENT_DATE = DateTime.parse("2020-11-22").toLocalDate();
private final Spec11RegistrarThreatMatchesParser threatMatchesParser =
mock(Spec11RegistrarThreatMatchesParser.class);
private DomainBase domainA;
@BeforeEach
void beforeEach() throws Exception {
createTld("com");
domainA = persistActiveDomain("a.com");
persistActiveDomain("b.com");
persistActiveDomain("c.com");
fakeClock.setTo(CURRENT_DATE.toDateTimeAtStartOfDay());
command.threatMatchesParser = threatMatchesParser;
command.clock = fakeClock;
when(threatMatchesParser.getRegistrarThreatMatches(any(LocalDate.class)))
.thenReturn(ImmutableSet.of());
}
@Test
void testSuccess_singleFile() throws Exception {
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE))
.thenReturn(sampleThreatMatches());
runCommandForced();
assertInStdout("Backfill Spec11 results from 692 files?");
assertInStdout("Successfully parsed through 692 files with 3 threats.");
verifyExactlyThreeEntriesInDbFromLastDay();
}
@Test
void testSuccess_sameDomain_multipleDays() throws Exception {
// If the same domains show up on multiple days, there should be multiple entries for them
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE))
.thenReturn(sampleThreatMatches());
when(threatMatchesParser.getRegistrarThreatMatches(LocalDate.parse("2019-01-01")))
.thenReturn(sampleThreatMatches());
runCommandForced();
assertInStdout("Backfill Spec11 results from 692 files?");
assertInStdout("Successfully parsed through 692 files with 6 threats.");
jpaTm()
.transact(
() -> {
ImmutableList<Spec11ThreatMatch> threatMatches =
jpaTm().loadAll(Spec11ThreatMatch.class);
assertThat(threatMatches).hasSize(6);
assertThat(
threatMatches.stream()
.map(Spec11ThreatMatch::getDomainName)
.collect(toImmutableSet()))
.containsExactly("a.com", "b.com", "c.com");
assertThat(
threatMatches.stream()
.map(Spec11ThreatMatch::getCheckDate)
.collect(toImmutableSet()))
.containsExactly(CURRENT_DATE, LocalDate.parse("2019-01-01"));
});
}
@Test
void testSuccess_empty() throws Exception {
runCommandForced();
assertInStdout("Backfill Spec11 results from 692 files?");
assertInStdout("Successfully parsed through 692 files with 0 threats.");
}
@Test
void testSuccess_sameDayTwice() throws Exception {
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE))
.thenReturn(sampleThreatMatches());
runCommandForced();
runCommandForced();
verifyExactlyThreeEntriesInDbFromLastDay();
}
@Test
void testSuccess_threeDomainsForDomainName() throws Exception {
// We should use the repo ID from the proper DomainBase object at the scan's point in time.
// First, domain was created at START_OF_TIME and deleted one year ago
DateTime now = fakeClock.nowUtc();
domainA = persistResource(domainA.asBuilder().setDeletionTime(now.minusYears(1)).build());
// Next, domain was created six months ago and deleted two months ago
DomainBase secondSave =
persistResource(
newDomainBase("a.com")
.asBuilder()
.setCreationTimeForTest(now.minusMonths(6))
.setDeletionTime(now.minusMonths(2))
.build());
// Lastly, domain was created one month ago and is still valid
DomainBase thirdSave =
persistResource(
newDomainBase("a.com").asBuilder().setCreationTimeForTest(now.minusMonths(1)).build());
// If the scan result was from three months ago, we should use the second save
when(threatMatchesParser.getRegistrarThreatMatches(now.toLocalDate().minusMonths(3)))
.thenReturn(sampleThreatMatches());
runCommandForced();
String threatMatchRepoId =
jpaTm()
.transact(
() ->
jpaTm().loadAll(Spec11ThreatMatch.class).stream()
.filter((match) -> match.getDomainName().equals("a.com"))
.findFirst()
.get()
.getDomainRepoId());
assertThat(threatMatchRepoId).isNotEqualTo(domainA.getRepoId());
assertThat(threatMatchRepoId).isEqualTo(secondSave.getRepoId());
assertThat(threatMatchRepoId).isNotEqualTo(thirdSave.getRepoId());
}
@Test
void testSuccess_skipsExistingDatesWithoutOverwrite() throws Exception {
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE))
.thenReturn(sampleThreatMatches());
Spec11ThreatMatch previous =
new Spec11ThreatMatch.Builder()
.setCheckDate(CURRENT_DATE)
.setDomainName("previous.tld")
.setDomainRepoId("1-DOMAIN")
.setRegistrarId("TheRegistrar")
.setThreatTypes(ImmutableSet.of(ThreatType.MALWARE))
.build();
jpaTm().transact(() -> jpaTm().put(previous));
runCommandForced();
ImmutableList<Spec11ThreatMatch> threatMatches =
jpaTm().transact(() -> jpaTm().loadAll(Spec11ThreatMatch.class));
assertAboutImmutableObjects()
.that(Iterables.getOnlyElement(threatMatches))
.isEqualExceptFields(previous, "id");
}
@Test
void testSuccess_overwritesExistingDatesWhenSpecified() throws Exception {
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE))
.thenReturn(sampleThreatMatches());
Spec11ThreatMatch previous =
new Spec11ThreatMatch.Builder()
.setCheckDate(CURRENT_DATE)
.setDomainName("previous.tld")
.setDomainRepoId("1-DOMAIN")
.setRegistrarId("TheRegistrar")
.setThreatTypes(ImmutableSet.of(ThreatType.MALWARE))
.build();
jpaTm().transact(() -> jpaTm().put(previous));
runCommandForced("--overwrite_existing_dates");
verifyExactlyThreeEntriesInDbFromLastDay();
}
@Test
void testFailure_oneFileFails() throws Exception {
// If there are any exceptions, we should fail loud and fast
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE))
.thenReturn(sampleThreatMatches());
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE.minusDays(1)))
.thenThrow(new IOException("hi"));
RuntimeException runtimeException =
assertThrows(RuntimeException.class, this::runCommandForced);
assertThat(runtimeException.getCause().getClass()).isEqualTo(IOException.class);
assertThat(runtimeException).hasCauseThat().hasMessageThat().isEqualTo("hi");
jpaTm().transact(() -> assertThat(jpaTm().loadAll(Spec11ThreatMatch.class)).isEmpty());
}
@Test
void testFailure_noDomainForDomainName() throws Exception {
deleteResource(domainA);
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE))
.thenReturn(sampleThreatMatches());
assertThat(assertThrows(IllegalStateException.class, this::runCommandForced))
.hasMessageThat()
.isEqualTo("Domain name a.com had no associated DomainBase objects.");
}
@Test
void testFailure_noDomainAtTimeOfScan() throws Exception {
// If the domain existed at some point(s) in time but not the time of the scan, fail.
// First, domain was created at START_OF_TIME and deleted one year ago
DateTime now = fakeClock.nowUtc();
domainA = persistResource(domainA.asBuilder().setDeletionTime(now.minusYears(1)).build());
// Second, domain was created one month ago and is still valid
persistResource(
newDomainBase("a.com").asBuilder().setCreationTimeForTest(now.minusMonths(1)).build());
// If we have a result for this domain from 3 months ago when it didn't exist, fail.
when(threatMatchesParser.getRegistrarThreatMatches(now.toLocalDate().minusMonths(3)))
.thenReturn(sampleThreatMatches());
assertThat(assertThrows(IllegalStateException.class, this::runCommandForced))
.hasMessageThat()
.isEqualTo("Could not find a DomainBase valid for a.com on day 2020-08-22.");
}
private void verifyExactlyThreeEntriesInDbFromLastDay() {
jpaTm()
.transact(
() -> {
ImmutableList<Spec11ThreatMatch> threatMatches =
jpaTm().loadAll(Spec11ThreatMatch.class);
assertThat(threatMatches)
.comparingElementsUsing(immutableObjectCorrespondence("id", "domainRepoId"))
.containsExactly(
expectedThreatMatch("TheRegistrar", "a.com"),
expectedThreatMatch("NewRegistrar", "b.com"),
expectedThreatMatch("NewRegistrar", "c.com"));
});
}
private Spec11ThreatMatch expectedThreatMatch(String registrarId, String domainName) {
return new Spec11ThreatMatch.Builder()
.setDomainRepoId("ignored")
.setDomainName(domainName)
.setRegistrarId(registrarId)
.setCheckDate(CURRENT_DATE)
.setThreatTypes(ImmutableSet.of(ThreatType.MALWARE))
.build();
}
}

File diff suppressed because it is too large Load diff

File diff suppressed because it is too large Load diff

View file

@ -78,3 +78,4 @@ V77__fixes_for_replay.sql
V78__add_history_id_for_redemption_history_entry.sql
V79__drop_foreign_keys_on_pollmessage.sql
V80__defer_bill_event_key.sql
V81__drop_spec11_fkeys.sql

View file

@ -0,0 +1,18 @@
-- Copyright 2020 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.
-- Some objects haven't been fully populated in SQL yet, so don't depend on them.
ALTER TABLE "Spec11ThreatMatch" DROP CONSTRAINT "fk_spec11_threat_match_domain_repo_id";
ALTER TABLE "Spec11ThreatMatch" DROP CONSTRAINT "fk_spec11_threat_match_registrar_id";
ALTER TABLE "Spec11ThreatMatch" DROP CONSTRAINT "fk_spec11_threat_match_tld";

View file

@ -2251,30 +2251,6 @@ ALTER TABLE ONLY public."RegistrarPoc"
ADD CONSTRAINT fk_registrar_poc_registrar_id FOREIGN KEY (registrar_id) REFERENCES public."Registrar"(registrar_id);
--
-- Name: Spec11ThreatMatch fk_spec11_threat_match_domain_repo_id; Type: FK CONSTRAINT; Schema: public; Owner: -
--
ALTER TABLE ONLY public."Spec11ThreatMatch"
ADD CONSTRAINT fk_spec11_threat_match_domain_repo_id FOREIGN KEY (domain_repo_id) REFERENCES public."Domain"(repo_id);
--
-- Name: Spec11ThreatMatch fk_spec11_threat_match_registrar_id; Type: FK CONSTRAINT; Schema: public; Owner: -
--
ALTER TABLE ONLY public."Spec11ThreatMatch"
ADD CONSTRAINT fk_spec11_threat_match_registrar_id FOREIGN KEY (registrar_id) REFERENCES public."Registrar"(registrar_id);
--
-- Name: Spec11ThreatMatch fk_spec11_threat_match_tld; Type: FK CONSTRAINT; Schema: public; Owner: -
--
ALTER TABLE ONLY public."Spec11ThreatMatch"
ADD CONSTRAINT fk_spec11_threat_match_tld FOREIGN KEY (tld) REFERENCES public."Tld"(tld_name);
--
-- Name: DomainHistoryHost fka9woh3hu8gx5x0vly6bai327n; Type: FK CONSTRAINT; Schema: public; Owner: -
--