diff --git a/core/src/main/java/google/registry/beam/common/RegistryJpaIO.java b/core/src/main/java/google/registry/beam/common/RegistryJpaIO.java index bde73fb89..2ad78f4da 100644 --- a/core/src/main/java/google/registry/beam/common/RegistryJpaIO.java +++ b/core/src/main/java/google/registry/beam/common/RegistryJpaIO.java @@ -22,18 +22,14 @@ import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableList; import com.google.common.collect.Streams; 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.TransactionManagerFactory; -import java.io.Serializable; import java.util.Map; import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; import javax.persistence.criteria.CriteriaQuery; 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.Metrics; import org.apache.beam.sdk.transforms.Create; @@ -136,6 +132,7 @@ public final class RegistryJpaIO { abstract SerializableFunction resultMapper(); + @Nullable abstract Coder coder(); @Nullable @@ -146,13 +143,16 @@ public final class RegistryJpaIO { @Override @SuppressWarnings("deprecation") // Reshuffle still recommended by GCP. public PCollection expand(PBegin input) { - return input - .apply("Starting " + name(), Create.of((Void) null)) - .apply( - "Run query for " + name(), - ParDo.of(new QueryRunner<>(query(), resultMapper(), snapshotId()))) - .setCoder(coder()) - .apply("Reshuffle", Reshuffle.viaRandomKey()); + PCollection output = + input + .apply("Starting " + name(), Create.of((Void) null)) + .apply( + "Run query for " + name(), + ParDo.of(new QueryRunner<>(query(), resultMapper(), snapshotId()))); + if (coder() != null) { + output = output.setCoder(coder()); + } + return output.apply("Reshuffle", Reshuffle.viaRandomKey()); } public Read withName(String name) { @@ -180,9 +180,7 @@ public final class RegistryJpaIO { } static Builder builder() { - return new AutoValue_RegistryJpaIO_Read.Builder() - .name(DEFAULT_NAME) - .coder(SerializableCoder.of(Serializable.class)); + return new AutoValue_RegistryJpaIO_Read.Builder().name(DEFAULT_NAME); } @AutoValue.Builder @@ -194,7 +192,7 @@ public final class RegistryJpaIO { abstract Builder resultMapper(SerializableFunction mapper); - abstract Builder coder(Coder coder); + abstract Builder coder(Coder coder); abstract Builder snapshotId(@Nullable String sharedSnapshotId); @@ -299,12 +297,6 @@ public final class RegistryJpaIO { public abstract SerializableFunction 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 withName(String name) { return toBuilder().name(name).build(); } @@ -325,10 +317,6 @@ public final class RegistryJpaIO { return toBuilder().jpaConverter(jpaConverter).build(); } - public Write disableUpdateAutoTimestamp() { - return toBuilder().withUpdateAutoTimestamp(false).build(); - } - abstract Builder toBuilder(); @Override @@ -345,7 +333,7 @@ public final class RegistryJpaIO { GroupIntoBatches.ofSize(batchSize()).withShardedKey()) .apply( "Write in batch for " + name(), - ParDo.of(new SqlBatchWriter<>(name(), jpaConverter(), withUpdateAutoTimestamp()))); + ParDo.of(new SqlBatchWriter<>(name(), jpaConverter()))); } static Builder builder() { @@ -353,8 +341,7 @@ public final class RegistryJpaIO { .name(DEFAULT_NAME) .batchSize(DEFAULT_BATCH_SIZE) .shards(DEFAULT_SHARDS) - .jpaConverter(x -> x) - .withUpdateAutoTimestamp(true); + .jpaConverter(x -> x); } @AutoValue.Builder @@ -368,8 +355,6 @@ public final class RegistryJpaIO { abstract Builder jpaConverter(SerializableFunction jpaConverter); - abstract Builder withUpdateAutoTimestamp(boolean withUpdateAutoTimestamp); - abstract Write build(); } } @@ -378,24 +363,15 @@ public final class RegistryJpaIO { private static class SqlBatchWriter extends DoFn, Iterable>, Void> { private final Counter counter; private final SerializableFunction jpaConverter; - private final boolean withAutoTimestamp; - SqlBatchWriter( - String type, SerializableFunction jpaConverter, boolean withAutoTimestamp) { + SqlBatchWriter(String type, SerializableFunction jpaConverter) { counter = Metrics.counter("SQL_WRITE", type); this.jpaConverter = jpaConverter; - this.withAutoTimestamp = withAutoTimestamp; } @ProcessElement public void processElement(@Element KV, Iterable> kv) { - if (withAutoTimestamp) { - actuallyProcessElement(kv); - return; - } - try (DisableAutoUpdateResource disable = UpdateAutoTimestamp.disableAutoUpdate()) { - actuallyProcessElement(kv); - } + actuallyProcessElement(kv); } private void actuallyProcessElement(@Element KV, Iterable> kv) { diff --git a/core/src/main/java/google/registry/beam/invoicing/InvoicingPipeline.java b/core/src/main/java/google/registry/beam/invoicing/InvoicingPipeline.java index 0cc90b8c4..b6af2d885 100644 --- a/core/src/main/java/google/registry/beam/invoicing/InvoicingPipeline.java +++ b/core/src/main/java/google/registry/beam/invoicing/InvoicingPipeline.java @@ -37,6 +37,7 @@ import java.util.Optional; import java.util.regex.Pattern; import org.apache.beam.sdk.Pipeline; 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.io.FileIO; import org.apache.beam.sdk.io.TextIO; @@ -93,8 +94,9 @@ public class InvoicingPipeline implements Serializable { static PCollection readFromCloudSql( InvoicingPipelineOptions options, Pipeline pipeline) { Read read = - RegistryJpaIO.read( - makeCloudSqlQuery(options.getYearMonth()), false, row -> parseRow(row).orElse(null)); + RegistryJpaIO.read( + makeCloudSqlQuery(options.getYearMonth()), false, row -> parseRow(row).orElse(null)) + .withCoder(SerializableCoder.of(BillingEvent.class)); PCollection billingEventsWithNulls = pipeline.apply("Read BillingEvents from Cloud SQL", read); diff --git a/core/src/main/java/google/registry/beam/rde/RdePipeline.java b/core/src/main/java/google/registry/beam/rde/RdePipeline.java index 07a4b06a6..14d879bab 100644 --- a/core/src/main/java/google/registry/beam/rde/RdePipeline.java +++ b/core/src/main/java/google/registry/beam/rde/RdePipeline.java @@ -301,10 +301,11 @@ public class RdePipeline implements Serializable { .apply( "Read all production Registrars", RegistryJpaIO.read( - "SELECT registrarId FROM Registrar WHERE type NOT IN (:types)", - ImmutableMap.of("types", IGNORED_REGISTRAR_TYPES), - String.class, - id -> VKey.createSql(Registrar.class, id))) + "SELECT registrarId FROM Registrar WHERE type NOT IN (:types)", + ImmutableMap.of("types", IGNORED_REGISTRAR_TYPES), + String.class, + x -> x) + .withCoder(StringUtf8Coder.of())) .apply( "Marshall Registrar into DepositFragment", FlatMapElements.into( @@ -312,7 +313,8 @@ public class RdePipeline implements Serializable { TypeDescriptor.of(PendingDeposit.class), TypeDescriptor.of(DepositFragment.class))) .via( - (VKey key) -> { + (String registrarRepoId) -> { + VKey key = VKey.createSql(Registrar.class, registrarRepoId); includedRegistrarCounter.inc(); Registrar registrar = jpaTm().transact(() -> jpaTm().loadByKey(key)); DepositFragment fragment = marshaller.marshalRegistrar(registrar); @@ -337,10 +339,9 @@ public class RdePipeline implements Serializable { Pipeline pipeline, Class historyClass) { String repoIdFieldName = HistoryEntryDao.REPO_ID_FIELD_NAMES.get(historyClass); String resourceFieldName = EPP_RESOURCE_FIELD_NAME.get(historyClass); - return pipeline - .apply( - String.format("Load most recent %s", historyClass.getSimpleName()), - RegistryJpaIO.read( + return pipeline.apply( + String.format("Load most recent %s", historyClass.getSimpleName()), + RegistryJpaIO.read( ("SELECT %repoIdField%, id FROM %entity% WHERE (%repoIdField%, modificationTime)" + " IN (SELECT %repoIdField%, MAX(modificationTime) FROM %entity% WHERE" + " modificationTime <= :watermark GROUP BY %repoIdField%) AND" @@ -358,8 +359,8 @@ public class RdePipeline implements Serializable { .replace("%resourceField%", resourceFieldName), ImmutableMap.of("watermark", watermark), Object[].class, - row -> KV.of((String) row[0], (long) row[1]))) - .setCoder(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())); + row -> KV.of((String) row[0], (long) row[1])) + .withCoder(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))); } private EppResource loadResourceByHistoryEntryId( diff --git a/core/src/main/java/google/registry/beam/resave/ResaveAllEppResourcesPipeline.java b/core/src/main/java/google/registry/beam/resave/ResaveAllEppResourcesPipeline.java index 361410768..2f44aec28 100644 --- a/core/src/main/java/google/registry/beam/resave/ResaveAllEppResourcesPipeline.java +++ b/core/src/main/java/google/registry/beam/resave/ResaveAllEppResourcesPipeline.java @@ -36,6 +36,7 @@ import java.io.Serializable; import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.Pipeline; 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.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupIntoBatches; @@ -103,10 +104,11 @@ public class ResaveAllEppResourcesPipeline implements Serializable { private void fastResaveContacts(Pipeline pipeline) { Read repoIdRead = RegistryJpaIO.read( - "SELECT repoId FROM Contact WHERE transferData.transferStatus = 'PENDING' AND" - + " transferData.pendingTransferExpirationTime < current_timestamp()", - String.class, - r -> r); + "SELECT repoId FROM Contact WHERE transferData.transferStatus = 'PENDING' AND" + + " transferData.pendingTransferExpirationTime < current_timestamp()", + String.class, + r -> r) + .withCoder(StringUtf8Coder.of()); projectAndResaveResources(pipeline, Contact.class, repoIdRead); } @@ -120,10 +122,11 @@ public class ResaveAllEppResourcesPipeline implements Serializable { private void fastResaveDomains(Pipeline pipeline) { Read repoIdRead = RegistryJpaIO.read( - DOMAINS_TO_PROJECT_QUERY, - ImmutableMap.of("END_OF_TIME", DateTimeUtils.END_OF_TIME), - String.class, - r -> r); + DOMAINS_TO_PROJECT_QUERY, + ImmutableMap.of("END_OF_TIME", DateTimeUtils.END_OF_TIME), + String.class, + r -> r) + .withCoder(StringUtf8Coder.of()); projectAndResaveResources(pipeline, Domain.class, repoIdRead); } @@ -131,8 +134,9 @@ public class ResaveAllEppResourcesPipeline implements Serializable { private void forceResaveAllResources(Pipeline pipeline, Class clazz) { Read repoIdRead = RegistryJpaIO.read( - // Note: cannot use SQL parameters for the table name - String.format("SELECT repoId FROM %s", clazz.getSimpleName()), String.class, r -> r); + // Note: cannot use SQL parameters for the table name + String.format("SELECT repoId FROM %s", clazz.getSimpleName()), String.class, r -> r) + .withCoder(StringUtf8Coder.of()); projectAndResaveResources(pipeline, clazz, repoIdRead); } diff --git a/core/src/main/java/google/registry/beam/spec11/Spec11Pipeline.java b/core/src/main/java/google/registry/beam/spec11/Spec11Pipeline.java index e478f551e..f9aafaa9c 100644 --- a/core/src/main/java/google/registry/beam/spec11/Spec11Pipeline.java +++ b/core/src/main/java/google/registry/beam/spec11/Spec11Pipeline.java @@ -37,6 +37,8 @@ import java.io.Serializable; import javax.inject.Singleton; import org.apache.beam.sdk.Pipeline; 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.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; @@ -112,11 +114,12 @@ public class Spec11Pipeline implements Serializable { static PCollection readFromCloudSql(Pipeline pipeline) { Read> read = RegistryJpaIO.read( - "select d.repoId, r.emailAddress from Domain d join Registrar r on" - + " d.currentSponsorClientId = r.registrarId where r.type = 'REAL' and" - + " d.deletionTime > now()", - false, - Spec11Pipeline::parseRow); + "select d.repoId, r.emailAddress from Domain d join Registrar r on" + + " d.currentSponsorClientId = r.registrarId where r.type = 'REAL' and" + + " d.deletionTime > now()", + false, + Spec11Pipeline::parseRow) + .withCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); return pipeline .apply("Read active domains from Cloud SQL", read) @@ -214,8 +217,7 @@ public class Spec11Pipeline implements Serializable { return output.toString(); } catch (JSONException e) { throw new RuntimeException( - String.format( - "Encountered an error constructing the JSON for %s", kv.toString()), + String.format("Encountered an error constructing the JSON for %s", kv), e); } })) diff --git a/core/src/main/java/google/registry/model/EppResource.java b/core/src/main/java/google/registry/model/EppResource.java index add9624ed..0696e1ef6 100644 --- a/core/src/main/java/google/registry/model/EppResource.java +++ b/core/src/main/java/google/registry/model/EppResource.java @@ -56,7 +56,7 @@ import org.joda.time.DateTime; /** An EPP entity object (i.e. a domain, contact, or host). */ @MappedSuperclass @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; @@ -339,7 +339,7 @@ public abstract class EppResource extends BackupGroupRoot implements Buildable { /** * Set the update timestamp. * - *

This is provided at EppResource since BackupGroupRoot doesn't have a Builder. + *

This is provided at EppResource since UpdateAutoTimestampEntity doesn't have a Builder. */ public B setUpdateTimestamp(UpdateAutoTimestamp updateTimestamp) { getInstance().setUpdateTimestamp(updateTimestamp); diff --git a/core/src/main/java/google/registry/model/UpdateAutoTimestamp.java b/core/src/main/java/google/registry/model/UpdateAutoTimestamp.java index cd376fc04..67d2e2280 100644 --- a/core/src/main/java/google/registry/model/UpdateAutoTimestamp.java +++ b/core/src/main/java/google/registry/model/UpdateAutoTimestamp.java @@ -29,12 +29,7 @@ import org.joda.time.DateTime; @Embeddable public class UpdateAutoTimestamp extends ImmutableObject implements UnsafeSerializable { - // When set to true, database converters/translators should do the auto update. When set to - // false, auto update should be suspended (this exists to allow us to preserve the original value - // during a replay). - private static final ThreadLocal autoUpdateEnabled = ThreadLocal.withInitial(() -> true); - - @Column(nullable = false) + @Column(name = "updateTimestamp") DateTime lastUpdateTime; // Unfortunately, we cannot use the @UpdateTimestamp annotation on "lastUpdateTime" in this class @@ -43,9 +38,7 @@ public class UpdateAutoTimestamp extends ImmutableObject implements UnsafeSerial @PrePersist @PreUpdate void setTimestamp() { - if (autoUpdateEnabled() || lastUpdateTime == null) { - lastUpdateTime = jpaTm().getTransactionTime(); - } + lastUpdateTime = jpaTm().getTransactionTime(); } /** Returns the timestamp, or {@code START_OF_TIME} if it's null. */ @@ -58,30 +51,4 @@ public class UpdateAutoTimestamp extends ImmutableObject implements UnsafeSerial instance.lastUpdateTime = timestamp; 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(); - } } diff --git a/core/src/main/java/google/registry/model/BackupGroupRoot.java b/core/src/main/java/google/registry/model/UpdateAutoTimestampEntity.java similarity index 79% rename from core/src/main/java/google/registry/model/BackupGroupRoot.java rename to core/src/main/java/google/registry/model/UpdateAutoTimestampEntity.java index c7212b539..af54ed679 100644 --- a/core/src/main/java/google/registry/model/BackupGroupRoot.java +++ b/core/src/main/java/google/registry/model/UpdateAutoTimestampEntity.java @@ -18,21 +18,16 @@ import com.googlecode.objectify.annotation.Ignore; import google.registry.util.PreconditionsUtils; import javax.persistence.Access; import javax.persistence.AccessType; -import javax.persistence.AttributeOverride; -import javax.persistence.Column; import javax.persistence.MappedSuperclass; 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 - * commit logs for backup purposes. - * - *

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. + * Base class for entities that contains an {@link UpdateAutoTimestamp} which is updated every time + * the entity is persisted. */ @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. @@ -44,7 +39,6 @@ public abstract class BackupGroupRoot extends ImmutableObject implements UnsafeS // Prevents subclasses from unexpectedly accessing as property (e.g., Host), which would // require an unnecessary non-private setter method. @Access(AccessType.FIELD) - @AttributeOverride(name = "lastUpdateTime", column = @Column(name = "updateTimestamp")) @Ignore UpdateAutoTimestamp updateTimestamp = UpdateAutoTimestamp.create(null); @@ -59,7 +53,7 @@ public abstract class BackupGroupRoot extends ImmutableObject implements UnsafeS *

