mirror of
https://github.com/google/nomulus
synced 2026-01-27 16:12:19 +00:00
Compare commits
6 Commits
nomulus-20
...
nomulus-20
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
e25885e25f | ||
|
|
cbdf4704ba | ||
|
|
207c7e7ca8 | ||
|
|
b3a0eb6bd8 | ||
|
|
c602aa6e67 | ||
|
|
c6008b65a0 |
@@ -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')
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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));
|
||||
}
|
||||
|
||||
@@ -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>
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
}
|
||||
@@ -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()));
|
||||
}
|
||||
}
|
||||
@@ -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";
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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)));
|
||||
}
|
||||
}
|
||||
@@ -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$"]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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");
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user