1
0
mirror of https://github.com/google/nomulus synced 2026-01-27 16:12:19 +00:00

Compare commits

..

6 Commits

Author SHA1 Message Date
Ben McIlwain
e25885e25f Remove obsolete scrap commands (#1502) 2022-01-25 15:23:00 -05:00
gbrodman
cbdf4704ba Add missing @Overrides (#1499)
Not sure how this snuck through
2022-01-24 16:58:38 -05:00
Weimin Yu
207c7e7ca8 Compare migration data with SQL as primary DB (#1497)
* Compare migration data with SQL as primary DB

Add a BEAM pipeline that compares the secondary Datastore against SQL.
This is a dumb pipeline to be launched by a driver (in a followup PR).
Manually tested pipeline in sandbox.

Also updated the ValidateSqlPipeline and the snapshot finder class so
that an appropriate Datastore export is found (one that ends before the
replay checkpoint value).
2022-01-24 11:20:48 -05:00
Lai Jiang
b3a0eb6bd8 Add a cron job to run the RDE Beam pipeline in parallel with MapReduce (#1500) 2022-01-21 23:36:13 -05:00
gbrodman
c602aa6e67 Use the read-only replica for JPA invoicing (#1494)
* Use the read-only replica for JPA invoicing
2022-01-20 20:50:10 +00:00
gbrodman
c6008b65a0 Use a read-only replica SQL instance in RdapDomainSearchAction (#1495)
We can use it more places later but this can serve as a template. We
should inject the connection to the read-only replica (only created
once) to the constructor of the action, then use that instead of the
regular transaction manager.

We add a transaction manager that simulates the read-only-replica
behavior for testing purposes as well.

In addition, we set the transaction isolation level to READ COMMITTED
for this transaction manager (this is fine since we're never writing to
it). Postgres requires this for replica SQL access (it fails if we try
to use SERIALIZABLE) transactions. We didn't see this with the pipelines
before since those already had transaction isolation level overrides
2022-01-20 15:39:07 -05:00
24 changed files with 704 additions and 1312 deletions

View File

@@ -708,6 +708,9 @@ createToolTask(
createToolTask(
'validateSqlPipeline', 'google.registry.beam.comparedb.ValidateSqlPipeline')
createToolTask(
'validateDatastorePipeline', 'google.registry.beam.comparedb.ValidateDatastorePipeline')
createToolTask(
'jpaDemoPipeline', 'google.registry.beam.common.JpaDemoPipeline')

View File

@@ -53,11 +53,11 @@ public class LatestDatastoreSnapshotFinder {
}
/**
* Finds information of the most recent Datastore snapshot, including the GCS folder of the
* exported data files and the start and stop times of the export. The folder of the CommitLogs is
* also included in the return.
* Finds information of the most recent Datastore snapshot that ends strictly before {@code
* exportEndTimeUpperBound}, including the GCS folder of the exported data files and the start and
* stop times of the export. The folder of the CommitLogs is also included in the return.
*/
public DatastoreSnapshotInfo getSnapshotInfo() {
public DatastoreSnapshotInfo getSnapshotInfo(Instant exportEndTimeUpperBound) {
String bucketName = RegistryConfig.getDatastoreBackupsBucket().substring("gs://".length());
/**
* Find the bucket-relative path to the overall metadata file of the last Datastore export.
@@ -65,7 +65,8 @@ public class LatestDatastoreSnapshotFinder {
* return value is like
* "2021-11-19T06:00:00_76493/2021-11-19T06:00:00_76493.overall_export_metadata".
*/
Optional<String> metaFilePathOptional = findMostRecentExportMetadataFile(bucketName, 2);
Optional<String> metaFilePathOptional =
findNewestExportMetadataFileBeforeTime(bucketName, exportEndTimeUpperBound, 2);
if (!metaFilePathOptional.isPresent()) {
throw new NoSuchElementException("No exports found over the past 2 days.");
}
@@ -85,8 +86,9 @@ public class LatestDatastoreSnapshotFinder {
}
/**
* Finds the bucket-relative path of the overall export metadata file, in the given bucket,
* searching back up to {@code lookBackDays} days, including today.
* Finds the latest Datastore export that ends strictly before {@code endTimeUpperBound} and
* returns the bucket-relative path of the overall export metadata file, in the given bucket. The
* search goes back for up to {@code lookBackDays} days in time, including today.
*
* <p>The overall export metadata file is the last file created during a Datastore export. All
* data has been exported by the creation time of this file. The name of this file, like that of
@@ -95,7 +97,8 @@ public class LatestDatastoreSnapshotFinder {
* <p>An example return value: {@code
* 2021-11-19T06:00:00_76493/2021-11-19T06:00:00_76493.overall_export_metadata}.
*/
private Optional<String> findMostRecentExportMetadataFile(String bucketName, int lookBackDays) {
private Optional<String> findNewestExportMetadataFileBeforeTime(
String bucketName, Instant endTimeUpperBound, int lookBackDays) {
DateTime today = clock.nowUtc();
for (int day = 0; day < lookBackDays; day++) {
String dateString = today.minusDays(day).toString("yyyy-MM-dd");
@@ -107,7 +110,11 @@ public class LatestDatastoreSnapshotFinder {
.sorted(Comparator.<String>naturalOrder().reversed())
.findFirst();
if (metaFilePath.isPresent()) {
return metaFilePath;
BlobInfo blobInfo = gcsUtils.getBlobInfo(BlobId.of(bucketName, metaFilePath.get()));
Instant exportEndTime = new Instant(blobInfo.getCreateTime());
if (exportEndTime.isBefore(endTimeUpperBound)) {
return metaFilePath;
}
}
} catch (IOException ioe) {
throw new RuntimeException(ioe);

View File

@@ -0,0 +1,83 @@
// Copyright 2022 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.comparedb;
import google.registry.beam.common.RegistryPipelineOptions;
import google.registry.beam.common.RegistryPipelineWorkerInitializer;
import google.registry.beam.comparedb.LatestDatastoreSnapshotFinder.DatastoreSnapshotInfo;
import google.registry.model.annotations.DeleteAfterMigration;
import google.registry.persistence.PersistenceModule.JpaTransactionManagerType;
import google.registry.persistence.PersistenceModule.TransactionIsolationLevel;
import java.util.Optional;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.joda.time.DateTime;
/**
* Validates the asynchronous data replication process from Cloud SQL (primary) to Datastore
* (secondary).
*
* <p>This pipeline simply compares the snapshots provided by an invoker, which is responsible for
* obtaining two consistent snapshots for the same point of time.
*/
// TODO(weiminyu): Implement the invoker action in a followup PR.
@DeleteAfterMigration
public class ValidateDatastorePipeline {
private final ValidateDatastorePipelineOptions options;
private final LatestDatastoreSnapshotFinder datastoreSnapshotFinder;
public ValidateDatastorePipeline(
ValidateDatastorePipelineOptions options,
LatestDatastoreSnapshotFinder datastoreSnapshotFinder) {
this.options = options;
this.datastoreSnapshotFinder = datastoreSnapshotFinder;
}
void run(Pipeline pipeline) {
DateTime latestCommitLogTime = DateTime.parse(options.getLatestCommitLogTimestamp());
DatastoreSnapshotInfo mostRecentExport =
datastoreSnapshotFinder.getSnapshotInfo(latestCommitLogTime.toInstant());
ValidateSqlPipeline.setupPipeline(
pipeline,
Optional.ofNullable(options.getSqlSnapshotId()),
mostRecentExport,
latestCommitLogTime,
Optional.ofNullable(options.getComparisonStartTimestamp()).map(DateTime::parse));
pipeline.run();
}
public static void main(String[] args) {
ValidateDatastorePipelineOptions options =
PipelineOptionsFactory.fromArgs(args)
.withValidation()
.as(ValidateDatastorePipelineOptions.class);
RegistryPipelineOptions.validateRegistryPipelineOptions(options);
// Defensively set important options.
options.setIsolationOverride(TransactionIsolationLevel.TRANSACTION_REPEATABLE_READ);
options.setJpaTransactionManagerType(JpaTransactionManagerType.BULK_QUERY);
// Reuse Dataflow worker initialization code to set up JPA in the pipeline harness.
new RegistryPipelineWorkerInitializer().beforeProcessing(options);
LatestDatastoreSnapshotFinder datastoreSnapshotFinder =
DaggerLatestDatastoreSnapshotFinder_LatestDatastoreSnapshotFinderFinderComponent.create()
.datastoreSnapshotInfoFinder();
new ValidateDatastorePipeline(options, datastoreSnapshotFinder).run(Pipeline.create(options));
}
}

View File

@@ -0,0 +1,39 @@
// Copyright 2022 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.comparedb;
import google.registry.model.annotations.DeleteAfterMigration;
import javax.annotation.Nullable;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.Validation;
/** BEAM pipeline options for {@link ValidateDatastorePipelineOptions}. */
@DeleteAfterMigration
public interface ValidateDatastorePipelineOptions extends ValidateSqlPipelineOptions {
@Description(
"The id of the SQL snapshot to be compared with Datastore. "
+ "If null, the current state of the SQL database is used.")
@Nullable
String getSqlSnapshotId();
void setSqlSnapshotId(String snapshotId);
@Description("The latest CommitLogs to load, in ISO8601 format.")
@Validation.Required
String getLatestCommitLogTimestamp();
void setLatestCommitLogTimestamp(String commitLogEndTimestamp);
}

View File

@@ -26,6 +26,9 @@ import google.registry.beam.common.RegistryPipelineWorkerInitializer;
import google.registry.beam.comparedb.LatestDatastoreSnapshotFinder.DatastoreSnapshotInfo;
import google.registry.beam.comparedb.ValidateSqlUtils.CompareSqlEntity;
import google.registry.model.annotations.DeleteAfterMigration;
import google.registry.model.common.DatabaseMigrationStateSchedule;
import google.registry.model.common.DatabaseMigrationStateSchedule.MigrationState;
import google.registry.model.common.DatabaseMigrationStateSchedule.ReplayDirection;
import google.registry.model.domain.DomainBase;
import google.registry.model.domain.DomainHistory;
import google.registry.model.replay.SqlEntity;
@@ -35,6 +38,7 @@ import google.registry.persistence.PersistenceModule.JpaTransactionManagerType;
import google.registry.persistence.PersistenceModule.TransactionIsolationLevel;
import google.registry.persistence.transaction.TransactionManagerFactory;
import google.registry.util.RequestStatusChecker;
import google.registry.util.SystemClock;
import java.io.Serializable;
import java.util.Optional;
import org.apache.beam.sdk.Pipeline;
@@ -76,21 +80,16 @@ public class ValidateSqlPipeline {
java.time.Duration.ofSeconds(30);
private final ValidateSqlPipelineOptions options;
private final DatastoreSnapshotInfo mostRecentExport;
private final LatestDatastoreSnapshotFinder datastoreSnapshotFinder;
public ValidateSqlPipeline(
ValidateSqlPipelineOptions options, DatastoreSnapshotInfo mostRecentExport) {
ValidateSqlPipelineOptions options, LatestDatastoreSnapshotFinder datastoreSnapshotFinder) {
this.options = options;
this.mostRecentExport = mostRecentExport;
}
void run() {
run(Pipeline.create(options));
this.datastoreSnapshotFinder = datastoreSnapshotFinder;
}
@VisibleForTesting
void run(Pipeline pipeline) {
// TODO(weiminyu): ensure migration stage is DATASTORE_PRIMARY or DATASTORE_PRIMARY_READ_ONLY
Optional<Lock> lock = acquireCommitLogReplayLock();
if (lock.isPresent()) {
logger.atInfo().log("Acquired CommitLog Replay lock.");
@@ -101,6 +100,8 @@ public class ValidateSqlPipeline {
try {
DateTime latestCommitLogTime =
TransactionManagerFactory.jpaTm().transact(() -> SqlReplayCheckpoint.get());
DatastoreSnapshotInfo mostRecentExport =
datastoreSnapshotFinder.getSnapshotInfo(latestCommitLogTime.toInstant());
Preconditions.checkState(
latestCommitLogTime.isAfter(mostRecentExport.exportInterval().getEnd()),
"Cannot recreate Datastore snapshot since target time is in the middle of an export.");
@@ -109,7 +110,16 @@ public class ValidateSqlPipeline {
lock.ifPresent(Lock::releaseSql);
lock = Optional.empty();
setupPipeline(pipeline, Optional.of(databaseSnapshot.getSnapshotId()), latestCommitLogTime);
logger.atInfo().log(
"Starting comparison with export at %s and latestCommitLogTime at %s",
mostRecentExport.exportDir(), latestCommitLogTime);
setupPipeline(
pipeline,
Optional.of(databaseSnapshot.getSnapshotId()),
mostRecentExport,
latestCommitLogTime,
Optional.ofNullable(options.getComparisonStartTimestamp()).map(DateTime::parse));
State state = pipeline.run().waitUntilFinish();
if (!State.DONE.equals(state)) {
throw new IllegalStateException("Unexpected pipeline state: " + state);
@@ -120,15 +130,16 @@ public class ValidateSqlPipeline {
}
}
void setupPipeline(
Pipeline pipeline, Optional<String> sqlSnapshotId, DateTime latestCommitLogTime) {
static void setupPipeline(
Pipeline pipeline,
Optional<String> sqlSnapshotId,
DatastoreSnapshotInfo mostRecentExport,
DateTime latestCommitLogTime,
Optional<DateTime> compareStartTime) {
pipeline
.getCoderRegistry()
.registerCoderForClass(SqlEntity.class, SerializableCoder.of(Serializable.class));
Optional<DateTime> compareStartTime =
Optional.ofNullable(options.getComparisonStartTimestamp()).map(DateTime::parse);
PCollectionTuple datastoreSnapshot =
DatastoreSnapshots.loadDatastoreSnapshotByKind(
pipeline,
@@ -216,6 +227,10 @@ public class ValidateSqlPipeline {
return "ValidateSqlPipeline";
}
LatestDatastoreSnapshotFinder datastoreSnapshotFinder =
DaggerLatestDatastoreSnapshotFinder_LatestDatastoreSnapshotFinderFinderComponent.create()
.datastoreSnapshotInfoFinder();
@Override
public boolean isRunning(String requestLogId) {
return true;
@@ -234,11 +249,16 @@ public class ValidateSqlPipeline {
// Reuse Dataflow worker initialization code to set up JPA in the pipeline harness.
new RegistryPipelineWorkerInitializer().beforeProcessing(options);
DatastoreSnapshotInfo mostRecentExport =
DaggerLatestDatastoreSnapshotFinder_LatestDatastoreSnapshotFinderFinderComponent.create()
.datastoreSnapshotInfoFinder()
.getSnapshotInfo();
MigrationState state =
DatabaseMigrationStateSchedule.getValueAtTime(new SystemClock().nowUtc());
if (!state.getReplayDirection().equals(ReplayDirection.DATASTORE_TO_SQL)) {
throw new IllegalStateException("This pipeline is not designed for migration phase " + state);
}
new ValidateSqlPipeline(options, mostRecentExport).run(Pipeline.create(options));
LatestDatastoreSnapshotFinder datastoreSnapshotFinder =
DaggerLatestDatastoreSnapshotFinder_LatestDatastoreSnapshotFinderFinderComponent.create()
.datastoreSnapshotInfoFinder();
new ValidateSqlPipeline(options, datastoreSnapshotFinder).run(Pipeline.create(options));
}
}

View File

@@ -248,6 +248,9 @@ public class RdeIO {
// Now that we're done, output roll the cursor forward.
if (key.manual()) {
logger.atInfo().log("Manual operation; not advancing cursor or enqueuing upload task.");
// Temporary measure to run RDE in beam in parallel with the daily MapReduce based RDE runs.
} else if (tm().isOfy()) {
logger.atInfo().log("Ofy is primary TM; not advancing cursor or enqueuing upload task.");
} else {
outputReceiver.output(KV.of(key, revision));
}

View File

@@ -36,6 +36,19 @@
<target>backend</target>
</cron>
<cron>
<url>/_dr/task/rdeStaging?beam=true</url>
<description>
This job generates a full RDE escrow deposit as a single gigantic XML
document using the Beam pipeline regardless of the current TM
configuration and streams it to cloud storage. It does not trigger the
subsequent upload tasks and is meant to run parallel with the main cron
job in order to compare the results from both runs.
</description>
<schedule>every 8 hours from 00:07 to 20:00</schedule>
<target>backend</target>
</cron>
<cron>
<url><![CDATA[/_dr/cron/fanout?queue=rde-upload&endpoint=/_dr/task/rdeUpload&forEachRealTld]]></url>
<description>

View File

@@ -30,6 +30,7 @@ import google.registry.keyring.api.KeyModule;
import google.registry.keyring.kms.KmsModule;
import google.registry.module.pubapi.PubApiRequestComponent.PubApiRequestComponentModule;
import google.registry.monitoring.whitebox.StackdriverModule;
import google.registry.persistence.PersistenceModule;
import google.registry.privileges.secretmanager.SecretManagerModule;
import google.registry.request.Modules.Jackson2Module;
import google.registry.request.Modules.NetHttpTransportModule;
@@ -56,6 +57,7 @@ import javax.inject.Singleton;
KeyringModule.class,
KmsModule.class,
NetHttpTransportModule.class,
PersistenceModule.class,
PubApiRequestComponentModule.class,
SecretManagerModule.class,
ServerTridProviderModule.class,

View File

@@ -277,6 +277,8 @@ public abstract class PersistenceModule {
setSqlCredential(credentialStore, new RobotUser(RobotId.NOMULUS), overrides);
replicaInstanceConnectionName.ifPresent(
name -> overrides.put(HIKARI_DS_CLOUD_SQL_INSTANCE, name));
overrides.put(
Environment.ISOLATION, TransactionIsolationLevel.TRANSACTION_READ_COMMITTED.name());
return new JpaTransactionManagerImpl(create(overrides), clock);
}
@@ -291,6 +293,8 @@ public abstract class PersistenceModule {
HashMap<String, String> overrides = Maps.newHashMap(beamCloudSqlConfigs);
replicaInstanceConnectionName.ifPresent(
name -> overrides.put(HIKARI_DS_CLOUD_SQL_INSTANCE, name));
overrides.put(
Environment.ISOLATION, TransactionIsolationLevel.TRANSACTION_READ_COMMITTED.name());
return new JpaTransactionManagerImpl(create(overrides), clock);
}

View File

@@ -18,7 +18,6 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static google.registry.model.EppResourceUtils.loadByForeignKey;
import static google.registry.model.index.ForeignKeyIndex.loadAndGetKey;
import static google.registry.model.ofy.ObjectifyService.auditedOfy;
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import static google.registry.request.Action.Method.GET;
import static google.registry.request.Action.Method.HEAD;
@@ -38,8 +37,10 @@ import com.google.common.primitives.Booleans;
import com.googlecode.objectify.cmd.Query;
import google.registry.model.domain.DomainBase;
import google.registry.model.host.HostResource;
import google.registry.persistence.PersistenceModule.ReadOnlyReplicaJpaTm;
import google.registry.persistence.VKey;
import google.registry.persistence.transaction.CriteriaQueryBuilder;
import google.registry.persistence.transaction.JpaTransactionManager;
import google.registry.rdap.RdapJsonFormatter.OutputDataType;
import google.registry.rdap.RdapMetrics.EndpointType;
import google.registry.rdap.RdapMetrics.SearchType;
@@ -91,7 +92,11 @@ public class RdapDomainSearchAction extends RdapSearchActionBase {
@Inject @Parameter("name") Optional<String> nameParam;
@Inject @Parameter("nsLdhName") Optional<String> nsLdhNameParam;
@Inject @Parameter("nsIp") Optional<String> nsIpParam;
@Inject public RdapDomainSearchAction() {
@Inject @ReadOnlyReplicaJpaTm JpaTransactionManager readOnlyJpaTm;
@Inject
public RdapDomainSearchAction() {
super("domain search", EndpointType.DOMAINS);
}
@@ -223,32 +228,31 @@ public class RdapDomainSearchAction extends RdapSearchActionBase {
resultSet = getMatchingResources(query, true, querySizeLimit);
} else {
resultSet =
jpaTm()
.transact(
() -> {
CriteriaBuilder criteriaBuilder =
jpaTm().getEntityManager().getCriteriaBuilder();
CriteriaQueryBuilder<DomainBase> queryBuilder =
CriteriaQueryBuilder.create(DomainBase.class)
.where(
"fullyQualifiedDomainName",
criteriaBuilder::like,
String.format("%s%%", partialStringQuery.getInitialString()))
.orderByAsc("fullyQualifiedDomainName");
if (cursorString.isPresent()) {
queryBuilder =
queryBuilder.where(
"fullyQualifiedDomainName",
criteriaBuilder::greaterThan,
cursorString.get());
}
if (partialStringQuery.getSuffix() != null) {
queryBuilder =
queryBuilder.where(
"tld", criteriaBuilder::equal, partialStringQuery.getSuffix());
}
return getMatchingResourcesSql(queryBuilder, true, querySizeLimit);
});
readOnlyJpaTm.transact(
() -> {
CriteriaBuilder criteriaBuilder =
readOnlyJpaTm.getEntityManager().getCriteriaBuilder();
CriteriaQueryBuilder<DomainBase> queryBuilder =
CriteriaQueryBuilder.create(DomainBase.class)
.where(
"fullyQualifiedDomainName",
criteriaBuilder::like,
String.format("%s%%", partialStringQuery.getInitialString()))
.orderByAsc("fullyQualifiedDomainName");
if (cursorString.isPresent()) {
queryBuilder =
queryBuilder.where(
"fullyQualifiedDomainName",
criteriaBuilder::greaterThan,
cursorString.get());
}
if (partialStringQuery.getSuffix() != null) {
queryBuilder =
queryBuilder.where(
"tld", criteriaBuilder::equal, partialStringQuery.getSuffix());
}
return getMatchingResourcesSql(queryBuilder, true, querySizeLimit);
});
}
return makeSearchResults(resultSet);
}
@@ -270,20 +274,19 @@ public class RdapDomainSearchAction extends RdapSearchActionBase {
resultSet = getMatchingResources(query, true, querySizeLimit);
} else {
resultSet =
jpaTm()
.transact(
() -> {
CriteriaQueryBuilder<DomainBase> builder =
queryItemsSql(
DomainBase.class,
"tld",
tld,
Optional.of("fullyQualifiedDomainName"),
cursorString,
DeletedItemHandling.INCLUDE)
.orderByAsc("fullyQualifiedDomainName");
return getMatchingResourcesSql(builder, true, querySizeLimit);
});
readOnlyJpaTm.transact(
() -> {
CriteriaQueryBuilder<DomainBase> builder =
queryItemsSql(
DomainBase.class,
"tld",
tld,
Optional.of("fullyQualifiedDomainName"),
cursorString,
DeletedItemHandling.INCLUDE)
.orderByAsc("fullyQualifiedDomainName");
return getMatchingResourcesSql(builder, true, querySizeLimit);
});
}
return makeSearchResults(resultSet);
}
@@ -354,28 +357,28 @@ public class RdapDomainSearchAction extends RdapSearchActionBase {
.map(VKey::from)
.collect(toImmutableSet());
} else {
return jpaTm()
.transact(
() -> {
CriteriaQueryBuilder<HostResource> builder =
queryItemsSql(
HostResource.class,
"fullyQualifiedHostName",
partialStringQuery,
Optional.empty(),
DeletedItemHandling.EXCLUDE);
if (desiredRegistrar.isPresent()) {
builder =
builder.where(
"currentSponsorClientId",
jpaTm().getEntityManager().getCriteriaBuilder()::equal,
desiredRegistrar.get());
}
return getMatchingResourcesSql(builder, true, maxNameserversInFirstStage)
.resources().stream()
.map(HostResource::createVKey)
.collect(toImmutableSet());
});
return readOnlyJpaTm.transact(
() -> {
CriteriaQueryBuilder<HostResource> builder =
queryItemsSql(
HostResource.class,
"fullyQualifiedHostName",
partialStringQuery,
Optional.empty(),
DeletedItemHandling.EXCLUDE);
if (desiredRegistrar.isPresent()) {
builder =
builder.where(
"currentSponsorClientId",
readOnlyJpaTm.getEntityManager().getCriteriaBuilder()::equal,
desiredRegistrar.get());
}
return getMatchingResourcesSql(builder, true, maxNameserversInFirstStage)
.resources()
.stream()
.map(HostResource::createVKey)
.collect(toImmutableSet());
});
}
}
@@ -509,21 +512,20 @@ public class RdapDomainSearchAction extends RdapSearchActionBase {
parameters.put("desiredRegistrar", desiredRegistrar.get());
}
hostKeys =
jpaTm()
.transact(
() -> {
javax.persistence.Query query =
jpaTm()
.getEntityManager()
.createNativeQuery(queryBuilder.toString())
.setMaxResults(maxNameserversInFirstStage);
parameters.build().forEach(query::setParameter);
@SuppressWarnings("unchecked")
Stream<String> resultStream = query.getResultStream();
return resultStream
.map(repoId -> VKey.create(HostResource.class, repoId))
.collect(toImmutableSet());
});
readOnlyJpaTm.transact(
() -> {
javax.persistence.Query query =
readOnlyJpaTm
.getEntityManager()
.createNativeQuery(queryBuilder.toString())
.setMaxResults(maxNameserversInFirstStage);
parameters.build().forEach(query::setParameter);
@SuppressWarnings("unchecked")
Stream<String> resultStream = query.getResultStream();
return resultStream
.map(repoId -> VKey.create(HostResource.class, repoId))
.collect(toImmutableSet());
});
}
return searchByNameserverRefs(hostKeys);
}
@@ -568,39 +570,38 @@ public class RdapDomainSearchAction extends RdapSearchActionBase {
}
stream.forEach(domainSetBuilder::add);
} else {
jpaTm()
.transact(
() -> {
for (VKey<HostResource> hostKey : hostKeys) {
CriteriaQueryBuilder<DomainBase> queryBuilder =
CriteriaQueryBuilder.create(DomainBase.class)
.whereFieldContains("nsHosts", hostKey)
.orderByAsc("fullyQualifiedDomainName");
CriteriaBuilder criteriaBuilder =
jpaTm().getEntityManager().getCriteriaBuilder();
if (!shouldIncludeDeleted()) {
queryBuilder =
queryBuilder.where(
"deletionTime", criteriaBuilder::greaterThan, getRequestTime());
}
if (cursorString.isPresent()) {
queryBuilder =
queryBuilder.where(
"fullyQualifiedDomainName",
criteriaBuilder::greaterThan,
cursorString.get());
}
jpaTm()
.criteriaQuery(queryBuilder.build())
.getResultStream()
.filter(this::isAuthorized)
.forEach(
(domain) -> {
Hibernate.initialize(domain.getDsData());
domainSetBuilder.add(domain);
});
}
});
readOnlyJpaTm.transact(
() -> {
for (VKey<HostResource> hostKey : hostKeys) {
CriteriaQueryBuilder<DomainBase> queryBuilder =
CriteriaQueryBuilder.create(DomainBase.class)
.whereFieldContains("nsHosts", hostKey)
.orderByAsc("fullyQualifiedDomainName");
CriteriaBuilder criteriaBuilder =
readOnlyJpaTm.getEntityManager().getCriteriaBuilder();
if (!shouldIncludeDeleted()) {
queryBuilder =
queryBuilder.where(
"deletionTime", criteriaBuilder::greaterThan, getRequestTime());
}
if (cursorString.isPresent()) {
queryBuilder =
queryBuilder.where(
"fullyQualifiedDomainName",
criteriaBuilder::greaterThan,
cursorString.get());
}
readOnlyJpaTm
.criteriaQuery(queryBuilder.build())
.getResultStream()
.filter(this::isAuthorized)
.forEach(
(domain) -> {
Hibernate.initialize(domain.getDsData());
domainSetBuilder.add(domain);
});
}
});
}
}
List<DomainBase> domains = domainSetBuilder.build().asList();

View File

@@ -32,6 +32,7 @@ import com.google.common.net.MediaType;
import google.registry.config.RegistryConfig.Config;
import google.registry.config.RegistryEnvironment;
import google.registry.model.common.DatabaseMigrationStateSchedule.PrimaryDatabase;
import google.registry.persistence.PersistenceModule;
import google.registry.reporting.ReportingModule;
import google.registry.request.Action;
import google.registry.request.Parameter;
@@ -120,17 +121,16 @@ public class GenerateInvoicesAction implements Runnable {
.setContainerSpecGcsPath(
String.format("%s/%s_metadata.json", stagingBucketUrl, PIPELINE_NAME))
.setParameters(
ImmutableMap.of(
"yearMonth",
yearMonth.toString("yyyy-MM"),
"invoiceFilePrefix",
invoiceFilePrefix,
"database",
database.name(),
"billingBucketUrl",
billingBucketUrl,
"registryEnvironment",
RegistryEnvironment.get().name()));
new ImmutableMap.Builder<String, String>()
.put("yearMonth", yearMonth.toString("yyyy-MM"))
.put("invoiceFilePrefix", invoiceFilePrefix)
.put("database", database.name())
.put("billingBucketUrl", billingBucketUrl)
.put("registryEnvironment", RegistryEnvironment.get().name())
.put(
"jpaTransactionManagerType",
PersistenceModule.JpaTransactionManagerType.READ_ONLY_REPLICA.toString())
.build());
LaunchFlexTemplateResponse launchResponse =
dataflow
.projects()

View File

@@ -15,14 +15,8 @@
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.CompareEscrowDepositsCommand;
import google.registry.tools.javascrap.DeleteContactByRoidCommand;
import google.registry.tools.javascrap.HardDeleteHostCommand;
import google.registry.tools.javascrap.PopulateNullRegistrarFieldsCommand;
import google.registry.tools.javascrap.RemoveIpAddressCommand;
import google.registry.tools.javascrap.ResaveAllTldsCommand;
/** Container class to create and run remote commands against a Datastore instance. */
public final class RegistryTool {
@@ -36,8 +30,6 @@ public final class RegistryTool {
public static final ImmutableMap<String, Class<? extends Command>> COMMAND_MAP =
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)
@@ -57,7 +49,6 @@ public final class RegistryTool {
.put("curl", CurlCommand.class)
.put("dedupe_one_time_billing_event_ids", DedupeOneTimeBillingEventIdsCommand.class)
.put("delete_allocation_tokens", DeleteAllocationTokensCommand.class)
.put("delete_contact_by_roid", DeleteContactByRoidCommand.class)
.put("delete_domain", DeleteDomainCommand.class)
.put("delete_host", DeleteHostCommand.class)
.put("delete_premium_list", DeletePremiumListCommand.class)
@@ -107,12 +98,9 @@ public final class RegistryTool {
.put("login", LoginCommand.class)
.put("logout", LogoutCommand.class)
.put("pending_escrow", PendingEscrowCommand.class)
.put("populate_null_registrar_fields", PopulateNullRegistrarFieldsCommand.class)
.put("registrar_contact", RegistrarContactCommand.class)
.put("remove_ip_address", RemoveIpAddressCommand.class)
.put("remove_registry_one_key", RemoveRegistryOneKeyCommand.class)
.put("renew_domain", RenewDomainCommand.class)
.put("resave_all_tlds", ResaveAllTldsCommand.class)
.put("resave_entities", ResaveEntitiesCommand.class)
.put("resave_environment_entities", ResaveEnvironmentEntitiesCommand.class)
.put("resave_epp_resource", ResaveEppResourceCommand.class)

View File

@@ -42,9 +42,7 @@ import google.registry.request.Modules.URLFetchServiceModule;
import google.registry.request.Modules.UrlFetchTransportModule;
import google.registry.request.Modules.UserServiceModule;
import google.registry.tools.AuthModule.LocalCredentialModule;
import google.registry.tools.javascrap.BackfillRegistryLocksCommand;
import google.registry.tools.javascrap.CompareEscrowDepositsCommand;
import google.registry.tools.javascrap.DeleteContactByRoidCommand;
import google.registry.tools.javascrap.HardDeleteHostCommand;
import google.registry.util.UtilsModule;
import google.registry.whois.NonCachingWhoisModule;
@@ -90,8 +88,6 @@ import javax.inject.Singleton;
interface RegistryToolComponent {
void inject(AckPollMessagesCommand command);
void inject(BackfillRegistryLocksCommand command);
void inject(CheckDomainClaimsCommand command);
void inject(CheckDomainCommand command);
@@ -112,8 +108,6 @@ interface RegistryToolComponent {
void inject(CreateTldCommand command);
void inject(DeleteContactByRoidCommand command);
void inject(EncryptEscrowDepositCommand command);
void inject(EnqueuePollMessageCommand command);

View File

@@ -1,157 +0,0 @@
// 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.checkArgument;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import static google.registry.persistence.transaction.TransactionManagerUtil.transactIfJpaTm;
import static google.registry.tools.LockOrUnlockDomainCommand.REGISTRY_LOCK_STATUSES;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.google.common.collect.ImmutableCollection;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.flogger.FluentLogger;
import google.registry.config.RegistryConfig.Config;
import google.registry.model.domain.DomainBase;
import google.registry.model.domain.RegistryLock;
import google.registry.model.reporting.HistoryEntry;
import google.registry.model.reporting.HistoryEntryDao;
import google.registry.model.tld.RegistryLockDao;
import google.registry.persistence.VKey;
import google.registry.tools.CommandWithRemoteApi;
import google.registry.tools.ConfirmingCommand;
import google.registry.util.Clock;
import google.registry.util.StringGenerator;
import java.util.Comparator;
import java.util.List;
import javax.inject.Inject;
import javax.inject.Named;
import org.joda.time.DateTime;
/**
* Scrap tool to backfill {@link RegistryLock}s for domains previously locked.
*
* <p>This will save new objects for all existing domains that are locked but don't have any
* corresponding lock objects already in the database.
*/
@Parameters(
separators = " =",
commandDescription =
"Backfills RegistryLock objects for specified domain resource IDs that are locked but don't"
+ " already have a corresponding RegistryLock object.")
public class BackfillRegistryLocksCommand extends ConfirmingCommand
implements CommandWithRemoteApi {
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
private static final int VERIFICATION_CODE_LENGTH = 32;
@Parameter(
names = {"--domain_roids"},
description = "Comma-separated list of domain roids to check")
protected List<String> roids;
// Inject here so that we can create the command automatically for tests
@Inject Clock clock;
@Inject
@Config("registryAdminClientId")
String registryAdminClientId;
@Inject
@Named("base58StringGenerator")
StringGenerator stringGenerator;
private ImmutableList<DomainBase> lockedDomains;
@Override
protected String prompt() {
checkArgument(
roids != null && !roids.isEmpty(), "Must provide non-empty domain_roids argument");
lockedDomains =
jpaTm().transact(() -> getLockedDomainsWithoutLocks(jpaTm().getTransactionTime()));
ImmutableList<String> lockedDomainNames =
lockedDomains.stream().map(DomainBase::getDomainName).collect(toImmutableList());
return String.format(
"Locked domains for which there does not exist a RegistryLock object: %s",
lockedDomainNames);
}
@Override
protected String execute() {
ImmutableSet.Builder<String> failedDomainsBuilder = new ImmutableSet.Builder<>();
jpaTm()
.transact(
() -> {
for (DomainBase domainBase : lockedDomains) {
try {
RegistryLockDao.save(
new RegistryLock.Builder()
.isSuperuser(true)
.setRegistrarId(registryAdminClientId)
.setRepoId(domainBase.getRepoId())
.setDomainName(domainBase.getDomainName())
.setLockCompletionTime(
getLockCompletionTimestamp(domainBase, jpaTm().getTransactionTime()))
.setVerificationCode(
stringGenerator.createString(VERIFICATION_CODE_LENGTH))
.build());
} catch (Throwable t) {
logger.atSevere().withCause(t).log(
"Error when creating lock object for domain '%s'.",
domainBase.getDomainName());
failedDomainsBuilder.add(domainBase.getDomainName());
}
}
});
ImmutableSet<String> failedDomains = failedDomainsBuilder.build();
if (failedDomains.isEmpty()) {
return String.format(
"Successfully created lock objects for %d domains.", lockedDomains.size());
} else {
return String.format(
"Successfully created lock objects for %d domains. We failed to create locks "
+ "for the following domains: %s",
lockedDomains.size() - failedDomains.size(), failedDomains);
}
}
private DateTime getLockCompletionTimestamp(DomainBase domainBase, DateTime now) {
// Best-effort, if a domain was URS-locked we should use that time
// If we can't find that, return now.
return HistoryEntryDao.loadHistoryObjectsForResource(domainBase.createVKey()).stream()
// sort by modification time descending so we get the most recent one if it was locked twice
.sorted(Comparator.comparing(HistoryEntry::getModificationTime).reversed())
.filter(entry -> "Uniform Rapid Suspension".equals(entry.getReason()))
.findFirst()
.map(HistoryEntry::getModificationTime)
.orElse(now);
}
private ImmutableList<DomainBase> getLockedDomainsWithoutLocks(DateTime now) {
ImmutableList<VKey<DomainBase>> domainKeys =
roids.stream().map(roid -> VKey.create(DomainBase.class, roid)).collect(toImmutableList());
ImmutableCollection<DomainBase> domains =
transactIfJpaTm(() -> tm().loadByKeys(domainKeys)).values();
return domains.stream()
.filter(d -> d.getDeletionTime().isAfter(now))
.filter(d -> d.getStatusValues().containsAll(REGISTRY_LOCK_STATUSES))
.filter(d -> !RegistryLockDao.getMostRecentByRepoId(d.getRepoId()).isPresent())
.collect(toImmutableList());
}
}

View File

@@ -1,223 +0,0 @@
// 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.persistence.transaction.TransactionManagerFactory.jpaTm;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import static google.registry.persistence.transaction.TransactionManagerUtil.transactIfJpaTm;
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.persistence.transaction.QueryComposer;
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.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) -> {
ImmutableList<DomainBase> domains = loadDomainsForFqdn(domainName);
checkState(
!domains.isEmpty(),
"Domain name %s had no associated DomainBase objects.",
domainName);
return domains.stream()
.sorted(Comparator.comparing(DomainBase::getCreationTime).reversed());
}));
}
/** Loads in all {@link DomainBase} objects for a given FQDN. */
private ImmutableList<DomainBase> loadDomainsForFqdn(String fullyQualifiedDomainName) {
return transactIfJpaTm(
() ->
tm().createQueryComposer(DomainBase.class)
.where(
"fullyQualifiedDomainName",
QueryComposer.Comparator.EQ,
fullyQualifiedDomainName)
.list());
}
/** 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()
.query(
"SELECT DISTINCT stm.checkDate FROM Spec11ThreatMatch stm", LocalDate.class)
.getResultStream()
.collect(toImmutableSet()));
}
}

View File

@@ -1,115 +0,0 @@
// Copyright 2021 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.Verify.verify;
import static google.registry.model.ofy.ObjectifyService.auditedOfy;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.googlecode.objectify.Key;
import google.registry.model.contact.ContactResource;
import google.registry.model.eppcommon.StatusValue;
import google.registry.model.index.EppResourceIndex;
import google.registry.model.index.ForeignKeyIndex;
import google.registry.tools.CommandWithRemoteApi;
import google.registry.tools.ConfirmingCommand;
import google.registry.util.SystemClock;
import java.util.List;
import java.util.Objects;
/**
* Deletes a {@link google.registry.model.contact.ContactResource} by its ROID.
*
* <p>This is a short-term tool for race condition clean up while the bug is being fixed.
*/
@Parameters(separators = " =", commandDescription = "Delete a contact by its ROID.")
public class DeleteContactByRoidCommand extends ConfirmingCommand implements CommandWithRemoteApi {
@Parameter(names = "--roid", description = "The roid of the contact to be deleted.")
String roid;
@Parameter(
names = "--contact_id",
description = "The user provided contactId, for verification purpose.")
String contactId;
ImmutableList<Key<?>> toDelete;
@Override
protected void init() {
System.out.printf("Deleting %s, which refers to %s.\n", roid, contactId);
tm().transact(
() -> {
Key<ContactResource> targetKey = Key.create(ContactResource.class, roid);
ContactResource targetContact = auditedOfy().load().key(targetKey).now();
verify(
Objects.equals(targetContact.getContactId(), contactId),
"contactId does not match.");
verify(
Objects.equals(targetContact.getStatusValues(), ImmutableSet.of(StatusValue.OK)));
System.out.println("Target contact has the expected contactId");
String canonicalResource =
ForeignKeyIndex.load(ContactResource.class, contactId, new SystemClock().nowUtc())
.getResourceKey()
.getOfyKey()
.getName();
verify(!Objects.equals(canonicalResource, roid), "Contact still in ForeignKeyIndex.");
System.out.printf(
"It is safe to delete %s, since the contactId is mapped to a different entry in"
+ " the Foreign key index (%s).\n\n",
roid, canonicalResource);
List<Object> ancestors =
auditedOfy().load().ancestor(Key.create(ContactResource.class, roid)).list();
System.out.println("Ancestor query returns: ");
for (Object entity : ancestors) {
System.out.println(Key.create(entity));
}
ImmutableSet<String> deletetableKinds =
ImmutableSet.of("HistoryEntry", "ContactResource");
toDelete =
ancestors.stream()
.map(Key::create)
.filter(key -> deletetableKinds.contains(key.getKind()))
.collect(ImmutableList.toImmutableList());
EppResourceIndex eppResourceIndex =
auditedOfy().load().entity(EppResourceIndex.create(targetKey)).now();
verify(eppResourceIndex.getKey().equals(targetKey), "Wrong EppResource Index loaded");
System.out.printf("\n\nEppResourceIndex found (%s).\n", Key.create(eppResourceIndex));
toDelete =
new ImmutableList.Builder<Key<?>>()
.addAll(toDelete)
.add(Key.create(eppResourceIndex))
.build();
System.out.printf("\n\nAbout to delete %s entities:\n", toDelete.size());
toDelete.forEach(System.out::println);
});
}
@Override
protected String execute() {
tm().transact(() -> auditedOfy().delete().keys(toDelete).now());
return "Done";
}
}

View File

@@ -1,70 +0,0 @@
// Copyright 2017 The Nomulus Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package google.registry.tools.javascrap;
import static com.google.common.base.MoreObjects.firstNonNull;
import com.beust.jcommander.Parameters;
import com.google.common.collect.ImmutableList;
import google.registry.model.registrar.Registrar;
import google.registry.model.registrar.RegistrarAddress;
import google.registry.tools.MutatingCommand;
import java.util.Objects;
/**
* Scrap tool to update Registrars with null registrarName or localizedAddress fields.
*
* <p>This sets a null registrarName to the key name, and null localizedAddress fields to fake data.
*/
@Parameters(
separators = " =",
commandDescription = "Populate previously null required registrar fields."
)
public class PopulateNullRegistrarFieldsCommand extends MutatingCommand {
@Override
protected void init() {
for (Registrar registrar : Registrar.loadAll()) {
Registrar.Builder changeBuilder = registrar.asBuilder();
changeBuilder.setRegistrarName(
firstNonNull(registrar.getRegistrarName(), registrar.getRegistrarId()));
RegistrarAddress address = registrar.getLocalizedAddress();
if (address == null) {
changeBuilder.setLocalizedAddress(
new RegistrarAddress.Builder()
.setCity("Fakington")
.setCountryCode("US")
.setState("FL")
.setZip("12345")
.setStreet(ImmutableList.of("123 Fake Street"))
.build());
} else {
changeBuilder.setLocalizedAddress(
new RegistrarAddress.Builder()
.setCity(firstNonNull(address.getCity(), "Fakington"))
.setCountryCode(firstNonNull(address.getCountryCode(), "US"))
.setState(firstNonNull(address.getState(), "FL"))
.setZip(firstNonNull(address.getZip(), "12345"))
.setStreet(firstNonNull(address.getStreet(), ImmutableList.of("123 Fake Street")))
.build());
}
Registrar changedRegistrar = changeBuilder.build();
if (!Objects.equals(registrar, changedRegistrar)) {
stageEntityChange(registrar, changedRegistrar);
}
}
}
}

View File

@@ -1,88 +0,0 @@
// Copyright 2017 The Nomulus Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package google.registry.tools.javascrap;
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
import static google.registry.persistence.transaction.TransactionManagerUtil.transactIfJpaTm;
import static java.nio.charset.StandardCharsets.UTF_8;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.google.template.soy.data.SoyMapData;
import google.registry.model.host.HostResource;
import google.registry.persistence.VKey;
import google.registry.tools.MutatingEppToolCommand;
import google.registry.tools.params.PathParameter;
import google.registry.tools.soy.RemoveIpAddressSoyInfo;
import java.net.Inet6Address;
import java.net.InetAddress;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
/**
* Command to remove external IP Addresses from HostResources identified by text file listing
* resource ids, one per line.
*
* <p>Written for b/23757755 so we can clean up records with IP addresses that should always be
* resolved by hostname.
*
* <p>The JSON file should contain a list of objects each of which has a "roid" attribute.
*/
@Parameters(separators = " =", commandDescription = "Remove all IP Addresses.")
public class RemoveIpAddressCommand extends MutatingEppToolCommand {
public static String registrarId = "CharlestonRoad";
@Parameter(names = "--roids_file",
description = "Text file containing a list of HostResource roids to remove",
required = true,
validateWith = PathParameter.InputFile.class)
private Path roidsFilePath;
@Override
protected void initMutatingEppToolCommand() throws Exception {
List<String> roids = Files.readAllLines(roidsFilePath, UTF_8);
for (String roid : roids) {
// Look up the HostResource from its roid.
Optional<HostResource> host =
transactIfJpaTm(() -> tm().loadByKeyIfPresent(VKey.create(HostResource.class, roid)));
if (!host.isPresent()) {
System.err.printf("Record for %s not found.\n", roid);
continue;
}
ArrayList<SoyMapData> ipAddresses = new ArrayList<>();
for (InetAddress address : host.get().getInetAddresses()) {
SoyMapData dataMap = new SoyMapData(
"address", address.getHostAddress(),
"version", address instanceof Inet6Address ? "v6" : "v4");
ipAddresses.add(dataMap);
}
// Build and execute the EPP command.
setSoyTemplate(
RemoveIpAddressSoyInfo.getInstance(), RemoveIpAddressSoyInfo.REMOVE_IP_ADDRESS);
addSoyRecord(
registrarId,
new SoyMapData(
"name", host.get().getHostName(),
"ipAddresses", ipAddresses,
"requestedByRegistrar", registrarId));
}
}
}

View File

@@ -1,30 +0,0 @@
// Copyright 2021 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 google.registry.persistence.transaction.TransactionManagerFactory.tm;
import com.beust.jcommander.Parameters;
import google.registry.model.tld.Registry;
import google.registry.tools.CommandWithRemoteApi;
/** Scrap command to resave all Registry entities. */
@Parameters(commandDescription = "Resave all TLDs")
public class ResaveAllTldsCommand implements CommandWithRemoteApi {
@Override
public void run() throws Exception {
tm().transact(() -> tm().putAll(tm().loadAllOf(Registry.class)));
}
}

View File

@@ -69,6 +69,12 @@
"regexes": [
"^DATASTORE|CLOUD_SQL$"
]
},
{
"name": "jpaTransactionManagerType",
"label": "The type of JPA transaction manager to use if using SQL",
"helpText": "The standard SQL instance or a read-only replica may be used",
"regexes": ["^REGULAR|READ_ONLY_REPLICA$"]
}
]
}

View File

@@ -0,0 +1,351 @@
// Copyright 2022 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.persistence.transaction;
import com.google.common.collect.ImmutableCollection;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import google.registry.model.ImmutableObject;
import google.registry.persistence.VKey;
import java.util.Optional;
import java.util.function.Supplier;
import java.util.stream.Stream;
import javax.persistence.EntityManager;
import javax.persistence.Query;
import javax.persistence.TypedQuery;
import javax.persistence.criteria.CriteriaQuery;
import org.joda.time.DateTime;
/**
* A {@link JpaTransactionManager} that simulates a read-only replica SQL instance.
*
* <p>We accomplish this by delegating all calls to the standard transaction manager except for
* calls that start transactions. For these, we create a transaction like normal but set it to READ
* ONLY mode before doing any work. This is similar to how the read-only Postgres replica works; it
* treats all transactions as read-only transactions.
*/
public class ReplicaSimulatingJpaTransactionManager implements JpaTransactionManager {
private final JpaTransactionManager delegate;
public ReplicaSimulatingJpaTransactionManager(JpaTransactionManager delegate) {
this.delegate = delegate;
}
@Override
public void teardown() {
delegate.teardown();
}
@Override
public EntityManager getStandaloneEntityManager() {
return delegate.getStandaloneEntityManager();
}
@Override
public EntityManager getEntityManager() {
return delegate.getEntityManager();
}
@Override
public JpaTransactionManager setDatabaseSnapshot(String snapshotId) {
return delegate.setDatabaseSnapshot(snapshotId);
}
@Override
public <T> TypedQuery<T> query(String sqlString, Class<T> resultClass) {
return delegate.query(sqlString, resultClass);
}
@Override
public <T> TypedQuery<T> criteriaQuery(CriteriaQuery<T> criteriaQuery) {
return delegate.criteriaQuery(criteriaQuery);
}
@Override
public Query query(String sqlString) {
return delegate.query(sqlString);
}
@Override
public boolean inTransaction() {
return delegate.inTransaction();
}
@Override
public void assertInTransaction() {
delegate.assertInTransaction();
}
@Override
public <T> T transact(Supplier<T> work) {
return delegate.transact(
() -> {
delegate.getEntityManager().createQuery("SET TRANSACTION READ ONLY").executeUpdate();
return work.get();
});
}
@Override
public <T> T transactWithoutBackup(Supplier<T> work) {
return transact(work);
}
@Override
public <T> T transactNoRetry(Supplier<T> work) {
return transact(work);
}
@Override
public void transact(Runnable work) {
transact(
() -> {
work.run();
return null;
});
}
@Override
public void transactNoRetry(Runnable work) {
transact(work);
}
@Override
public <T> T transactNew(Supplier<T> work) {
return transact(work);
}
@Override
public void transactNew(Runnable work) {
transact(work);
}
@Override
public <T> T transactNewReadOnly(Supplier<T> work) {
return transact(work);
}
@Override
public void transactNewReadOnly(Runnable work) {
transact(work);
}
@Override
public <T> T doTransactionless(Supplier<T> work) {
return delegate.doTransactionless(work);
}
@Override
public DateTime getTransactionTime() {
return delegate.getTransactionTime();
}
@Override
public void insert(Object entity) {
delegate.insert(entity);
}
@Override
public void insertAll(ImmutableCollection<?> entities) {
delegate.insertAll(entities);
}
@Override
public void insertAll(ImmutableObject... entities) {
delegate.insertAll(entities);
}
@Override
public void insertWithoutBackup(ImmutableObject entity) {
delegate.insertWithoutBackup(entity);
}
@Override
public void insertAllWithoutBackup(ImmutableCollection<?> entities) {
delegate.insertAllWithoutBackup(entities);
}
@Override
public void put(Object entity) {
delegate.put(entity);
}
@Override
public void putAll(ImmutableObject... entities) {
delegate.putAll(entities);
}
@Override
public void putAll(ImmutableCollection<?> entities) {
delegate.putAll(entities);
}
@Override
public void putWithoutBackup(ImmutableObject entity) {
delegate.putWithoutBackup(entity);
}
@Override
public void putAllWithoutBackup(ImmutableCollection<?> entities) {
delegate.putAllWithoutBackup(entities);
}
@Override
public void update(Object entity) {
delegate.update(entity);
}
@Override
public void updateAll(ImmutableCollection<?> entities) {
delegate.updateAll(entities);
}
@Override
public void updateAll(ImmutableObject... entities) {
delegate.updateAll(entities);
}
@Override
public void updateWithoutBackup(ImmutableObject entity) {
delegate.updateWithoutBackup(entity);
}
@Override
public void updateAllWithoutBackup(ImmutableCollection<?> entities) {
delegate.updateAllWithoutBackup(entities);
}
@Override
public <T> boolean exists(VKey<T> key) {
return delegate.exists(key);
}
@Override
public boolean exists(Object entity) {
return delegate.exists(entity);
}
@Override
public <T> Optional<T> loadByKeyIfPresent(VKey<T> key) {
return delegate.loadByKeyIfPresent(key);
}
@Override
public <T> ImmutableMap<VKey<? extends T>, T> loadByKeysIfPresent(
Iterable<? extends VKey<? extends T>> vKeys) {
return delegate.loadByKeysIfPresent(vKeys);
}
@Override
public <T> ImmutableList<T> loadByEntitiesIfPresent(Iterable<T> entities) {
return delegate.loadByEntitiesIfPresent(entities);
}
@Override
public <T> T loadByKey(VKey<T> key) {
return delegate.loadByKey(key);
}
@Override
public <T> ImmutableMap<VKey<? extends T>, T> loadByKeys(
Iterable<? extends VKey<? extends T>> vKeys) {
return delegate.loadByKeys(vKeys);
}
@Override
public <T> T loadByEntity(T entity) {
return delegate.loadByEntity(entity);
}
@Override
public <T> ImmutableList<T> loadByEntities(Iterable<T> entities) {
return delegate.loadByEntities(entities);
}
@Override
public <T> ImmutableList<T> loadAllOf(Class<T> clazz) {
return delegate.loadAllOf(clazz);
}
@Override
public <T> Stream<T> loadAllOfStream(Class<T> clazz) {
return delegate.loadAllOfStream(clazz);
}
@Override
public <T> Optional<T> loadSingleton(Class<T> clazz) {
return delegate.loadSingleton(clazz);
}
@Override
public void delete(VKey<?> key) {
delegate.delete(key);
}
@Override
public void delete(Iterable<? extends VKey<?>> vKeys) {
delegate.delete(vKeys);
}
@Override
public <T> T delete(T entity) {
return delegate.delete(entity);
}
@Override
public void deleteWithoutBackup(VKey<?> key) {
delegate.deleteWithoutBackup(key);
}
@Override
public void deleteWithoutBackup(Iterable<? extends VKey<?>> keys) {
delegate.deleteWithoutBackup(keys);
}
@Override
public void deleteWithoutBackup(Object entity) {
delegate.deleteWithoutBackup(entity);
}
@Override
public <T> QueryComposer<T> createQueryComposer(Class<T> entity) {
return delegate.createQueryComposer(entity);
}
@Override
public void clearSessionCache() {
delegate.clearSessionCache();
}
@Override
public boolean isOfy() {
return delegate.isOfy();
}
@Override
public void putIgnoringReadOnlyWithoutBackup(Object entity) {
delegate.putIgnoringReadOnlyWithoutBackup(entity);
}
@Override
public void deleteIgnoringReadOnlyWithoutBackup(VKey<?> key) {
delegate.deleteIgnoringReadOnlyWithoutBackup(key);
}
@Override
public <T> void assertDelete(VKey<T> key) {
delegate.assertDelete(key);
}
}

View File

@@ -15,6 +15,7 @@
package google.registry.rdap;
import static com.google.common.truth.Truth.assertThat;
import static google.registry.persistence.transaction.TransactionManagerFactory.jpaTm;
import static google.registry.rdap.RdapTestHelper.assertThat;
import static google.registry.rdap.RdapTestHelper.parseJsonObject;
import static google.registry.request.Action.Method.POST;
@@ -45,6 +46,7 @@ import google.registry.model.registrar.Registrar;
import google.registry.model.reporting.HistoryEntry;
import google.registry.model.tld.Registry;
import google.registry.persistence.VKey;
import google.registry.persistence.transaction.ReplicaSimulatingJpaTransactionManager;
import google.registry.rdap.RdapMetrics.EndpointType;
import google.registry.rdap.RdapMetrics.SearchType;
import google.registry.rdap.RdapMetrics.WildcardType;
@@ -93,39 +95,27 @@ class RdapDomainSearchActionTest extends RdapSearchActionTestCase<RdapDomainSear
}
private JsonObject generateActualJson(RequestType requestType, String paramValue, String cursor) {
action.requestPath = actionPath;
action.requestMethod = POST;
String requestTypeParam = null;
String requestTypeParam;
switch (requestType) {
case NAME:
action.nameParam = Optional.of(paramValue);
action.nsLdhNameParam = Optional.empty();
action.nsIpParam = Optional.empty();
requestTypeParam = "name";
break;
case NS_LDH_NAME:
action.nameParam = Optional.empty();
action.nsLdhNameParam = Optional.of(paramValue);
action.nsIpParam = Optional.empty();
requestTypeParam = "nsLdhName";
break;
case NS_IP:
action.nameParam = Optional.empty();
action.nsLdhNameParam = Optional.empty();
action.nsIpParam = Optional.of(paramValue);
requestTypeParam = "nsIp";
break;
default:
action.nameParam = Optional.empty();
action.nsLdhNameParam = Optional.empty();
action.nsIpParam = Optional.empty();
requestTypeParam = "";
break;
}
if (paramValue != null) {
if (cursor == null) {
action.parameterMap = ImmutableListMultimap.of(requestTypeParam, paramValue);
action.cursorTokenParam = Optional.empty();
} else {
action.parameterMap =
ImmutableListMultimap.of(requestTypeParam, paramValue, "cursor", cursor);
@@ -381,6 +371,12 @@ class RdapDomainSearchActionTest extends RdapSearchActionTestCase<RdapDomainSear
clock.nowUtc()));
action.requestMethod = POST;
action.nameParam = Optional.empty();
action.nsLdhNameParam = Optional.empty();
action.nsIpParam = Optional.empty();
action.cursorTokenParam = Optional.empty();
action.requestPath = actionPath;
action.readOnlyJpaTm = jpaTm();
}
private JsonObject generateExpectedJsonForTwoDomainsNsReply() {
@@ -728,6 +724,18 @@ class RdapDomainSearchActionTest extends RdapSearchActionTestCase<RdapDomainSear
verifyMetrics(SearchType.BY_DOMAIN_NAME, Optional.of(1L));
}
@TestSqlOnly
void testDomainMatch_readOnlyReplica() {
login("evilregistrar");
rememberWildcardType("cat.lol");
action.readOnlyJpaTm = new ReplicaSimulatingJpaTransactionManager(jpaTm());
action.nameParam = Optional.of("cat.lol");
action.parameterMap = ImmutableListMultimap.of("name", "cat.lol");
action.run();
assertThat(response.getPayload()).contains("Yes Virginia <script>");
assertThat(response.getStatus()).isEqualTo(200);
}
@TestOfyAndSql
void testDomainMatch_foundWithUpperCase() {
login("evilregistrar");

View File

@@ -1,173 +0,0 @@
// 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.truth.Truth.assertThat;
import static com.google.common.truth.Truth8.assertThat;
import static google.registry.testing.DatabaseHelper.createTld;
import static google.registry.testing.DatabaseHelper.persistActiveDomain;
import static google.registry.testing.DatabaseHelper.persistDeletedDomain;
import static google.registry.testing.DatabaseHelper.persistNewRegistrar;
import static google.registry.testing.DatabaseHelper.persistResource;
import static google.registry.testing.SqlHelper.getMostRecentRegistryLockByRepoId;
import static google.registry.testing.SqlHelper.getMostRecentVerifiedRegistryLockByRepoId;
import static google.registry.testing.SqlHelper.getRegistryLocksByRegistrarId;
import static google.registry.testing.SqlHelper.saveRegistryLock;
import static google.registry.tools.LockOrUnlockDomainCommand.REGISTRY_LOCK_STATUSES;
import static org.junit.jupiter.api.Assertions.assertThrows;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.truth.Truth8;
import google.registry.model.domain.DomainBase;
import google.registry.model.domain.DomainHistory;
import google.registry.model.domain.RegistryLock;
import google.registry.model.registrar.Registrar;
import google.registry.model.reporting.HistoryEntry;
import google.registry.testing.DeterministicStringGenerator;
import google.registry.testing.DualDatabaseTest;
import google.registry.testing.TestOfyAndSql;
import google.registry.tools.CommandTestCase;
import google.registry.util.StringGenerator.Alphabets;
import java.util.Optional;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.junit.jupiter.api.BeforeEach;
/** Unit tests for {@link BackfillRegistryLocksCommand}. */
@DualDatabaseTest
class BackfillRegistryLocksCommandTest extends CommandTestCase<BackfillRegistryLocksCommand> {
@BeforeEach
void beforeEach() {
persistNewRegistrar("adminreg", "Admin Registrar", Registrar.Type.REAL, 693L);
createTld("tld");
command.registryAdminClientId = "adminreg";
command.clock = fakeClock;
command.stringGenerator = new DeterministicStringGenerator(Alphabets.BASE_58);
}
@TestOfyAndSql
void testSimpleBackfill() throws Exception {
DomainBase domain = persistLockedDomain("example.tld");
Truth8.assertThat(getMostRecentRegistryLockByRepoId(domain.getRepoId())).isEmpty();
runCommandForced("--domain_roids", domain.getRepoId());
Optional<RegistryLock> lockOptional = getMostRecentRegistryLockByRepoId(domain.getRepoId());
Truth8.assertThat(lockOptional).isPresent();
Truth8.assertThat(lockOptional.get().getLockCompletionTime()).isPresent();
}
@TestOfyAndSql
void testBackfill_onlyLockedDomains() throws Exception {
DomainBase neverLockedDomain = persistActiveDomain("neverlocked.tld");
DomainBase previouslyLockedDomain = persistLockedDomain("unlocked.tld");
persistResource(previouslyLockedDomain.asBuilder().setStatusValues(ImmutableSet.of()).build());
DomainBase lockedDomain = persistLockedDomain("locked.tld");
runCommandForced(
"--domain_roids",
String.format(
"%s,%s,%s",
neverLockedDomain.getRepoId(),
previouslyLockedDomain.getRepoId(),
lockedDomain.getRepoId()));
ImmutableList<RegistryLock> locks = getRegistryLocksByRegistrarId("adminreg");
assertThat(locks).hasSize(1);
assertThat(Iterables.getOnlyElement(locks).getDomainName()).isEqualTo("locked.tld");
}
@TestOfyAndSql
void testBackfill_skipsDeletedDomains() throws Exception {
DomainBase domain = persistDeletedDomain("example.tld", fakeClock.nowUtc());
persistResource(domain.asBuilder().setStatusValues(REGISTRY_LOCK_STATUSES).build());
fakeClock.advanceBy(Duration.standardSeconds(1));
runCommandForced("--domain_roids", domain.getRepoId());
Truth8.assertThat(getMostRecentRegistryLockByRepoId(domain.getRepoId())).isEmpty();
}
@TestOfyAndSql
void testBackfill_skipsDomains_ifLockAlreadyExists() throws Exception {
DomainBase domain = persistLockedDomain("example.tld");
RegistryLock previousLock =
saveRegistryLock(
new RegistryLock.Builder()
.isSuperuser(true)
.setRegistrarId("adminreg")
.setRepoId(domain.getRepoId())
.setDomainName(domain.getDomainName())
.setLockCompletionTime(fakeClock.nowUtc())
.setVerificationCode(command.stringGenerator.createString(32))
.build());
fakeClock.advanceBy(Duration.standardDays(1));
runCommandForced("--domain_roids", domain.getRepoId());
assertThat(getMostRecentRegistryLockByRepoId(domain.getRepoId()).get().getLockCompletionTime())
.isEqualTo(previousLock.getLockCompletionTime());
}
@TestOfyAndSql
void testBackfill_usesUrsTime_ifExists() throws Exception {
DateTime ursTime = fakeClock.nowUtc();
DomainBase ursDomain = persistLockedDomain("urs.tld");
persistResource(
new DomainHistory.Builder()
.setBySuperuser(true)
.setRegistrarId("adminreg")
.setModificationTime(ursTime)
.setDomain(ursDomain)
.setReason("Uniform Rapid Suspension")
.setType(HistoryEntry.Type.DOMAIN_UPDATE)
.setRequestedByRegistrar(false)
.build());
DomainBase nonUrsDomain = persistLockedDomain("nonurs.tld");
persistResource(
new DomainHistory.Builder()
.setBySuperuser(true)
.setRegistrarId("adminreg")
.setDomain(nonUrsDomain)
.setType(HistoryEntry.Type.DOMAIN_UPDATE)
.setRequestedByRegistrar(false)
.setModificationTime(ursTime)
.build());
fakeClock.advanceBy(Duration.standardDays(10));
runCommandForced(
"--domain_roids", String.format("%s,%s", ursDomain.getRepoId(), nonUrsDomain.getRepoId()));
RegistryLock ursLock = getMostRecentVerifiedRegistryLockByRepoId(ursDomain.getRepoId()).get();
assertThat(ursLock.getLockCompletionTime()).hasValue(ursTime);
RegistryLock nonUrsLock =
getMostRecentVerifiedRegistryLockByRepoId(nonUrsDomain.getRepoId()).get();
assertThat(nonUrsLock.getLockCompletionTime()).hasValue(fakeClock.nowUtc());
}
@TestOfyAndSql
void testFailure_mustProvideDomainRoids() {
assertThat(assertThrows(IllegalArgumentException.class, this::runCommandForced))
.hasMessageThat()
.isEqualTo("Must provide non-empty domain_roids argument");
}
private static DomainBase persistLockedDomain(String domainName) {
DomainBase domain = persistActiveDomain(domainName);
return persistResource(domain.asBuilder().setStatusValues(REGISTRY_LOCK_STATUSES).build());
}
}

View File

@@ -1,274 +0,0 @@
// 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.insertInDb;
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.testing.DualDatabaseTest;
import google.registry.testing.TestOfyAndSql;
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;
/** Tests for {@link BackfillSpec11ThreatMatchesCommand}. */
@DualDatabaseTest
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());
}
@TestOfyAndSql
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();
}
@TestOfyAndSql
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().loadAllOf(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"));
});
}
@TestOfyAndSql
void testSuccess_empty() throws Exception {
runCommandForced();
assertInStdout("Backfill Spec11 results from 692 files?");
assertInStdout("Successfully parsed through 692 files with 0 threats.");
}
@TestOfyAndSql
void testSuccess_sameDayTwice() throws Exception {
when(threatMatchesParser.getRegistrarThreatMatches(CURRENT_DATE))
.thenReturn(sampleThreatMatches());
runCommandForced();
runCommandForced();
verifyExactlyThreeEntriesInDbFromLastDay();
}
@TestOfyAndSql
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().loadAllOf(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());
}
@TestOfyAndSql
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();
insertInDb(previous);
runCommandForced();
ImmutableList<Spec11ThreatMatch> threatMatches =
jpaTm().transact(() -> jpaTm().loadAllOf(Spec11ThreatMatch.class));
assertAboutImmutableObjects()
.that(Iterables.getOnlyElement(threatMatches))
.isEqualExceptFields(previous, "id");
}
@TestOfyAndSql
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();
insertInDb(previous);
runCommandForced("--overwrite_existing_dates");
verifyExactlyThreeEntriesInDbFromLastDay();
}
@TestOfyAndSql
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");
assertThat(jpaTm().transact(() -> jpaTm().loadAllOf(Spec11ThreatMatch.class))).isEmpty();
}
@TestOfyAndSql
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.");
}
@TestOfyAndSql
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().loadAllOf(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();
}
}