This method is for the few cases when {@code updateTimestamp} is copied between different * 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; } diff --git a/core/src/main/java/google/registry/model/common/Cursor.java b/core/src/main/java/google/registry/model/common/Cursor.java index a050590cd..a79a496bd 100644 --- a/core/src/main/java/google/registry/model/common/Cursor.java +++ b/core/src/main/java/google/registry/model/common/Cursor.java @@ -20,11 +20,12 @@ import static google.registry.util.DateTimeUtils.START_OF_TIME; import google.registry.model.ImmutableObject; 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.tld.Registry; import google.registry.persistence.VKey; import java.util.Optional; +import javax.persistence.AttributeOverride; import javax.persistence.Column; import javax.persistence.Entity; import javax.persistence.EnumType; @@ -41,7 +42,10 @@ import org.joda.time.DateTime; */ @Entity @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; @@ -122,10 +126,6 @@ public class Cursor extends ImmutableObject implements UnsafeSerializable { @Column(nullable = false) 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 public VKey createVKey() { return createVKey(type, scope); @@ -145,9 +145,10 @@ public class Cursor extends ImmutableObject implements UnsafeSerializable { } public DateTime getLastUpdateTime() { - return lastUpdateTime.getTimestamp(); + return getUpdateTimestamp().getTimestamp(); } + public String getScope() { return scope; } diff --git a/core/src/main/java/google/registry/model/console/User.java b/core/src/main/java/google/registry/model/console/User.java index 11977307b..a9685c242 100644 --- a/core/src/main/java/google/registry/model/console/User.java +++ b/core/src/main/java/google/registry/model/console/User.java @@ -22,8 +22,8 @@ import static google.registry.util.PasswordUtils.SALT_SUPPLIER; import static google.registry.util.PasswordUtils.hashPassword; import static google.registry.util.PreconditionsUtils.checkArgumentNotNull; -import google.registry.model.BackupGroupRoot; import google.registry.model.Buildable; +import google.registry.model.UpdateAutoTimestampEntity; import javax.persistence.Column; import javax.persistence.Entity; import javax.persistence.GeneratedValue; @@ -39,7 +39,7 @@ import javax.persistence.Table; @Index(columnList = "gaiaId", name = "user_gaia_id_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. */ @Id diff --git a/core/src/main/java/google/registry/model/domain/RegistryLock.java b/core/src/main/java/google/registry/model/domain/RegistryLock.java index 24933df20..46495a25b 100644 --- a/core/src/main/java/google/registry/model/domain/RegistryLock.java +++ b/core/src/main/java/google/registry/model/domain/RegistryLock.java @@ -20,10 +20,11 @@ import static google.registry.util.PreconditionsUtils.checkArgumentNotNull; import google.registry.model.Buildable; import google.registry.model.CreateAutoTimestamp; -import google.registry.model.ImmutableObject; -import google.registry.model.UpdateAutoTimestamp; +import google.registry.model.UpdateAutoTimestampEntity; import java.util.Optional; import javax.annotation.Nullable; +import javax.persistence.Access; +import javax.persistence.AccessType; import javax.persistence.AttributeOverride; import javax.persistence.AttributeOverrides; 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 * unlock actions. * - *

Note that there will be at most one row per domain with a null copmleted time -- this means + *

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. * *

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 @Table( - /** + /* * Unique constraint to get around Hibernate's failure to handle auto-increment field in * composite primary key. * - *

Note: indexes use the camelCase version of the field names because the {@link - * google.registry.persistence.NomulusNamingStrategy} does not translate the field name into the + * Note: indexes use the camelCase version of the field names because + * google.registry.persistence.NomulusNamingStrategy does not translate the field name into the * snake_case column name until the write itself. */ indexes = { @@ -72,7 +73,11 @@ import org.joda.time.Duration; @Index(name = "idx_registry_lock_verification_code", columnList = "verificationCode"), @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 @GeneratedValue(strategy = GenerationType.IDENTITY) @@ -103,7 +108,7 @@ public final class RegistryLock extends ImmutableObject implements Buildable { name = "creationTime", 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. */ 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. */ private Duration relockDuration; - /** Time that this entity was last updated. */ - private UpdateAutoTimestamp lastUpdateTime = UpdateAutoTimestamp.create(null); - public String getRepoId() { return repoId; } @@ -189,7 +191,7 @@ public final class RegistryLock extends ImmutableObject implements Buildable { } public DateTime getLastUpdateTime() { - return lastUpdateTime.getTimestamp(); + return getUpdateTimestamp().getTimestamp(); } 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. * - *

Note: this is lazily loaded, so it may not be initialized if referenced outside of the + *

Note: this is lazily loaded, so it may not be initialized if referenced outside the * transaction in which this lock is loaded. */ public RegistryLock getRelock() { diff --git a/core/src/main/java/google/registry/model/domain/token/AllocationToken.java b/core/src/main/java/google/registry/model/domain/token/AllocationToken.java index dc9c41e85..f57f8852f 100644 --- a/core/src/main/java/google/registry/model/domain/token/AllocationToken.java +++ b/core/src/main/java/google/registry/model/domain/token/AllocationToken.java @@ -33,9 +33,9 @@ import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.Range; import google.registry.flows.EppException; import google.registry.flows.domain.DomainFlowUtils; -import google.registry.model.BackupGroupRoot; import google.registry.model.Buildable; import google.registry.model.CreateAutoTimestamp; +import google.registry.model.UpdateAutoTimestampEntity; import google.registry.model.billing.BillingEvent.RenewalPriceBehavior; import google.registry.model.common.TimedTransitionProperty; import google.registry.model.reporting.HistoryEntry; @@ -66,7 +66,7 @@ import org.joda.time.DateTime; @Index(columnList = "tokenType"), @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 String REMOVE_PACKAGE = "__REMOVEPACKAGE__"; diff --git a/core/src/main/java/google/registry/model/ofy/TimestampInversionException.java b/core/src/main/java/google/registry/model/ofy/TimestampInversionException.java index 653cec8c3..6a7bf24f1 100644 --- a/core/src/main/java/google/registry/model/ofy/TimestampInversionException.java +++ b/core/src/main/java/google/registry/model/ofy/TimestampInversionException.java @@ -16,7 +16,7 @@ package google.registry.model.ofy; import com.googlecode.objectify.Key; import com.googlecode.objectify.Objectify; -import google.registry.model.BackupGroupRoot; +import google.registry.model.UpdateAutoTimestampEntity; import google.registry.model.annotations.DeleteAfterMigration; import java.util.Arrays; import java.util.Map; @@ -34,7 +34,7 @@ class TimestampInversionException extends RuntimeException { } TimestampInversionException( - DateTime transactionTime, Map, DateTime> problematicRoots) { + DateTime transactionTime, Map, DateTime> problematicRoots) { this(transactionTime, "entities rooted under:\n" + problematicRoots); } diff --git a/core/src/main/java/google/registry/model/rde/RdeRevision.java b/core/src/main/java/google/registry/model/rde/RdeRevision.java index aada6a8dc..fdb58dd61 100644 --- a/core/src/main/java/google/registry/model/rde/RdeRevision.java +++ b/core/src/main/java/google/registry/model/rde/RdeRevision.java @@ -19,8 +19,8 @@ import static google.registry.model.rde.RdeNamingUtils.makePartialName; import static google.registry.persistence.transaction.TransactionManagerFactory.tm; import com.google.common.base.VerifyException; -import google.registry.model.BackupGroupRoot; import google.registry.model.ImmutableObject; +import google.registry.model.UpdateAutoTimestampEntity; import google.registry.model.rde.RdeRevision.RdeRevisionId; import google.registry.persistence.VKey; import google.registry.persistence.converter.LocalDateConverter; @@ -45,7 +45,7 @@ import org.joda.time.LocalDate; */ @Entity @IdClass(RdeRevisionId.class) -public final class RdeRevision extends BackupGroupRoot { +public final class RdeRevision extends UpdateAutoTimestampEntity { @Id String tld; diff --git a/core/src/main/java/google/registry/model/registrar/Registrar.java b/core/src/main/java/google/registry/model/registrar/Registrar.java index 1d7659686..6432f9e09 100644 --- a/core/src/main/java/google/registry/model/registrar/Registrar.java +++ b/core/src/main/java/google/registry/model/registrar/Registrar.java @@ -54,11 +54,10 @@ import com.google.common.collect.Streams; import com.google.re2j.Pattern; import google.registry.model.Buildable; import google.registry.model.CreateAutoTimestamp; -import google.registry.model.ImmutableObject; import google.registry.model.JsonMapBuilder; import google.registry.model.Jsonifiable; -import google.registry.model.UnsafeSerializable; import google.registry.model.UpdateAutoTimestamp; +import google.registry.model.UpdateAutoTimestampEntity; import google.registry.model.tld.Registry; import google.registry.model.tld.Registry.TldType; import google.registry.persistence.VKey; @@ -95,8 +94,10 @@ import org.joda.time.DateTime; @Index(columnList = "registrarName", name = "registrar_name_idx"), @Index(columnList = "ianaIdentifier", name = "registrar_iana_identifier_idx"), }) -public class Registrar extends ImmutableObject - implements Buildable, Jsonifiable, UnsafeSerializable { +@AttributeOverride( + name = "updateTimestamp.lastUpdateTime", + column = @Column(nullable = false, name = "lastUpdateTime")) +public class Registrar extends UpdateAutoTimestampEntity implements Buildable, Jsonifiable { /** Represents the type of a registrar entity. */ public enum Type { @@ -378,9 +379,6 @@ public class Registrar extends ImmutableObject /** The time when this registrar was created. */ CreateAutoTimestamp creationTime = CreateAutoTimestamp.create(null); - /** An automatically managed last-saved timestamp. */ - UpdateAutoTimestamp lastUpdateTime = UpdateAutoTimestamp.create(null); - /** The time that the certificate was last updated. */ DateTime lastCertificateUpdateTime; @@ -428,7 +426,7 @@ public class Registrar extends ImmutableObject } public DateTime getLastUpdateTime() { - return lastUpdateTime.getTimestamp(); + return getUpdateTimestamp().getTimestamp(); } public DateTime getLastCertificateUpdateTime() { @@ -595,7 +593,7 @@ public class Registrar extends ImmutableObject .put("registrarId", registrarId) .put("ianaIdentifier", ianaIdentifier) .putString("creationTime", creationTime.getTimestamp()) - .putString("lastUpdateTime", lastUpdateTime.getTimestamp()) + .putString("lastUpdateTime", getUpdateTimestamp().getTimestamp()) .putString("lastCertificateUpdateTime", lastCertificateUpdateTime) .putString("lastExpiringCertNotificationSentDate", lastExpiringCertNotificationSentDate) .putString( @@ -886,7 +884,7 @@ public class Registrar extends ImmutableObject */ @VisibleForTesting public Builder setLastUpdateTime(DateTime timestamp) { - getInstance().lastUpdateTime = UpdateAutoTimestamp.create(timestamp); + getInstance().setUpdateTimestamp(UpdateAutoTimestamp.create(timestamp)); return this; } diff --git a/core/src/main/java/google/registry/persistence/VKey.java b/core/src/main/java/google/registry/persistence/VKey.java index 9983036da..bf5aecd40 100644 --- a/core/src/main/java/google/registry/persistence/VKey.java +++ b/core/src/main/java/google/registry/persistence/VKey.java @@ -16,14 +16,13 @@ package google.registry.persistence; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import static google.registry.model.ImmutableObject.Insignificant; import static google.registry.util.PreconditionsUtils.checkArgumentNotNull; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableMap; import com.googlecode.objectify.Key; -import google.registry.model.BackupGroupRoot; import google.registry.model.ImmutableObject; +import google.registry.model.UpdateAutoTimestampEntity; import google.registry.model.common.ClassPathManager; import google.registry.model.translators.VKeyTranslatorFactory; import google.registry.util.SerializeUtils; @@ -104,9 +103,9 @@ public class VKey extends ImmutableObject implements Serializable { */ public static VKey create(Class kind, long id) { checkArgument( - BackupGroupRoot.class.isAssignableFrom(kind), - "The kind %s is not a BackupGroupRoot and thus needs its entire entity group chain" - + " specified in a parent", + UpdateAutoTimestampEntity.class.isAssignableFrom(kind), + "The kind %s is not a UpdateAutoTimestampEntity and thus needs its entire entity group" + + " chain specified in a parent", kind.getCanonicalName()); return new VKey(kind, Key.create(kind, id), id); } @@ -122,9 +121,9 @@ public class VKey extends ImmutableObject implements Serializable { */ public static VKey create(Class kind, String name) { checkArgument( - BackupGroupRoot.class.isAssignableFrom(kind), - "The kind %s is not a BackupGroupRoot and thus needs its entire entity group chain" - + " specified in a parent", + UpdateAutoTimestampEntity.class.isAssignableFrom(kind), + "The kind %s is not a UpdateAutoTimestampEntity and thus needs its entire entity group" + + " chain specified in a parent", kind.getCanonicalName()); return new VKey(kind, Key.create(kind, name), name); } diff --git a/core/src/main/java/google/registry/tools/GetSchemaTreeCommand.java b/core/src/main/java/google/registry/tools/GetSchemaTreeCommand.java deleted file mode 100644 index fa3a9f581..000000000 --- a/core/src/main/java/google/registry/tools/GetSchemaTreeCommand.java +++ /dev/null @@ -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> hierarchy = - TreeMultimap.create(arbitrary(), new PrintableNameOrdering()); - - /** Mapping from superclasses used in parentage to concrete subclasses. */ - private Multimap, Class> superclassToSubclasses; - - @Override - public void run() { - // Get the @Parent type for each class. - Map, 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> 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> 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> implements Serializable { - @Override - public int compare(Class left, Class right) { - return getPrintableName(left).compareTo(getPrintableName(right)); - } - } -} diff --git a/core/src/main/java/google/registry/tools/RegistryTool.java b/core/src/main/java/google/registry/tools/RegistryTool.java index f6709e233..20f9a2bac 100644 --- a/core/src/main/java/google/registry/tools/RegistryTool.java +++ b/core/src/main/java/google/registry/tools/RegistryTool.java @@ -79,7 +79,6 @@ public final class RegistryTool { .put("get_reserved_list", GetReservedListCommand.class) .put("get_routing_map", GetRoutingMapCommand.class) .put("get_schema", GetSchemaCommand.class) - .put("get_schema_tree", GetSchemaTreeCommand.class) .put("get_sql_credential", GetSqlCredentialCommand.class) .put("get_tld", GetTldCommand.class) .put("ghostryde", GhostrydeCommand.class) diff --git a/core/src/test/java/google/registry/beam/common/DatabaseSnapshotTest.java b/core/src/test/java/google/registry/beam/common/DatabaseSnapshotTest.java index f287cc6c0..d8558948b 100644 --- a/core/src/test/java/google/registry/beam/common/DatabaseSnapshotTest.java +++ b/core/src/test/java/google/registry/beam/common/DatabaseSnapshotTest.java @@ -30,6 +30,7 @@ import google.registry.persistence.transaction.TransactionManagerFactory; import google.registry.testing.DatabaseHelper; import google.registry.testing.FakeClock; import javax.persistence.Persistence; +import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.values.PCollection; import org.hibernate.cfg.Environment; @@ -161,6 +162,7 @@ public class DatabaseSnapshotTest { Read read = RegistryJpaIO.read(() -> CriteriaQueryBuilder.create(Registry.class).build(), x -> x) + .withCoder(SerializableCoder.of(Registry.class)) .withSnapshot(databaseSnapshot.getSnapshotId()); PCollection registries = testPipeline.apply(read); diff --git a/core/src/test/java/google/registry/beam/common/RegistryJpaReadTest.java b/core/src/test/java/google/registry/beam/common/RegistryJpaReadTest.java index 0d5b822fc..ce215cd8f 100644 --- a/core/src/test/java/google/registry/beam/common/RegistryJpaReadTest.java +++ b/core/src/test/java/google/registry/beam/common/RegistryJpaReadTest.java @@ -45,6 +45,7 @@ import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationT import google.registry.testing.AppEngineExtension; import google.registry.testing.DatastoreEntityExtension; import google.registry.testing.FakeClock; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.values.PCollection; import org.joda.time.DateTime; @@ -97,7 +98,8 @@ public class RegistryJpaReadTest { void readWithCriteriaQuery() { Read read = RegistryJpaIO.read( - () -> CriteriaQueryBuilder.create(Contact.class).build(), ContactBase::getContactId); + () -> CriteriaQueryBuilder.create(Contact.class).build(), ContactBase::getContactId) + .withCoder(StringUtf8Coder.of()); PCollection repoIds = testPipeline.apply(read); PAssert.that(repoIds).containsInAnyOrder("contact_0", "contact_1", "contact_2"); @@ -109,16 +111,17 @@ public class RegistryJpaReadTest { setupForJoinQuery(); Read read = RegistryJpaIO.read( - "select d, r.emailAddress from Domain d join Registrar r on" - + " d.currentSponsorClientId = r.registrarId where r.type = :type" - + " and d.deletionTime > now()", - ImmutableMap.of("type", Registrar.Type.REAL), - false, - (Object[] row) -> { - Domain domain = (Domain) row[0]; - String emailAddress = (String) row[1]; - return domain.getRepoId() + "-" + emailAddress; - }); + "select d, r.emailAddress from Domain d join Registrar r on" + + " d.currentSponsorClientId = r.registrarId where r.type = :type" + + " and d.deletionTime > now()", + ImmutableMap.of("type", Registrar.Type.REAL), + false, + (Object[] row) -> { + Domain domain = (Domain) row[0]; + String emailAddress = (String) row[1]; + return domain.getRepoId() + "-" + emailAddress; + }) + .withCoder(StringUtf8Coder.of()); PCollection joinedStrings = testPipeline.apply(read); PAssert.that(joinedStrings).containsInAnyOrder("4-COM-me@google.com"); @@ -130,16 +133,17 @@ public class RegistryJpaReadTest { setupForJoinQuery(); Read read = RegistryJpaIO.read( - "select d.repo_id, r.email_address from \"Domain\" d join \"Registrar\" r on" - + " d.current_sponsor_registrar_id = r.registrar_id where r.type = :type" - + " and d.deletion_time > now()", - ImmutableMap.of("type", "REAL"), - true, - (Object[] row) -> { - String repoId = (String) row[0]; - String emailAddress = (String) row[1]; - return repoId + "-" + emailAddress; - }); + "select d.repo_id, r.email_address from \"Domain\" d join \"Registrar\" r on" + + " d.current_sponsor_registrar_id = r.registrar_id where r.type = :type" + + " and d.deletion_time > now()", + ImmutableMap.of("type", "REAL"), + true, + (Object[] row) -> { + String repoId = (String) row[0]; + String emailAddress = (String) row[1]; + return repoId + "-" + emailAddress; + }) + .withCoder(StringUtf8Coder.of()); PCollection joinedStrings = testPipeline.apply(read); PAssert.that(joinedStrings).containsInAnyOrder("4-COM-me@google.com"); @@ -151,12 +155,13 @@ public class RegistryJpaReadTest { setupForJoinQuery(); Read read = RegistryJpaIO.read( - "select d from Domain d join Registrar r on" - + " d.currentSponsorClientId = r.registrarId where r.type = :type" - + " and d.deletionTime > now()", - ImmutableMap.of("type", Registrar.Type.REAL), - Domain.class, - Domain::getRepoId); + "select d from Domain d join Registrar r on" + + " d.currentSponsorClientId = r.registrarId where r.type = :type" + + " and d.deletionTime > now()", + ImmutableMap.of("type", Registrar.Type.REAL), + Domain.class, + Domain::getRepoId) + .withCoder(StringUtf8Coder.of()); PCollection repoIds = testPipeline.apply(read); PAssert.that(repoIds).containsInAnyOrder("4-COM"); diff --git a/core/src/test/java/google/registry/model/UpdateAutoTimestampTest.java b/core/src/test/java/google/registry/model/UpdateAutoTimestampTest.java index c34e63bc2..e9f9a29d4 100644 --- a/core/src/test/java/google/registry/model/UpdateAutoTimestampTest.java +++ b/core/src/test/java/google/registry/model/UpdateAutoTimestampTest.java @@ -68,32 +68,6 @@ public class UpdateAutoTimestampTest { 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 void testResavingOverwritesOriginalTime() { DateTime transactionTime = diff --git a/core/src/test/java/google/registry/persistence/VKeyTest.java b/core/src/test/java/google/registry/persistence/VKeyTest.java index 5a04c0392..66c35f876 100644 --- a/core/src/test/java/google/registry/persistence/VKeyTest.java +++ b/core/src/test/java/google/registry/persistence/VKeyTest.java @@ -57,7 +57,7 @@ class VKeyTest { void testCreateById_failsWhenParentIsNullButShouldntBe() { IllegalArgumentException thrown = assertThrows(IllegalArgumentException.class, () -> VKey.create(OneTime.class, 134L)); - assertThat(thrown).hasMessageThat().contains("BackupGroupRoot"); + assertThat(thrown).hasMessageThat().contains("UpdateAutoTimestampEntity"); } @Test @@ -66,7 +66,7 @@ class VKeyTest { assertThrows( IllegalArgumentException.class, () -> VKey.create(RegistrarPoc.class, "fake@example.com")); - assertThat(thrown).hasMessageThat().contains("BackupGroupRoot"); + assertThat(thrown).hasMessageThat().contains("UpdateAutoTimestampEntity"); } @Test diff --git a/core/src/test/java/google/registry/tools/GetSchemaTreeCommandTest.java b/core/src/test/java/google/registry/tools/GetSchemaTreeCommandTest.java deleted file mode 100644 index 639b7ce85..000000000 --- a/core/src/test/java/google/registry/tools/GetSchemaTreeCommandTest.java +++ /dev/null @@ -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 { - - @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); - } - } -} diff --git a/core/src/test/java/google/registry/ui/server/registrar/RegistrarSettingsActionTest.java b/core/src/test/java/google/registry/ui/server/registrar/RegistrarSettingsActionTest.java index 0e96db60b..38bfce740 100644 --- a/core/src/test/java/google/registry/ui/server/registrar/RegistrarSettingsActionTest.java +++ b/core/src/test/java/google/registry/ui/server/registrar/RegistrarSettingsActionTest.java @@ -289,7 +289,7 @@ class RegistrarSettingsActionTest extends RegistrarSettingsActionTestCase { assertAboutImmutableObjects() .that(updatedRegistrar) .isEqualExceptFields( - setter.apply(registrar.asBuilder(), newValue).build(), "lastUpdateTime"); + setter.apply(registrar.asBuilder(), newValue).build(), "updateTimestamp"); // We increased the correct metric assertMetric(CLIENT_ID, "update", String.format("[%s]", role), "SUCCESS"); } diff --git a/core/src/test/java/google/registry/ui/server/registrar/SecuritySettingsTest.java b/core/src/test/java/google/registry/ui/server/registrar/SecuritySettingsTest.java index b6bfcea22..7695e15b8 100644 --- a/core/src/test/java/google/registry/ui/server/registrar/SecuritySettingsTest.java +++ b/core/src/test/java/google/registry/ui/server/registrar/SecuritySettingsTest.java @@ -61,7 +61,7 @@ class SecuritySettingsTest extends RegistrarSettingsActionTestCase { assertThat(response).containsEntry("results", ImmutableList.of(modifiedJsonMap)); assertAboutImmutableObjects() .that(loadRegistrar(CLIENT_ID)) - .isEqualExceptFields(modified, "lastUpdateTime"); + .isEqualExceptFields(modified, "updateTimestamp"); assertMetric(CLIENT_ID, "update", "[OWNER]", "SUCCESS"); verifyNotificationEmailsSent(); } diff --git a/db/src/main/resources/sql/schema/db-schema.sql.generated b/db/src/main/resources/sql/schema/db-schema.sql.generated index 34bbe0906..c5c77f816 100644 --- a/db/src/main/resources/sql/schema/db-schema.sql.generated +++ b/db/src/main/resources/sql/schema/db-schema.sql.generated @@ -234,8 +234,8 @@ create table "Cursor" ( scope text not null, type text not null, - cursor_time timestamptz not null, last_update_time timestamptz not null, + cursor_time timestamptz not null, primary key (scope, type) ); @@ -561,6 +561,7 @@ create table "Registrar" ( registrar_id text not null, + last_update_time timestamptz not null, allowed_tlds text[], billing_account_map hstore, block_premium_names boolean not null, @@ -586,7 +587,6 @@ last_certificate_update_time timestamptz, last_expiring_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_country_code text, localized_address_state text, @@ -629,9 +629,9 @@ create table "RegistryLock" ( revision_id bigserial not null, + last_update_time timestamptz not null, domain_name text not null, is_superuser boolean not null, - last_update_time timestamptz not null, lock_completion_time timestamptz, lock_request_time timestamptz not null, registrar_id text not null,