mirror of
https://github.com/google/nomulus.git
synced 2025-05-09 16:28:21 +02:00
Rename BackupGroupRoot (#1829)
Also removed the ability to disable update timestamp auto update as it was only needed during the migration. Lastly, rectified the use of raw Coder in RegistryJpaIO.
This commit is contained in:
parent
9e3513702e
commit
7a9d4437ed
26 changed files with 152 additions and 436 deletions
|
@ -22,18 +22,14 @@ import com.google.auto.value.AutoValue;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Streams;
|
import com.google.common.collect.Streams;
|
||||||
import google.registry.beam.common.RegistryQuery.CriteriaQuerySupplier;
|
import google.registry.beam.common.RegistryQuery.CriteriaQuerySupplier;
|
||||||
import google.registry.model.UpdateAutoTimestamp;
|
|
||||||
import google.registry.model.UpdateAutoTimestamp.DisableAutoUpdateResource;
|
|
||||||
import google.registry.persistence.transaction.JpaTransactionManager;
|
import google.registry.persistence.transaction.JpaTransactionManager;
|
||||||
import google.registry.persistence.transaction.TransactionManagerFactory;
|
import google.registry.persistence.transaction.TransactionManagerFactory;
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.concurrent.ThreadLocalRandom;
|
import java.util.concurrent.ThreadLocalRandom;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import javax.persistence.criteria.CriteriaQuery;
|
import javax.persistence.criteria.CriteriaQuery;
|
||||||
import org.apache.beam.sdk.coders.Coder;
|
import org.apache.beam.sdk.coders.Coder;
|
||||||
import org.apache.beam.sdk.coders.SerializableCoder;
|
|
||||||
import org.apache.beam.sdk.metrics.Counter;
|
import org.apache.beam.sdk.metrics.Counter;
|
||||||
import org.apache.beam.sdk.metrics.Metrics;
|
import org.apache.beam.sdk.metrics.Metrics;
|
||||||
import org.apache.beam.sdk.transforms.Create;
|
import org.apache.beam.sdk.transforms.Create;
|
||||||
|
@ -136,6 +132,7 @@ public final class RegistryJpaIO {
|
||||||
|
|
||||||
abstract SerializableFunction<R, T> resultMapper();
|
abstract SerializableFunction<R, T> resultMapper();
|
||||||
|
|
||||||
|
@Nullable
|
||||||
abstract Coder<T> coder();
|
abstract Coder<T> coder();
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -146,13 +143,16 @@ public final class RegistryJpaIO {
|
||||||
@Override
|
@Override
|
||||||
@SuppressWarnings("deprecation") // Reshuffle still recommended by GCP.
|
@SuppressWarnings("deprecation") // Reshuffle still recommended by GCP.
|
||||||
public PCollection<T> expand(PBegin input) {
|
public PCollection<T> expand(PBegin input) {
|
||||||
return input
|
PCollection<T> output =
|
||||||
|
input
|
||||||
.apply("Starting " + name(), Create.of((Void) null))
|
.apply("Starting " + name(), Create.of((Void) null))
|
||||||
.apply(
|
.apply(
|
||||||
"Run query for " + name(),
|
"Run query for " + name(),
|
||||||
ParDo.of(new QueryRunner<>(query(), resultMapper(), snapshotId())))
|
ParDo.of(new QueryRunner<>(query(), resultMapper(), snapshotId())));
|
||||||
.setCoder(coder())
|
if (coder() != null) {
|
||||||
.apply("Reshuffle", Reshuffle.viaRandomKey());
|
output = output.setCoder(coder());
|
||||||
|
}
|
||||||
|
return output.apply("Reshuffle", Reshuffle.viaRandomKey());
|
||||||
}
|
}
|
||||||
|
|
||||||
public Read<R, T> withName(String name) {
|
public Read<R, T> withName(String name) {
|
||||||
|
@ -180,9 +180,7 @@ public final class RegistryJpaIO {
|
||||||
}
|
}
|
||||||
|
|
||||||
static <R, T> Builder<R, T> builder() {
|
static <R, T> Builder<R, T> builder() {
|
||||||
return new AutoValue_RegistryJpaIO_Read.Builder<R, T>()
|
return new AutoValue_RegistryJpaIO_Read.Builder<R, T>().name(DEFAULT_NAME);
|
||||||
.name(DEFAULT_NAME)
|
|
||||||
.coder(SerializableCoder.of(Serializable.class));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@AutoValue.Builder
|
@AutoValue.Builder
|
||||||
|
@ -194,7 +192,7 @@ public final class RegistryJpaIO {
|
||||||
|
|
||||||
abstract Builder<R, T> resultMapper(SerializableFunction<R, T> mapper);
|
abstract Builder<R, T> resultMapper(SerializableFunction<R, T> mapper);
|
||||||
|
|
||||||
abstract Builder<R, T> coder(Coder coder);
|
abstract Builder<R, T> coder(Coder<T> coder);
|
||||||
|
|
||||||
abstract Builder<R, T> snapshotId(@Nullable String sharedSnapshotId);
|
abstract Builder<R, T> snapshotId(@Nullable String sharedSnapshotId);
|
||||||
|
|
||||||
|
@ -299,12 +297,6 @@ public final class RegistryJpaIO {
|
||||||
|
|
||||||
public abstract SerializableFunction<T, Object> jpaConverter();
|
public abstract SerializableFunction<T, Object> jpaConverter();
|
||||||
|
|
||||||
/**
|
|
||||||
* Signal to the writer that the {@link UpdateAutoTimestamp} property should be allowed to
|
|
||||||
* manipulate its value before persistence. The default value is {@code true}.
|
|
||||||
*/
|
|
||||||
abstract boolean withUpdateAutoTimestamp();
|
|
||||||
|
|
||||||
public Write<T> withName(String name) {
|
public Write<T> withName(String name) {
|
||||||
return toBuilder().name(name).build();
|
return toBuilder().name(name).build();
|
||||||
}
|
}
|
||||||
|
@ -325,10 +317,6 @@ public final class RegistryJpaIO {
|
||||||
return toBuilder().jpaConverter(jpaConverter).build();
|
return toBuilder().jpaConverter(jpaConverter).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Write<T> disableUpdateAutoTimestamp() {
|
|
||||||
return toBuilder().withUpdateAutoTimestamp(false).build();
|
|
||||||
}
|
|
||||||
|
|
||||||
abstract Builder<T> toBuilder();
|
abstract Builder<T> toBuilder();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -345,7 +333,7 @@ public final class RegistryJpaIO {
|
||||||
GroupIntoBatches.<Integer, T>ofSize(batchSize()).withShardedKey())
|
GroupIntoBatches.<Integer, T>ofSize(batchSize()).withShardedKey())
|
||||||
.apply(
|
.apply(
|
||||||
"Write in batch for " + name(),
|
"Write in batch for " + name(),
|
||||||
ParDo.of(new SqlBatchWriter<>(name(), jpaConverter(), withUpdateAutoTimestamp())));
|
ParDo.of(new SqlBatchWriter<>(name(), jpaConverter())));
|
||||||
}
|
}
|
||||||
|
|
||||||
static <T> Builder<T> builder() {
|
static <T> Builder<T> builder() {
|
||||||
|
@ -353,8 +341,7 @@ public final class RegistryJpaIO {
|
||||||
.name(DEFAULT_NAME)
|
.name(DEFAULT_NAME)
|
||||||
.batchSize(DEFAULT_BATCH_SIZE)
|
.batchSize(DEFAULT_BATCH_SIZE)
|
||||||
.shards(DEFAULT_SHARDS)
|
.shards(DEFAULT_SHARDS)
|
||||||
.jpaConverter(x -> x)
|
.jpaConverter(x -> x);
|
||||||
.withUpdateAutoTimestamp(true);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@AutoValue.Builder
|
@AutoValue.Builder
|
||||||
|
@ -368,8 +355,6 @@ public final class RegistryJpaIO {
|
||||||
|
|
||||||
abstract Builder<T> jpaConverter(SerializableFunction<T, Object> jpaConverter);
|
abstract Builder<T> jpaConverter(SerializableFunction<T, Object> jpaConverter);
|
||||||
|
|
||||||
abstract Builder<T> withUpdateAutoTimestamp(boolean withUpdateAutoTimestamp);
|
|
||||||
|
|
||||||
abstract Write<T> build();
|
abstract Write<T> build();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -378,24 +363,15 @@ public final class RegistryJpaIO {
|
||||||
private static class SqlBatchWriter<T> extends DoFn<KV<ShardedKey<Integer>, Iterable<T>>, Void> {
|
private static class SqlBatchWriter<T> extends DoFn<KV<ShardedKey<Integer>, Iterable<T>>, Void> {
|
||||||
private final Counter counter;
|
private final Counter counter;
|
||||||
private final SerializableFunction<T, Object> jpaConverter;
|
private final SerializableFunction<T, Object> jpaConverter;
|
||||||
private final boolean withAutoTimestamp;
|
|
||||||
|
|
||||||
SqlBatchWriter(
|
SqlBatchWriter(String type, SerializableFunction<T, Object> jpaConverter) {
|
||||||
String type, SerializableFunction<T, Object> jpaConverter, boolean withAutoTimestamp) {
|
|
||||||
counter = Metrics.counter("SQL_WRITE", type);
|
counter = Metrics.counter("SQL_WRITE", type);
|
||||||
this.jpaConverter = jpaConverter;
|
this.jpaConverter = jpaConverter;
|
||||||
this.withAutoTimestamp = withAutoTimestamp;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@ProcessElement
|
@ProcessElement
|
||||||
public void processElement(@Element KV<ShardedKey<Integer>, Iterable<T>> kv) {
|
public void processElement(@Element KV<ShardedKey<Integer>, Iterable<T>> kv) {
|
||||||
if (withAutoTimestamp) {
|
|
||||||
actuallyProcessElement(kv);
|
actuallyProcessElement(kv);
|
||||||
return;
|
|
||||||
}
|
|
||||||
try (DisableAutoUpdateResource disable = UpdateAutoTimestamp.disableAutoUpdate()) {
|
|
||||||
actuallyProcessElement(kv);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void actuallyProcessElement(@Element KV<ShardedKey<Integer>, Iterable<T>> kv) {
|
private void actuallyProcessElement(@Element KV<ShardedKey<Integer>, Iterable<T>> kv) {
|
||||||
|
|
|
@ -37,6 +37,7 @@ import java.util.Optional;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
import org.apache.beam.sdk.Pipeline;
|
import org.apache.beam.sdk.Pipeline;
|
||||||
import org.apache.beam.sdk.PipelineResult;
|
import org.apache.beam.sdk.PipelineResult;
|
||||||
|
import org.apache.beam.sdk.coders.SerializableCoder;
|
||||||
import org.apache.beam.sdk.coders.StringUtf8Coder;
|
import org.apache.beam.sdk.coders.StringUtf8Coder;
|
||||||
import org.apache.beam.sdk.io.FileIO;
|
import org.apache.beam.sdk.io.FileIO;
|
||||||
import org.apache.beam.sdk.io.TextIO;
|
import org.apache.beam.sdk.io.TextIO;
|
||||||
|
@ -93,8 +94,9 @@ public class InvoicingPipeline implements Serializable {
|
||||||
static PCollection<BillingEvent> readFromCloudSql(
|
static PCollection<BillingEvent> readFromCloudSql(
|
||||||
InvoicingPipelineOptions options, Pipeline pipeline) {
|
InvoicingPipelineOptions options, Pipeline pipeline) {
|
||||||
Read<Object[], BillingEvent> read =
|
Read<Object[], BillingEvent> read =
|
||||||
RegistryJpaIO.read(
|
RegistryJpaIO.<Object[], BillingEvent>read(
|
||||||
makeCloudSqlQuery(options.getYearMonth()), false, row -> parseRow(row).orElse(null));
|
makeCloudSqlQuery(options.getYearMonth()), false, row -> parseRow(row).orElse(null))
|
||||||
|
.withCoder(SerializableCoder.of(BillingEvent.class));
|
||||||
|
|
||||||
PCollection<BillingEvent> billingEventsWithNulls =
|
PCollection<BillingEvent> billingEventsWithNulls =
|
||||||
pipeline.apply("Read BillingEvents from Cloud SQL", read);
|
pipeline.apply("Read BillingEvents from Cloud SQL", read);
|
||||||
|
|
|
@ -304,7 +304,8 @@ public class RdePipeline implements Serializable {
|
||||||
"SELECT registrarId FROM Registrar WHERE type NOT IN (:types)",
|
"SELECT registrarId FROM Registrar WHERE type NOT IN (:types)",
|
||||||
ImmutableMap.of("types", IGNORED_REGISTRAR_TYPES),
|
ImmutableMap.of("types", IGNORED_REGISTRAR_TYPES),
|
||||||
String.class,
|
String.class,
|
||||||
id -> VKey.createSql(Registrar.class, id)))
|
x -> x)
|
||||||
|
.withCoder(StringUtf8Coder.of()))
|
||||||
.apply(
|
.apply(
|
||||||
"Marshall Registrar into DepositFragment",
|
"Marshall Registrar into DepositFragment",
|
||||||
FlatMapElements.into(
|
FlatMapElements.into(
|
||||||
|
@ -312,7 +313,8 @@ public class RdePipeline implements Serializable {
|
||||||
TypeDescriptor.of(PendingDeposit.class),
|
TypeDescriptor.of(PendingDeposit.class),
|
||||||
TypeDescriptor.of(DepositFragment.class)))
|
TypeDescriptor.of(DepositFragment.class)))
|
||||||
.via(
|
.via(
|
||||||
(VKey<Registrar> key) -> {
|
(String registrarRepoId) -> {
|
||||||
|
VKey<Registrar> key = VKey.createSql(Registrar.class, registrarRepoId);
|
||||||
includedRegistrarCounter.inc();
|
includedRegistrarCounter.inc();
|
||||||
Registrar registrar = jpaTm().transact(() -> jpaTm().loadByKey(key));
|
Registrar registrar = jpaTm().transact(() -> jpaTm().loadByKey(key));
|
||||||
DepositFragment fragment = marshaller.marshalRegistrar(registrar);
|
DepositFragment fragment = marshaller.marshalRegistrar(registrar);
|
||||||
|
@ -337,8 +339,7 @@ public class RdePipeline implements Serializable {
|
||||||
Pipeline pipeline, Class<T> historyClass) {
|
Pipeline pipeline, Class<T> historyClass) {
|
||||||
String repoIdFieldName = HistoryEntryDao.REPO_ID_FIELD_NAMES.get(historyClass);
|
String repoIdFieldName = HistoryEntryDao.REPO_ID_FIELD_NAMES.get(historyClass);
|
||||||
String resourceFieldName = EPP_RESOURCE_FIELD_NAME.get(historyClass);
|
String resourceFieldName = EPP_RESOURCE_FIELD_NAME.get(historyClass);
|
||||||
return pipeline
|
return pipeline.apply(
|
||||||
.apply(
|
|
||||||
String.format("Load most recent %s", historyClass.getSimpleName()),
|
String.format("Load most recent %s", historyClass.getSimpleName()),
|
||||||
RegistryJpaIO.read(
|
RegistryJpaIO.read(
|
||||||
("SELECT %repoIdField%, id FROM %entity% WHERE (%repoIdField%, modificationTime)"
|
("SELECT %repoIdField%, id FROM %entity% WHERE (%repoIdField%, modificationTime)"
|
||||||
|
@ -358,8 +359,8 @@ public class RdePipeline implements Serializable {
|
||||||
.replace("%resourceField%", resourceFieldName),
|
.replace("%resourceField%", resourceFieldName),
|
||||||
ImmutableMap.of("watermark", watermark),
|
ImmutableMap.of("watermark", watermark),
|
||||||
Object[].class,
|
Object[].class,
|
||||||
row -> KV.of((String) row[0], (long) row[1])))
|
row -> KV.of((String) row[0], (long) row[1]))
|
||||||
.setCoder(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()));
|
.withCoder(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())));
|
||||||
}
|
}
|
||||||
|
|
||||||
private <T extends HistoryEntry> EppResource loadResourceByHistoryEntryId(
|
private <T extends HistoryEntry> EppResource loadResourceByHistoryEntryId(
|
||||||
|
|
|
@ -36,6 +36,7 @@ import java.io.Serializable;
|
||||||
import java.util.concurrent.ThreadLocalRandom;
|
import java.util.concurrent.ThreadLocalRandom;
|
||||||
import org.apache.beam.sdk.Pipeline;
|
import org.apache.beam.sdk.Pipeline;
|
||||||
import org.apache.beam.sdk.PipelineResult;
|
import org.apache.beam.sdk.PipelineResult;
|
||||||
|
import org.apache.beam.sdk.coders.StringUtf8Coder;
|
||||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||||
import org.apache.beam.sdk.transforms.DoFn;
|
import org.apache.beam.sdk.transforms.DoFn;
|
||||||
import org.apache.beam.sdk.transforms.GroupIntoBatches;
|
import org.apache.beam.sdk.transforms.GroupIntoBatches;
|
||||||
|
@ -106,7 +107,8 @@ public class ResaveAllEppResourcesPipeline implements Serializable {
|
||||||
"SELECT repoId FROM Contact WHERE transferData.transferStatus = 'PENDING' AND"
|
"SELECT repoId FROM Contact WHERE transferData.transferStatus = 'PENDING' AND"
|
||||||
+ " transferData.pendingTransferExpirationTime < current_timestamp()",
|
+ " transferData.pendingTransferExpirationTime < current_timestamp()",
|
||||||
String.class,
|
String.class,
|
||||||
r -> r);
|
r -> r)
|
||||||
|
.withCoder(StringUtf8Coder.of());
|
||||||
projectAndResaveResources(pipeline, Contact.class, repoIdRead);
|
projectAndResaveResources(pipeline, Contact.class, repoIdRead);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -123,7 +125,8 @@ public class ResaveAllEppResourcesPipeline implements Serializable {
|
||||||
DOMAINS_TO_PROJECT_QUERY,
|
DOMAINS_TO_PROJECT_QUERY,
|
||||||
ImmutableMap.of("END_OF_TIME", DateTimeUtils.END_OF_TIME),
|
ImmutableMap.of("END_OF_TIME", DateTimeUtils.END_OF_TIME),
|
||||||
String.class,
|
String.class,
|
||||||
r -> r);
|
r -> r)
|
||||||
|
.withCoder(StringUtf8Coder.of());
|
||||||
projectAndResaveResources(pipeline, Domain.class, repoIdRead);
|
projectAndResaveResources(pipeline, Domain.class, repoIdRead);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -132,7 +135,8 @@ public class ResaveAllEppResourcesPipeline implements Serializable {
|
||||||
Read<String, String> repoIdRead =
|
Read<String, String> repoIdRead =
|
||||||
RegistryJpaIO.read(
|
RegistryJpaIO.read(
|
||||||
// Note: cannot use SQL parameters for the table name
|
// Note: cannot use SQL parameters for the table name
|
||||||
String.format("SELECT repoId FROM %s", clazz.getSimpleName()), String.class, r -> r);
|
String.format("SELECT repoId FROM %s", clazz.getSimpleName()), String.class, r -> r)
|
||||||
|
.withCoder(StringUtf8Coder.of());
|
||||||
projectAndResaveResources(pipeline, clazz, repoIdRead);
|
projectAndResaveResources(pipeline, clazz, repoIdRead);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -37,6 +37,8 @@ import java.io.Serializable;
|
||||||
import javax.inject.Singleton;
|
import javax.inject.Singleton;
|
||||||
import org.apache.beam.sdk.Pipeline;
|
import org.apache.beam.sdk.Pipeline;
|
||||||
import org.apache.beam.sdk.PipelineResult;
|
import org.apache.beam.sdk.PipelineResult;
|
||||||
|
import org.apache.beam.sdk.coders.KvCoder;
|
||||||
|
import org.apache.beam.sdk.coders.StringUtf8Coder;
|
||||||
import org.apache.beam.sdk.io.TextIO;
|
import org.apache.beam.sdk.io.TextIO;
|
||||||
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
import org.apache.beam.sdk.options.PipelineOptionsFactory;
|
||||||
import org.apache.beam.sdk.transforms.DoFn;
|
import org.apache.beam.sdk.transforms.DoFn;
|
||||||
|
@ -116,7 +118,8 @@ public class Spec11Pipeline implements Serializable {
|
||||||
+ " d.currentSponsorClientId = r.registrarId where r.type = 'REAL' and"
|
+ " d.currentSponsorClientId = r.registrarId where r.type = 'REAL' and"
|
||||||
+ " d.deletionTime > now()",
|
+ " d.deletionTime > now()",
|
||||||
false,
|
false,
|
||||||
Spec11Pipeline::parseRow);
|
Spec11Pipeline::parseRow)
|
||||||
|
.withCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
|
||||||
|
|
||||||
return pipeline
|
return pipeline
|
||||||
.apply("Read active domains from Cloud SQL", read)
|
.apply("Read active domains from Cloud SQL", read)
|
||||||
|
@ -214,8 +217,7 @@ public class Spec11Pipeline implements Serializable {
|
||||||
return output.toString();
|
return output.toString();
|
||||||
} catch (JSONException e) {
|
} catch (JSONException e) {
|
||||||
throw new RuntimeException(
|
throw new RuntimeException(
|
||||||
String.format(
|
String.format("Encountered an error constructing the JSON for %s", kv),
|
||||||
"Encountered an error constructing the JSON for %s", kv.toString()),
|
|
||||||
e);
|
e);
|
||||||
}
|
}
|
||||||
}))
|
}))
|
||||||
|
|
|
@ -56,7 +56,7 @@ import org.joda.time.DateTime;
|
||||||
/** An EPP entity object (i.e. a domain, contact, or host). */
|
/** An EPP entity object (i.e. a domain, contact, or host). */
|
||||||
@MappedSuperclass
|
@MappedSuperclass
|
||||||
@Access(AccessType.FIELD) // otherwise it'll use the default if the repoId (property)
|
@Access(AccessType.FIELD) // otherwise it'll use the default if the repoId (property)
|
||||||
public abstract class EppResource extends BackupGroupRoot implements Buildable {
|
public abstract class EppResource extends UpdateAutoTimestampEntity implements Buildable {
|
||||||
|
|
||||||
private static final long serialVersionUID = -252782773382339534L;
|
private static final long serialVersionUID = -252782773382339534L;
|
||||||
|
|
||||||
|
@ -339,7 +339,7 @@ public abstract class EppResource extends BackupGroupRoot implements Buildable {
|
||||||
/**
|
/**
|
||||||
* Set the update timestamp.
|
* Set the update timestamp.
|
||||||
*
|
*
|
||||||
* <p>This is provided at EppResource since BackupGroupRoot doesn't have a Builder.
|
* <p>This is provided at EppResource since UpdateAutoTimestampEntity doesn't have a Builder.
|
||||||
*/
|
*/
|
||||||
public B setUpdateTimestamp(UpdateAutoTimestamp updateTimestamp) {
|
public B setUpdateTimestamp(UpdateAutoTimestamp updateTimestamp) {
|
||||||
getInstance().setUpdateTimestamp(updateTimestamp);
|
getInstance().setUpdateTimestamp(updateTimestamp);
|
||||||
|
|
|
@ -29,12 +29,7 @@ import org.joda.time.DateTime;
|
||||||
@Embeddable
|
@Embeddable
|
||||||
public class UpdateAutoTimestamp extends ImmutableObject implements UnsafeSerializable {
|
public class UpdateAutoTimestamp extends ImmutableObject implements UnsafeSerializable {
|
||||||
|
|
||||||
// When set to true, database converters/translators should do the auto update. When set to
|
@Column(name = "updateTimestamp")
|
||||||
// false, auto update should be suspended (this exists to allow us to preserve the original value
|
|
||||||
// during a replay).
|
|
||||||
private static final ThreadLocal<Boolean> autoUpdateEnabled = ThreadLocal.withInitial(() -> true);
|
|
||||||
|
|
||||||
@Column(nullable = false)
|
|
||||||
DateTime lastUpdateTime;
|
DateTime lastUpdateTime;
|
||||||
|
|
||||||
// Unfortunately, we cannot use the @UpdateTimestamp annotation on "lastUpdateTime" in this class
|
// Unfortunately, we cannot use the @UpdateTimestamp annotation on "lastUpdateTime" in this class
|
||||||
|
@ -43,10 +38,8 @@ public class UpdateAutoTimestamp extends ImmutableObject implements UnsafeSerial
|
||||||
@PrePersist
|
@PrePersist
|
||||||
@PreUpdate
|
@PreUpdate
|
||||||
void setTimestamp() {
|
void setTimestamp() {
|
||||||
if (autoUpdateEnabled() || lastUpdateTime == null) {
|
|
||||||
lastUpdateTime = jpaTm().getTransactionTime();
|
lastUpdateTime = jpaTm().getTransactionTime();
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns the timestamp, or {@code START_OF_TIME} if it's null. */
|
/** Returns the timestamp, or {@code START_OF_TIME} if it's null. */
|
||||||
public DateTime getTimestamp() {
|
public DateTime getTimestamp() {
|
||||||
|
@ -58,30 +51,4 @@ public class UpdateAutoTimestamp extends ImmutableObject implements UnsafeSerial
|
||||||
instance.lastUpdateTime = timestamp;
|
instance.lastUpdateTime = timestamp;
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO(b/175610935): Remove the auto-update disabling code below after migration.
|
|
||||||
|
|
||||||
/** Class to allow us to safely disable auto-update in a try-with-resources block. */
|
|
||||||
public static class DisableAutoUpdateResource implements AutoCloseable {
|
|
||||||
DisableAutoUpdateResource() {
|
|
||||||
autoUpdateEnabled.set(false);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() {
|
|
||||||
autoUpdateEnabled.set(true);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Resturns a resource that disables auto-updates on all {@link UpdateAutoTimestamp}s in the
|
|
||||||
* current thread, suitable for use with in a try-with-resources block.
|
|
||||||
*/
|
|
||||||
public static DisableAutoUpdateResource disableAutoUpdate() {
|
|
||||||
return new DisableAutoUpdateResource();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static boolean autoUpdateEnabled() {
|
|
||||||
return autoUpdateEnabled.get();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,21 +18,16 @@ import com.googlecode.objectify.annotation.Ignore;
|
||||||
import google.registry.util.PreconditionsUtils;
|
import google.registry.util.PreconditionsUtils;
|
||||||
import javax.persistence.Access;
|
import javax.persistence.Access;
|
||||||
import javax.persistence.AccessType;
|
import javax.persistence.AccessType;
|
||||||
import javax.persistence.AttributeOverride;
|
|
||||||
import javax.persistence.Column;
|
|
||||||
import javax.persistence.MappedSuperclass;
|
import javax.persistence.MappedSuperclass;
|
||||||
import javax.xml.bind.annotation.XmlTransient;
|
import javax.xml.bind.annotation.XmlTransient;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for entities that are the root of a Registry 2.0 entity group that gets enrolled in
|
* Base class for entities that contains an {@link UpdateAutoTimestamp} which is updated every time
|
||||||
* commit logs for backup purposes.
|
* the entity is persisted.
|
||||||
*
|
|
||||||
* <p>The commit log system needs to preserve the ordering of closely timed mutations to entities in
|
|
||||||
* a single entity group. We require an {@link UpdateAutoTimestamp} field on the root of a group so
|
|
||||||
* that we can enforce strictly increasing timestamps.
|
|
||||||
*/
|
*/
|
||||||
@MappedSuperclass
|
@MappedSuperclass
|
||||||
public abstract class BackupGroupRoot extends ImmutableObject implements UnsafeSerializable {
|
public abstract class UpdateAutoTimestampEntity extends ImmutableObject
|
||||||
|
implements UnsafeSerializable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An automatically managed timestamp of when this object was last written to Datastore.
|
* An automatically managed timestamp of when this object was last written to Datastore.
|
||||||
|
@ -44,7 +39,6 @@ public abstract class BackupGroupRoot extends ImmutableObject implements UnsafeS
|
||||||
// Prevents subclasses from unexpectedly accessing as property (e.g., Host), which would
|
// Prevents subclasses from unexpectedly accessing as property (e.g., Host), which would
|
||||||
// require an unnecessary non-private setter method.
|
// require an unnecessary non-private setter method.
|
||||||
@Access(AccessType.FIELD)
|
@Access(AccessType.FIELD)
|
||||||
@AttributeOverride(name = "lastUpdateTime", column = @Column(name = "updateTimestamp"))
|
|
||||||
@Ignore
|
@Ignore
|
||||||
UpdateAutoTimestamp updateTimestamp = UpdateAutoTimestamp.create(null);
|
UpdateAutoTimestamp updateTimestamp = UpdateAutoTimestamp.create(null);
|
||||||
|
|
||||||
|
@ -59,7 +53,7 @@ public abstract class BackupGroupRoot extends ImmutableObject implements UnsafeS
|
||||||
* <p>This method is for the few cases when {@code updateTimestamp} is copied between different
|
* <p>This method is for the few cases when {@code updateTimestamp} is copied between different
|
||||||
* types of entities. Use {@link #clone} for same-type copying.
|
* types of entities. Use {@link #clone} for same-type copying.
|
||||||
*/
|
*/
|
||||||
protected void copyUpdateTimestamp(BackupGroupRoot other) {
|
protected void copyUpdateTimestamp(UpdateAutoTimestampEntity other) {
|
||||||
this.updateTimestamp = PreconditionsUtils.checkArgumentNotNull(other, "other").updateTimestamp;
|
this.updateTimestamp = PreconditionsUtils.checkArgumentNotNull(other, "other").updateTimestamp;
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,11 +20,12 @@ import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||||
|
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
import google.registry.model.UnsafeSerializable;
|
import google.registry.model.UnsafeSerializable;
|
||||||
import google.registry.model.UpdateAutoTimestamp;
|
import google.registry.model.UpdateAutoTimestampEntity;
|
||||||
import google.registry.model.common.Cursor.CursorId;
|
import google.registry.model.common.Cursor.CursorId;
|
||||||
import google.registry.model.tld.Registry;
|
import google.registry.model.tld.Registry;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import javax.persistence.AttributeOverride;
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
import javax.persistence.Entity;
|
import javax.persistence.Entity;
|
||||||
import javax.persistence.EnumType;
|
import javax.persistence.EnumType;
|
||||||
|
@ -41,7 +42,10 @@ import org.joda.time.DateTime;
|
||||||
*/
|
*/
|
||||||
@Entity
|
@Entity
|
||||||
@IdClass(CursorId.class)
|
@IdClass(CursorId.class)
|
||||||
public class Cursor extends ImmutableObject implements UnsafeSerializable {
|
@AttributeOverride(
|
||||||
|
name = "updateTimestamp.lastUpdateTime",
|
||||||
|
column = @Column(nullable = false, name = "lastUpdateTime"))
|
||||||
|
public class Cursor extends UpdateAutoTimestampEntity {
|
||||||
|
|
||||||
private static final long serialVersionUID = 5777891565780594961L;
|
private static final long serialVersionUID = 5777891565780594961L;
|
||||||
|
|
||||||
|
@ -122,10 +126,6 @@ public class Cursor extends ImmutableObject implements UnsafeSerializable {
|
||||||
@Column(nullable = false)
|
@Column(nullable = false)
|
||||||
DateTime cursorTime = START_OF_TIME;
|
DateTime cursorTime = START_OF_TIME;
|
||||||
|
|
||||||
/** An automatically managed timestamp of when this object was last written to Datastore. */
|
|
||||||
@Column(nullable = false)
|
|
||||||
UpdateAutoTimestamp lastUpdateTime = UpdateAutoTimestamp.create(null);
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VKey<Cursor> createVKey() {
|
public VKey<Cursor> createVKey() {
|
||||||
return createVKey(type, scope);
|
return createVKey(type, scope);
|
||||||
|
@ -145,9 +145,10 @@ public class Cursor extends ImmutableObject implements UnsafeSerializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
public DateTime getLastUpdateTime() {
|
public DateTime getLastUpdateTime() {
|
||||||
return lastUpdateTime.getTimestamp();
|
return getUpdateTimestamp().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
public String getScope() {
|
public String getScope() {
|
||||||
return scope;
|
return scope;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,8 +22,8 @@ import static google.registry.util.PasswordUtils.SALT_SUPPLIER;
|
||||||
import static google.registry.util.PasswordUtils.hashPassword;
|
import static google.registry.util.PasswordUtils.hashPassword;
|
||||||
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
||||||
|
|
||||||
import google.registry.model.BackupGroupRoot;
|
|
||||||
import google.registry.model.Buildable;
|
import google.registry.model.Buildable;
|
||||||
|
import google.registry.model.UpdateAutoTimestampEntity;
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
import javax.persistence.Entity;
|
import javax.persistence.Entity;
|
||||||
import javax.persistence.GeneratedValue;
|
import javax.persistence.GeneratedValue;
|
||||||
|
@ -39,7 +39,7 @@ import javax.persistence.Table;
|
||||||
@Index(columnList = "gaiaId", name = "user_gaia_id_idx"),
|
@Index(columnList = "gaiaId", name = "user_gaia_id_idx"),
|
||||||
@Index(columnList = "emailAddress", name = "user_email_address_idx")
|
@Index(columnList = "emailAddress", name = "user_email_address_idx")
|
||||||
})
|
})
|
||||||
public class User extends BackupGroupRoot implements Buildable {
|
public class User extends UpdateAutoTimestampEntity implements Buildable {
|
||||||
|
|
||||||
/** Autogenerated unique ID of this user. */
|
/** Autogenerated unique ID of this user. */
|
||||||
@Id
|
@Id
|
||||||
|
|
|
@ -20,10 +20,11 @@ import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
||||||
|
|
||||||
import google.registry.model.Buildable;
|
import google.registry.model.Buildable;
|
||||||
import google.registry.model.CreateAutoTimestamp;
|
import google.registry.model.CreateAutoTimestamp;
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.UpdateAutoTimestampEntity;
|
||||||
import google.registry.model.UpdateAutoTimestamp;
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
import javax.persistence.Access;
|
||||||
|
import javax.persistence.AccessType;
|
||||||
import javax.persistence.AttributeOverride;
|
import javax.persistence.AttributeOverride;
|
||||||
import javax.persistence.AttributeOverrides;
|
import javax.persistence.AttributeOverrides;
|
||||||
import javax.persistence.Column;
|
import javax.persistence.Column;
|
||||||
|
@ -48,7 +49,7 @@ import org.joda.time.Duration;
|
||||||
* the completion time will remain null and the lock will have no effect. The same applies for
|
* the completion time will remain null and the lock will have no effect. The same applies for
|
||||||
* unlock actions.
|
* unlock actions.
|
||||||
*
|
*
|
||||||
* <p>Note that there will be at most one row per domain with a null copmleted time -- this means
|
* <p>Note that there will be at most one row per domain with a null completed time -- this means
|
||||||
* that there is at most one pending action per domain. This is enforced at the logic level.
|
* that there is at most one pending action per domain. This is enforced at the logic level.
|
||||||
*
|
*
|
||||||
* <p>Note as well that in the case of a retry of a write after an unexpected success, the unique
|
* <p>Note as well that in the case of a retry of a write after an unexpected success, the unique
|
||||||
|
@ -56,12 +57,12 @@ import org.joda.time.Duration;
|
||||||
*/
|
*/
|
||||||
@Entity
|
@Entity
|
||||||
@Table(
|
@Table(
|
||||||
/**
|
/*
|
||||||
* Unique constraint to get around Hibernate's failure to handle auto-increment field in
|
* Unique constraint to get around Hibernate's failure to handle auto-increment field in
|
||||||
* composite primary key.
|
* composite primary key.
|
||||||
*
|
*
|
||||||
* <p>Note: indexes use the camelCase version of the field names because the {@link
|
* Note: indexes use the camelCase version of the field names because
|
||||||
* google.registry.persistence.NomulusNamingStrategy} does not translate the field name into the
|
* google.registry.persistence.NomulusNamingStrategy does not translate the field name into the
|
||||||
* snake_case column name until the write itself.
|
* snake_case column name until the write itself.
|
||||||
*/
|
*/
|
||||||
indexes = {
|
indexes = {
|
||||||
|
@ -72,7 +73,11 @@ import org.joda.time.Duration;
|
||||||
@Index(name = "idx_registry_lock_verification_code", columnList = "verificationCode"),
|
@Index(name = "idx_registry_lock_verification_code", columnList = "verificationCode"),
|
||||||
@Index(name = "idx_registry_lock_registrar_id", columnList = "registrarId")
|
@Index(name = "idx_registry_lock_registrar_id", columnList = "registrarId")
|
||||||
})
|
})
|
||||||
public final class RegistryLock extends ImmutableObject implements Buildable {
|
@Access(AccessType.FIELD)
|
||||||
|
@AttributeOverride(
|
||||||
|
name = "updateTimestamp.lastUpdateTime",
|
||||||
|
column = @Column(nullable = false, name = "lastUpdateTime"))
|
||||||
|
public final class RegistryLock extends UpdateAutoTimestampEntity implements Buildable {
|
||||||
|
|
||||||
@Id
|
@Id
|
||||||
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
||||||
|
@ -103,7 +108,7 @@ public final class RegistryLock extends ImmutableObject implements Buildable {
|
||||||
name = "creationTime",
|
name = "creationTime",
|
||||||
column = @Column(name = "lockRequestTime", nullable = false))
|
column = @Column(name = "lockRequestTime", nullable = false))
|
||||||
})
|
})
|
||||||
private CreateAutoTimestamp lockRequestTime = CreateAutoTimestamp.create(null);
|
private final CreateAutoTimestamp lockRequestTime = CreateAutoTimestamp.create(null);
|
||||||
|
|
||||||
/** When the unlock is first requested. */
|
/** When the unlock is first requested. */
|
||||||
private DateTime unlockRequestTime;
|
private DateTime unlockRequestTime;
|
||||||
|
@ -140,9 +145,6 @@ public final class RegistryLock extends ImmutableObject implements Buildable {
|
||||||
/** The duration after which we will re-lock this domain after it is unlocked. */
|
/** The duration after which we will re-lock this domain after it is unlocked. */
|
||||||
private Duration relockDuration;
|
private Duration relockDuration;
|
||||||
|
|
||||||
/** Time that this entity was last updated. */
|
|
||||||
private UpdateAutoTimestamp lastUpdateTime = UpdateAutoTimestamp.create(null);
|
|
||||||
|
|
||||||
public String getRepoId() {
|
public String getRepoId() {
|
||||||
return repoId;
|
return repoId;
|
||||||
}
|
}
|
||||||
|
@ -189,7 +191,7 @@ public final class RegistryLock extends ImmutableObject implements Buildable {
|
||||||
}
|
}
|
||||||
|
|
||||||
public DateTime getLastUpdateTime() {
|
public DateTime getLastUpdateTime() {
|
||||||
return lastUpdateTime.getTimestamp();
|
return getUpdateTimestamp().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Long getRevisionId() {
|
public Long getRevisionId() {
|
||||||
|
@ -199,7 +201,7 @@ public final class RegistryLock extends ImmutableObject implements Buildable {
|
||||||
/**
|
/**
|
||||||
* The lock that undoes this lock, if this lock has been unlocked and the domain locked again.
|
* The lock that undoes this lock, if this lock has been unlocked and the domain locked again.
|
||||||
*
|
*
|
||||||
* <p>Note: this is lazily loaded, so it may not be initialized if referenced outside of the
|
* <p>Note: this is lazily loaded, so it may not be initialized if referenced outside the
|
||||||
* transaction in which this lock is loaded.
|
* transaction in which this lock is loaded.
|
||||||
*/
|
*/
|
||||||
public RegistryLock getRelock() {
|
public RegistryLock getRelock() {
|
||||||
|
|
|
@ -33,9 +33,9 @@ import com.google.common.collect.ImmutableSortedMap;
|
||||||
import com.google.common.collect.Range;
|
import com.google.common.collect.Range;
|
||||||
import google.registry.flows.EppException;
|
import google.registry.flows.EppException;
|
||||||
import google.registry.flows.domain.DomainFlowUtils;
|
import google.registry.flows.domain.DomainFlowUtils;
|
||||||
import google.registry.model.BackupGroupRoot;
|
|
||||||
import google.registry.model.Buildable;
|
import google.registry.model.Buildable;
|
||||||
import google.registry.model.CreateAutoTimestamp;
|
import google.registry.model.CreateAutoTimestamp;
|
||||||
|
import google.registry.model.UpdateAutoTimestampEntity;
|
||||||
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
|
import google.registry.model.billing.BillingEvent.RenewalPriceBehavior;
|
||||||
import google.registry.model.common.TimedTransitionProperty;
|
import google.registry.model.common.TimedTransitionProperty;
|
||||||
import google.registry.model.reporting.HistoryEntry;
|
import google.registry.model.reporting.HistoryEntry;
|
||||||
|
@ -66,7 +66,7 @@ import org.joda.time.DateTime;
|
||||||
@Index(columnList = "tokenType"),
|
@Index(columnList = "tokenType"),
|
||||||
@Index(columnList = "redemption_domain_repo_id")
|
@Index(columnList = "redemption_domain_repo_id")
|
||||||
})
|
})
|
||||||
public class AllocationToken extends BackupGroupRoot implements Buildable {
|
public class AllocationToken extends UpdateAutoTimestampEntity implements Buildable {
|
||||||
|
|
||||||
private static final long serialVersionUID = -3954475393220876903L;
|
private static final long serialVersionUID = -3954475393220876903L;
|
||||||
private static final String REMOVE_PACKAGE = "__REMOVEPACKAGE__";
|
private static final String REMOVE_PACKAGE = "__REMOVEPACKAGE__";
|
||||||
|
|
|
@ -16,7 +16,7 @@ package google.registry.model.ofy;
|
||||||
|
|
||||||
import com.googlecode.objectify.Key;
|
import com.googlecode.objectify.Key;
|
||||||
import com.googlecode.objectify.Objectify;
|
import com.googlecode.objectify.Objectify;
|
||||||
import google.registry.model.BackupGroupRoot;
|
import google.registry.model.UpdateAutoTimestampEntity;
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
import google.registry.model.annotations.DeleteAfterMigration;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -34,7 +34,7 @@ class TimestampInversionException extends RuntimeException {
|
||||||
}
|
}
|
||||||
|
|
||||||
TimestampInversionException(
|
TimestampInversionException(
|
||||||
DateTime transactionTime, Map<Key<BackupGroupRoot>, DateTime> problematicRoots) {
|
DateTime transactionTime, Map<Key<UpdateAutoTimestampEntity>, DateTime> problematicRoots) {
|
||||||
this(transactionTime, "entities rooted under:\n" + problematicRoots);
|
this(transactionTime, "entities rooted under:\n" + problematicRoots);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,8 +19,8 @@ import static google.registry.model.rde.RdeNamingUtils.makePartialName;
|
||||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||||
|
|
||||||
import com.google.common.base.VerifyException;
|
import com.google.common.base.VerifyException;
|
||||||
import google.registry.model.BackupGroupRoot;
|
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
|
import google.registry.model.UpdateAutoTimestampEntity;
|
||||||
import google.registry.model.rde.RdeRevision.RdeRevisionId;
|
import google.registry.model.rde.RdeRevision.RdeRevisionId;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
import google.registry.persistence.converter.LocalDateConverter;
|
import google.registry.persistence.converter.LocalDateConverter;
|
||||||
|
@ -45,7 +45,7 @@ import org.joda.time.LocalDate;
|
||||||
*/
|
*/
|
||||||
@Entity
|
@Entity
|
||||||
@IdClass(RdeRevisionId.class)
|
@IdClass(RdeRevisionId.class)
|
||||||
public final class RdeRevision extends BackupGroupRoot {
|
public final class RdeRevision extends UpdateAutoTimestampEntity {
|
||||||
|
|
||||||
@Id String tld;
|
@Id String tld;
|
||||||
|
|
||||||
|
|
|
@ -54,11 +54,10 @@ import com.google.common.collect.Streams;
|
||||||
import com.google.re2j.Pattern;
|
import com.google.re2j.Pattern;
|
||||||
import google.registry.model.Buildable;
|
import google.registry.model.Buildable;
|
||||||
import google.registry.model.CreateAutoTimestamp;
|
import google.registry.model.CreateAutoTimestamp;
|
||||||
import google.registry.model.ImmutableObject;
|
|
||||||
import google.registry.model.JsonMapBuilder;
|
import google.registry.model.JsonMapBuilder;
|
||||||
import google.registry.model.Jsonifiable;
|
import google.registry.model.Jsonifiable;
|
||||||
import google.registry.model.UnsafeSerializable;
|
|
||||||
import google.registry.model.UpdateAutoTimestamp;
|
import google.registry.model.UpdateAutoTimestamp;
|
||||||
|
import google.registry.model.UpdateAutoTimestampEntity;
|
||||||
import google.registry.model.tld.Registry;
|
import google.registry.model.tld.Registry;
|
||||||
import google.registry.model.tld.Registry.TldType;
|
import google.registry.model.tld.Registry.TldType;
|
||||||
import google.registry.persistence.VKey;
|
import google.registry.persistence.VKey;
|
||||||
|
@ -95,8 +94,10 @@ import org.joda.time.DateTime;
|
||||||
@Index(columnList = "registrarName", name = "registrar_name_idx"),
|
@Index(columnList = "registrarName", name = "registrar_name_idx"),
|
||||||
@Index(columnList = "ianaIdentifier", name = "registrar_iana_identifier_idx"),
|
@Index(columnList = "ianaIdentifier", name = "registrar_iana_identifier_idx"),
|
||||||
})
|
})
|
||||||
public class Registrar extends ImmutableObject
|
@AttributeOverride(
|
||||||
implements Buildable, Jsonifiable, UnsafeSerializable {
|
name = "updateTimestamp.lastUpdateTime",
|
||||||
|
column = @Column(nullable = false, name = "lastUpdateTime"))
|
||||||
|
public class Registrar extends UpdateAutoTimestampEntity implements Buildable, Jsonifiable {
|
||||||
|
|
||||||
/** Represents the type of a registrar entity. */
|
/** Represents the type of a registrar entity. */
|
||||||
public enum Type {
|
public enum Type {
|
||||||
|
@ -378,9 +379,6 @@ public class Registrar extends ImmutableObject
|
||||||
/** The time when this registrar was created. */
|
/** The time when this registrar was created. */
|
||||||
CreateAutoTimestamp creationTime = CreateAutoTimestamp.create(null);
|
CreateAutoTimestamp creationTime = CreateAutoTimestamp.create(null);
|
||||||
|
|
||||||
/** An automatically managed last-saved timestamp. */
|
|
||||||
UpdateAutoTimestamp lastUpdateTime = UpdateAutoTimestamp.create(null);
|
|
||||||
|
|
||||||
/** The time that the certificate was last updated. */
|
/** The time that the certificate was last updated. */
|
||||||
DateTime lastCertificateUpdateTime;
|
DateTime lastCertificateUpdateTime;
|
||||||
|
|
||||||
|
@ -428,7 +426,7 @@ public class Registrar extends ImmutableObject
|
||||||
}
|
}
|
||||||
|
|
||||||
public DateTime getLastUpdateTime() {
|
public DateTime getLastUpdateTime() {
|
||||||
return lastUpdateTime.getTimestamp();
|
return getUpdateTimestamp().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
public DateTime getLastCertificateUpdateTime() {
|
public DateTime getLastCertificateUpdateTime() {
|
||||||
|
@ -595,7 +593,7 @@ public class Registrar extends ImmutableObject
|
||||||
.put("registrarId", registrarId)
|
.put("registrarId", registrarId)
|
||||||
.put("ianaIdentifier", ianaIdentifier)
|
.put("ianaIdentifier", ianaIdentifier)
|
||||||
.putString("creationTime", creationTime.getTimestamp())
|
.putString("creationTime", creationTime.getTimestamp())
|
||||||
.putString("lastUpdateTime", lastUpdateTime.getTimestamp())
|
.putString("lastUpdateTime", getUpdateTimestamp().getTimestamp())
|
||||||
.putString("lastCertificateUpdateTime", lastCertificateUpdateTime)
|
.putString("lastCertificateUpdateTime", lastCertificateUpdateTime)
|
||||||
.putString("lastExpiringCertNotificationSentDate", lastExpiringCertNotificationSentDate)
|
.putString("lastExpiringCertNotificationSentDate", lastExpiringCertNotificationSentDate)
|
||||||
.putString(
|
.putString(
|
||||||
|
@ -886,7 +884,7 @@ public class Registrar extends ImmutableObject
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public Builder setLastUpdateTime(DateTime timestamp) {
|
public Builder setLastUpdateTime(DateTime timestamp) {
|
||||||
getInstance().lastUpdateTime = UpdateAutoTimestamp.create(timestamp);
|
getInstance().setUpdateTimestamp(UpdateAutoTimestamp.create(timestamp));
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -16,14 +16,13 @@ package google.registry.persistence;
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkArgument;
|
import static com.google.common.base.Preconditions.checkArgument;
|
||||||
import static com.google.common.base.Preconditions.checkState;
|
import static com.google.common.base.Preconditions.checkState;
|
||||||
import static google.registry.model.ImmutableObject.Insignificant;
|
|
||||||
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
||||||
|
|
||||||
import com.google.common.base.Splitter;
|
import com.google.common.base.Splitter;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.googlecode.objectify.Key;
|
import com.googlecode.objectify.Key;
|
||||||
import google.registry.model.BackupGroupRoot;
|
|
||||||
import google.registry.model.ImmutableObject;
|
import google.registry.model.ImmutableObject;
|
||||||
|
import google.registry.model.UpdateAutoTimestampEntity;
|
||||||
import google.registry.model.common.ClassPathManager;
|
import google.registry.model.common.ClassPathManager;
|
||||||
import google.registry.model.translators.VKeyTranslatorFactory;
|
import google.registry.model.translators.VKeyTranslatorFactory;
|
||||||
import google.registry.util.SerializeUtils;
|
import google.registry.util.SerializeUtils;
|
||||||
|
@ -104,9 +103,9 @@ public class VKey<T> extends ImmutableObject implements Serializable {
|
||||||
*/
|
*/
|
||||||
public static <T> VKey<T> create(Class<T> kind, long id) {
|
public static <T> VKey<T> create(Class<T> kind, long id) {
|
||||||
checkArgument(
|
checkArgument(
|
||||||
BackupGroupRoot.class.isAssignableFrom(kind),
|
UpdateAutoTimestampEntity.class.isAssignableFrom(kind),
|
||||||
"The kind %s is not a BackupGroupRoot and thus needs its entire entity group chain"
|
"The kind %s is not a UpdateAutoTimestampEntity and thus needs its entire entity group"
|
||||||
+ " specified in a parent",
|
+ " chain specified in a parent",
|
||||||
kind.getCanonicalName());
|
kind.getCanonicalName());
|
||||||
return new VKey<T>(kind, Key.create(kind, id), id);
|
return new VKey<T>(kind, Key.create(kind, id), id);
|
||||||
}
|
}
|
||||||
|
@ -122,9 +121,9 @@ public class VKey<T> extends ImmutableObject implements Serializable {
|
||||||
*/
|
*/
|
||||||
public static <T> VKey<T> create(Class<T> kind, String name) {
|
public static <T> VKey<T> create(Class<T> kind, String name) {
|
||||||
checkArgument(
|
checkArgument(
|
||||||
BackupGroupRoot.class.isAssignableFrom(kind),
|
UpdateAutoTimestampEntity.class.isAssignableFrom(kind),
|
||||||
"The kind %s is not a BackupGroupRoot and thus needs its entire entity group chain"
|
"The kind %s is not a UpdateAutoTimestampEntity and thus needs its entire entity group"
|
||||||
+ " specified in a parent",
|
+ " chain specified in a parent",
|
||||||
kind.getCanonicalName());
|
kind.getCanonicalName());
|
||||||
return new VKey<T>(kind, Key.create(kind, name), name);
|
return new VKey<T>(kind, Key.create(kind, name), name);
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,169 +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;
|
|
||||||
|
|
||||||
import static com.google.common.collect.Ordering.arbitrary;
|
|
||||||
import static google.registry.model.EntityClasses.ALL_CLASSES;
|
|
||||||
import static java.lang.ClassLoader.getSystemClassLoader;
|
|
||||||
import static java.lang.reflect.Modifier.isAbstract;
|
|
||||||
|
|
||||||
import com.beust.jcommander.Parameters;
|
|
||||||
import com.google.common.base.Strings;
|
|
||||||
import com.google.common.collect.Multimap;
|
|
||||||
import com.google.common.collect.Multimaps;
|
|
||||||
import com.google.common.collect.Ordering;
|
|
||||||
import com.google.common.collect.TreeMultimap;
|
|
||||||
import com.googlecode.objectify.annotation.Entity;
|
|
||||||
import com.googlecode.objectify.annotation.EntitySubclass;
|
|
||||||
import com.googlecode.objectify.annotation.Parent;
|
|
||||||
import google.registry.model.BackupGroupRoot;
|
|
||||||
import google.registry.model.annotations.DeleteAfterMigration;
|
|
||||||
import google.registry.model.annotations.NotBackedUp;
|
|
||||||
import google.registry.model.annotations.VirtualEntity;
|
|
||||||
import java.io.Serializable;
|
|
||||||
import java.lang.reflect.Field;
|
|
||||||
import java.lang.reflect.ParameterizedType;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
/** Visualizes the schema parentage tree. */
|
|
||||||
@DeleteAfterMigration
|
|
||||||
@Parameters(commandDescription = "Generate a model schema file")
|
|
||||||
final class GetSchemaTreeCommand implements Command {
|
|
||||||
|
|
||||||
/** Mapping from parent classes in the Datastore sense to child classes. */
|
|
||||||
private final Multimap<Class<?>, Class<?>> hierarchy =
|
|
||||||
TreeMultimap.create(arbitrary(), new PrintableNameOrdering());
|
|
||||||
|
|
||||||
/** Mapping from superclasses used in parentage to concrete subclasses. */
|
|
||||||
private Multimap<Class<?>, Class<?>> superclassToSubclasses;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
// Get the @Parent type for each class.
|
|
||||||
Map<Class<?>, Class<?>> entityToParentType = new HashMap<>();
|
|
||||||
for (Class<?> clazz : ALL_CLASSES) {
|
|
||||||
entityToParentType.put(clazz, getParentType(clazz));
|
|
||||||
}
|
|
||||||
// Find super types like EppResource that are used as parents in place of actual entity types.
|
|
||||||
Set<Class<?>> superclasses = new HashSet<>();
|
|
||||||
for (Class<?> clazz : ALL_CLASSES) {
|
|
||||||
Class<?> parentType = entityToParentType.get(clazz);
|
|
||||||
if (!ALL_CLASSES.contains(parentType) && !Object.class.equals(parentType)) {
|
|
||||||
superclasses.add(parentType);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Find the subclasses for each superclass we just found, and map them to their superclasses.
|
|
||||||
Map<Class<?>, Class<?>> subclassToSuperclass = new HashMap<>();
|
|
||||||
for (Class<?> clazz : ALL_CLASSES) {
|
|
||||||
for (Class<?> superclass : superclasses) {
|
|
||||||
if (superclass.isAssignableFrom(clazz)) {
|
|
||||||
subclassToSuperclass.put(clazz, superclass);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Map @EntitySubclass classes to their superclasses.
|
|
||||||
for (Class<?> clazz : ALL_CLASSES) {
|
|
||||||
if (clazz.isAnnotationPresent(EntitySubclass.class)) {
|
|
||||||
Class<?> entityClass = clazz;
|
|
||||||
while (!entityClass.isAnnotationPresent(Entity.class)) {
|
|
||||||
entityClass = entityClass.getSuperclass();
|
|
||||||
}
|
|
||||||
if (subclassToSuperclass.containsKey(clazz)) {
|
|
||||||
subclassToSuperclass.put(entityClass, subclassToSuperclass.get(clazz));
|
|
||||||
}
|
|
||||||
subclassToSuperclass.put(clazz, entityClass);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Build the parentage hierarchy, replacing subclasses with superclasses wherever possible.
|
|
||||||
for (Class<?> clazz : ALL_CLASSES) {
|
|
||||||
Class<?> superclass = clazz;
|
|
||||||
while (subclassToSuperclass.containsKey(superclass)) {
|
|
||||||
superclass = subclassToSuperclass.get(superclass);
|
|
||||||
}
|
|
||||||
hierarchy.put(entityToParentType.get(clazz), superclass == null ? clazz : superclass);
|
|
||||||
}
|
|
||||||
// Build up the superclass to subclass mapping.
|
|
||||||
superclassToSubclasses = Multimaps.invertFrom(
|
|
||||||
Multimaps.forMap(subclassToSuperclass),
|
|
||||||
TreeMultimap.create(arbitrary(), new PrintableNameOrdering()));
|
|
||||||
printTree(Object.class, 0);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Class<?> getParentType(Class<?> clazz) {
|
|
||||||
for (; clazz != null; clazz = clazz.getSuperclass()) {
|
|
||||||
for (Field field : clazz.getDeclaredFields()) {
|
|
||||||
if (field.isAnnotationPresent(Parent.class)) {
|
|
||||||
try {
|
|
||||||
return getSystemClassLoader().loadClass(
|
|
||||||
((ParameterizedType) field.getGenericType()).getActualTypeArguments()[0]
|
|
||||||
.toString()
|
|
||||||
.replace("? extends ", "")
|
|
||||||
.replace("class ", ""));
|
|
||||||
} catch (ClassNotFoundException e) {
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return Object.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void printTree(Class<?> parent, int indent) {
|
|
||||||
for (Class<?> clazz : hierarchy.get(parent)) {
|
|
||||||
System.out.println(new StringBuilder(Strings.repeat(" ", indent))
|
|
||||||
.append(indent == 0 ? "" : "↳ ")
|
|
||||||
.append(getPrintableName(clazz))
|
|
||||||
.append(isAbstract(clazz.getModifiers()) ? " (abstract)" : "")
|
|
||||||
.append(clazz.isAnnotationPresent(VirtualEntity.class) ? " (virtual)" : "")
|
|
||||||
.append(clazz.isAnnotationPresent(NotBackedUp.class) ? " (not backed up)" : "")
|
|
||||||
.append(BackupGroupRoot.class.isAssignableFrom(clazz) ? " (bgr)" : ""));
|
|
||||||
printSubclasses(clazz, indent + 2);
|
|
||||||
printTree(clazz, indent + 2);
|
|
||||||
if (indent == 0) {
|
|
||||||
System.out.println(); // Separate the entity groups with a line.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void printSubclasses(Class<?> parent, int indent) {
|
|
||||||
for (Class<?> clazz : superclassToSubclasses.get(parent)) {
|
|
||||||
System.out.println(new StringBuilder(Strings.repeat(" ", indent))
|
|
||||||
.append("- ")
|
|
||||||
.append(getPrintableName(clazz))
|
|
||||||
.append(clazz.isAnnotationPresent(EntitySubclass.class) ? " (subclass)" : ""));
|
|
||||||
printSubclasses(clazz, indent + 2);
|
|
||||||
printTree(clazz, indent + 2);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Returns the simple name of the class prefixed with its wrapper's simple name, if any. */
|
|
||||||
static String getPrintableName(Class<?> clazz) {
|
|
||||||
return clazz.isMemberClass()
|
|
||||||
? getPrintableName(clazz.getDeclaringClass()) + "." + clazz.getSimpleName()
|
|
||||||
: clazz.getSimpleName();
|
|
||||||
}
|
|
||||||
|
|
||||||
/** An ordering that sorts on {@link #getPrintableName}. */
|
|
||||||
static class PrintableNameOrdering extends Ordering<Class<?>> implements Serializable {
|
|
||||||
@Override
|
|
||||||
public int compare(Class<?> left, Class<?> right) {
|
|
||||||
return getPrintableName(left).compareTo(getPrintableName(right));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -79,7 +79,6 @@ public final class RegistryTool {
|
||||||
.put("get_reserved_list", GetReservedListCommand.class)
|
.put("get_reserved_list", GetReservedListCommand.class)
|
||||||
.put("get_routing_map", GetRoutingMapCommand.class)
|
.put("get_routing_map", GetRoutingMapCommand.class)
|
||||||
.put("get_schema", GetSchemaCommand.class)
|
.put("get_schema", GetSchemaCommand.class)
|
||||||
.put("get_schema_tree", GetSchemaTreeCommand.class)
|
|
||||||
.put("get_sql_credential", GetSqlCredentialCommand.class)
|
.put("get_sql_credential", GetSqlCredentialCommand.class)
|
||||||
.put("get_tld", GetTldCommand.class)
|
.put("get_tld", GetTldCommand.class)
|
||||||
.put("ghostryde", GhostrydeCommand.class)
|
.put("ghostryde", GhostrydeCommand.class)
|
||||||
|
|
|
@ -30,6 +30,7 @@ import google.registry.persistence.transaction.TransactionManagerFactory;
|
||||||
import google.registry.testing.DatabaseHelper;
|
import google.registry.testing.DatabaseHelper;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
import javax.persistence.Persistence;
|
import javax.persistence.Persistence;
|
||||||
|
import org.apache.beam.sdk.coders.SerializableCoder;
|
||||||
import org.apache.beam.sdk.testing.PAssert;
|
import org.apache.beam.sdk.testing.PAssert;
|
||||||
import org.apache.beam.sdk.values.PCollection;
|
import org.apache.beam.sdk.values.PCollection;
|
||||||
import org.hibernate.cfg.Environment;
|
import org.hibernate.cfg.Environment;
|
||||||
|
@ -161,6 +162,7 @@ public class DatabaseSnapshotTest {
|
||||||
|
|
||||||
Read<Registry, Registry> read =
|
Read<Registry, Registry> read =
|
||||||
RegistryJpaIO.read(() -> CriteriaQueryBuilder.create(Registry.class).build(), x -> x)
|
RegistryJpaIO.read(() -> CriteriaQueryBuilder.create(Registry.class).build(), x -> x)
|
||||||
|
.withCoder(SerializableCoder.of(Registry.class))
|
||||||
.withSnapshot(databaseSnapshot.getSnapshotId());
|
.withSnapshot(databaseSnapshot.getSnapshotId());
|
||||||
PCollection<Registry> registries = testPipeline.apply(read);
|
PCollection<Registry> registries = testPipeline.apply(read);
|
||||||
|
|
||||||
|
|
|
@ -45,6 +45,7 @@ import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationT
|
||||||
import google.registry.testing.AppEngineExtension;
|
import google.registry.testing.AppEngineExtension;
|
||||||
import google.registry.testing.DatastoreEntityExtension;
|
import google.registry.testing.DatastoreEntityExtension;
|
||||||
import google.registry.testing.FakeClock;
|
import google.registry.testing.FakeClock;
|
||||||
|
import org.apache.beam.sdk.coders.StringUtf8Coder;
|
||||||
import org.apache.beam.sdk.testing.PAssert;
|
import org.apache.beam.sdk.testing.PAssert;
|
||||||
import org.apache.beam.sdk.values.PCollection;
|
import org.apache.beam.sdk.values.PCollection;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -97,7 +98,8 @@ public class RegistryJpaReadTest {
|
||||||
void readWithCriteriaQuery() {
|
void readWithCriteriaQuery() {
|
||||||
Read<Contact, String> read =
|
Read<Contact, String> read =
|
||||||
RegistryJpaIO.read(
|
RegistryJpaIO.read(
|
||||||
() -> CriteriaQueryBuilder.create(Contact.class).build(), ContactBase::getContactId);
|
() -> CriteriaQueryBuilder.create(Contact.class).build(), ContactBase::getContactId)
|
||||||
|
.withCoder(StringUtf8Coder.of());
|
||||||
PCollection<String> repoIds = testPipeline.apply(read);
|
PCollection<String> repoIds = testPipeline.apply(read);
|
||||||
|
|
||||||
PAssert.that(repoIds).containsInAnyOrder("contact_0", "contact_1", "contact_2");
|
PAssert.that(repoIds).containsInAnyOrder("contact_0", "contact_1", "contact_2");
|
||||||
|
@ -118,7 +120,8 @@ public class RegistryJpaReadTest {
|
||||||
Domain domain = (Domain) row[0];
|
Domain domain = (Domain) row[0];
|
||||||
String emailAddress = (String) row[1];
|
String emailAddress = (String) row[1];
|
||||||
return domain.getRepoId() + "-" + emailAddress;
|
return domain.getRepoId() + "-" + emailAddress;
|
||||||
});
|
})
|
||||||
|
.withCoder(StringUtf8Coder.of());
|
||||||
PCollection<String> joinedStrings = testPipeline.apply(read);
|
PCollection<String> joinedStrings = testPipeline.apply(read);
|
||||||
|
|
||||||
PAssert.that(joinedStrings).containsInAnyOrder("4-COM-me@google.com");
|
PAssert.that(joinedStrings).containsInAnyOrder("4-COM-me@google.com");
|
||||||
|
@ -139,7 +142,8 @@ public class RegistryJpaReadTest {
|
||||||
String repoId = (String) row[0];
|
String repoId = (String) row[0];
|
||||||
String emailAddress = (String) row[1];
|
String emailAddress = (String) row[1];
|
||||||
return repoId + "-" + emailAddress;
|
return repoId + "-" + emailAddress;
|
||||||
});
|
})
|
||||||
|
.withCoder(StringUtf8Coder.of());
|
||||||
PCollection<String> joinedStrings = testPipeline.apply(read);
|
PCollection<String> joinedStrings = testPipeline.apply(read);
|
||||||
|
|
||||||
PAssert.that(joinedStrings).containsInAnyOrder("4-COM-me@google.com");
|
PAssert.that(joinedStrings).containsInAnyOrder("4-COM-me@google.com");
|
||||||
|
@ -156,7 +160,8 @@ public class RegistryJpaReadTest {
|
||||||
+ " and d.deletionTime > now()",
|
+ " and d.deletionTime > now()",
|
||||||
ImmutableMap.of("type", Registrar.Type.REAL),
|
ImmutableMap.of("type", Registrar.Type.REAL),
|
||||||
Domain.class,
|
Domain.class,
|
||||||
Domain::getRepoId);
|
Domain::getRepoId)
|
||||||
|
.withCoder(StringUtf8Coder.of());
|
||||||
PCollection<String> repoIds = testPipeline.apply(read);
|
PCollection<String> repoIds = testPipeline.apply(read);
|
||||||
|
|
||||||
PAssert.that(repoIds).containsInAnyOrder("4-COM");
|
PAssert.that(repoIds).containsInAnyOrder("4-COM");
|
||||||
|
|
|
@ -68,32 +68,6 @@ public class UpdateAutoTimestampTest {
|
||||||
assertThat(reload().updateTime.getTimestamp()).isEqualTo(transactionTime);
|
assertThat(reload().updateTime.getTimestamp()).isEqualTo(transactionTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
void testDisabledUpdates() throws Exception {
|
|
||||||
DateTime initialTime =
|
|
||||||
tm().transact(
|
|
||||||
() -> {
|
|
||||||
clock.advanceOneMilli();
|
|
||||||
tm().insert(new UpdateAutoTimestampTestObject());
|
|
||||||
return tm().getTransactionTime();
|
|
||||||
});
|
|
||||||
|
|
||||||
UpdateAutoTimestampTestObject object = reload();
|
|
||||||
clock.advanceOneMilli();
|
|
||||||
|
|
||||||
try (UpdateAutoTimestamp.DisableAutoUpdateResource ignoredDisabler =
|
|
||||||
new UpdateAutoTimestamp.DisableAutoUpdateResource()) {
|
|
||||||
DateTime secondTransactionTime =
|
|
||||||
tm().transact(
|
|
||||||
() -> {
|
|
||||||
tm().put(object);
|
|
||||||
return tm().getTransactionTime();
|
|
||||||
});
|
|
||||||
assertThat(secondTransactionTime).isGreaterThan(initialTime);
|
|
||||||
}
|
|
||||||
assertThat(reload().updateTime.getTimestamp()).isEqualTo(initialTime);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void testResavingOverwritesOriginalTime() {
|
void testResavingOverwritesOriginalTime() {
|
||||||
DateTime transactionTime =
|
DateTime transactionTime =
|
||||||
|
|
|
@ -57,7 +57,7 @@ class VKeyTest {
|
||||||
void testCreateById_failsWhenParentIsNullButShouldntBe() {
|
void testCreateById_failsWhenParentIsNullButShouldntBe() {
|
||||||
IllegalArgumentException thrown =
|
IllegalArgumentException thrown =
|
||||||
assertThrows(IllegalArgumentException.class, () -> VKey.create(OneTime.class, 134L));
|
assertThrows(IllegalArgumentException.class, () -> VKey.create(OneTime.class, 134L));
|
||||||
assertThat(thrown).hasMessageThat().contains("BackupGroupRoot");
|
assertThat(thrown).hasMessageThat().contains("UpdateAutoTimestampEntity");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -66,7 +66,7 @@ class VKeyTest {
|
||||||
assertThrows(
|
assertThrows(
|
||||||
IllegalArgumentException.class,
|
IllegalArgumentException.class,
|
||||||
() -> VKey.create(RegistrarPoc.class, "fake@example.com"));
|
() -> VKey.create(RegistrarPoc.class, "fake@example.com"));
|
||||||
assertThat(thrown).hasMessageThat().contains("BackupGroupRoot");
|
assertThat(thrown).hasMessageThat().contains("UpdateAutoTimestampEntity");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -1,41 +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;
|
|
||||||
|
|
||||||
import static com.google.common.truth.Truth.assertWithMessage;
|
|
||||||
|
|
||||||
import com.google.re2j.Matcher;
|
|
||||||
import com.google.re2j.Pattern;
|
|
||||||
import google.registry.model.EntityClasses;
|
|
||||||
import org.junit.jupiter.api.Test;
|
|
||||||
|
|
||||||
/** Unit tests for {@link GetSchemaTreeCommand}. */
|
|
||||||
class GetSchemaTreeCommandTest extends CommandTestCase<GetSchemaTreeCommand> {
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void testAllClassesPrintedExactlyOnce() throws Exception {
|
|
||||||
runCommand();
|
|
||||||
String stdout = getStdoutAsString();
|
|
||||||
for (Class<?> clazz : EntityClasses.ALL_CLASSES) {
|
|
||||||
String printableName = GetSchemaTreeCommand.getPrintableName(clazz);
|
|
||||||
int count = 0;
|
|
||||||
Matcher matcher = Pattern.compile("(^|\\s)" + printableName + "\\s").matcher(stdout);
|
|
||||||
while (matcher.find()) {
|
|
||||||
count++;
|
|
||||||
}
|
|
||||||
assertWithMessage(printableName + " occurences").that(count).isEqualTo(1);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -289,7 +289,7 @@ class RegistrarSettingsActionTest extends RegistrarSettingsActionTestCase {
|
||||||
assertAboutImmutableObjects()
|
assertAboutImmutableObjects()
|
||||||
.that(updatedRegistrar)
|
.that(updatedRegistrar)
|
||||||
.isEqualExceptFields(
|
.isEqualExceptFields(
|
||||||
setter.apply(registrar.asBuilder(), newValue).build(), "lastUpdateTime");
|
setter.apply(registrar.asBuilder(), newValue).build(), "updateTimestamp");
|
||||||
// We increased the correct metric
|
// We increased the correct metric
|
||||||
assertMetric(CLIENT_ID, "update", String.format("[%s]", role), "SUCCESS");
|
assertMetric(CLIENT_ID, "update", String.format("[%s]", role), "SUCCESS");
|
||||||
}
|
}
|
||||||
|
|
|
@ -61,7 +61,7 @@ class SecuritySettingsTest extends RegistrarSettingsActionTestCase {
|
||||||
assertThat(response).containsEntry("results", ImmutableList.of(modifiedJsonMap));
|
assertThat(response).containsEntry("results", ImmutableList.of(modifiedJsonMap));
|
||||||
assertAboutImmutableObjects()
|
assertAboutImmutableObjects()
|
||||||
.that(loadRegistrar(CLIENT_ID))
|
.that(loadRegistrar(CLIENT_ID))
|
||||||
.isEqualExceptFields(modified, "lastUpdateTime");
|
.isEqualExceptFields(modified, "updateTimestamp");
|
||||||
assertMetric(CLIENT_ID, "update", "[OWNER]", "SUCCESS");
|
assertMetric(CLIENT_ID, "update", "[OWNER]", "SUCCESS");
|
||||||
verifyNotificationEmailsSent();
|
verifyNotificationEmailsSent();
|
||||||
}
|
}
|
||||||
|
|
|
@ -234,8 +234,8 @@
|
||||||
create table "Cursor" (
|
create table "Cursor" (
|
||||||
scope text not null,
|
scope text not null,
|
||||||
type text not null,
|
type text not null,
|
||||||
cursor_time timestamptz not null,
|
|
||||||
last_update_time timestamptz not null,
|
last_update_time timestamptz not null,
|
||||||
|
cursor_time timestamptz not null,
|
||||||
primary key (scope, type)
|
primary key (scope, type)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -561,6 +561,7 @@
|
||||||
|
|
||||||
create table "Registrar" (
|
create table "Registrar" (
|
||||||
registrar_id text not null,
|
registrar_id text not null,
|
||||||
|
last_update_time timestamptz not null,
|
||||||
allowed_tlds text[],
|
allowed_tlds text[],
|
||||||
billing_account_map hstore,
|
billing_account_map hstore,
|
||||||
block_premium_names boolean not null,
|
block_premium_names boolean not null,
|
||||||
|
@ -586,7 +587,6 @@
|
||||||
last_certificate_update_time timestamptz,
|
last_certificate_update_time timestamptz,
|
||||||
last_expiring_cert_notification_sent_date timestamptz,
|
last_expiring_cert_notification_sent_date timestamptz,
|
||||||
last_expiring_failover_cert_notification_sent_date timestamptz,
|
last_expiring_failover_cert_notification_sent_date timestamptz,
|
||||||
last_update_time timestamptz not null,
|
|
||||||
localized_address_city text,
|
localized_address_city text,
|
||||||
localized_address_country_code text,
|
localized_address_country_code text,
|
||||||
localized_address_state text,
|
localized_address_state text,
|
||||||
|
@ -629,9 +629,9 @@
|
||||||
|
|
||||||
create table "RegistryLock" (
|
create table "RegistryLock" (
|
||||||
revision_id bigserial not null,
|
revision_id bigserial not null,
|
||||||
|
last_update_time timestamptz not null,
|
||||||
domain_name text not null,
|
domain_name text not null,
|
||||||
is_superuser boolean not null,
|
is_superuser boolean not null,
|
||||||
last_update_time timestamptz not null,
|
|
||||||
lock_completion_time timestamptz,
|
lock_completion_time timestamptz,
|
||||||
lock_request_time timestamptz not null,
|
lock_request_time timestamptz not null,
|
||||||
registrar_id text not null,
|
registrar_id text not null,
|
||||||
|
|
Loading…
Add table
Reference in a new issue