diff --git a/.scalafmt.conf b/.scalafmt.conf index d123d197..ee87d5ed 100644 --- a/.scalafmt.conf +++ b/.scalafmt.conf @@ -1,56 +1,9 @@ -version = 3.0.6 +version = 3.5.8 -style = defaultWithAlign +runner.dialect = scala213 -docstrings.style = Asterisk -docstrings.wrap = no -indentOperator.preset = spray -maxColumn = 120 -rewrite.rules = [RedundantParens, SortImports, AvoidInfix] -unindentTopLevelOperators = true -align.tokens = [{code = "=>", owner = "Case"}] -align.openParenDefnSite = false -align.openParenCallSite = false -optIn.breakChainOnFirstMethodDot = false -optIn.configStyleArguments = false -danglingParentheses.defnSite = false -danglingParentheses.callSite = false -spaces.inImportCurlyBraces = true -rewrite.neverInfix.excludeFilters = [ - and - min - max - until - to - by - eq - ne - "should.*" - "contain.*" - "must.*" - in - ignore - be - taggedAs - thrownBy - synchronized - have - when - size - only - noneOf - oneElementOf - noElementsOf - atLeastOneElementOf - atMostOneElementOf - allElementsOf - inOrderElementsOf - theSameElementsAs -] -rewriteTokens = { - "⇒": "=>" - "→": "->" - "←": "<-" -} -newlines.afterCurlyLambda = preserve -newlines.implicitParamListModifierPrefer = before +maxColumn = 120 + +rewrite.rules = [SortModifiers, Imports] +rewrite.imports.sort = scalastyle +rewrite.imports.groups = [["(?!javax?\\.).*", "(?!scala\\.).*"], ["javax?\\..*", "scala\\..*"]] diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/config/PekkoPersistenceConfig.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/config/PekkoPersistenceConfig.scala index 29bfa29f..b00e9b8e 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/config/PekkoPersistenceConfig.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/config/PekkoPersistenceConfig.scala @@ -5,8 +5,8 @@ package org.apache.pekko.persistence.postgres.config -import org.apache.pekko.persistence.postgres.util.ConfigOps._ import com.typesafe.config.Config +import org.apache.pekko.persistence.postgres.util.ConfigOps._ import scala.concurrent.duration._ @@ -169,14 +169,16 @@ object JournalSequenceRetrievalConfig { maxTries = config.asInt("journal-sequence-retrieval.max-tries", 10), queryDelay = config.asFiniteDuration("journal-sequence-retrieval.query-delay", 1.second), maxBackoffQueryDelay = config.asFiniteDuration("journal-sequence-retrieval.max-backoff-query-delay", 1.minute), - askTimeout = config.asFiniteDuration("journal-sequence-retrieval.ask-timeout", 1.second)) + askTimeout = config.asFiniteDuration("journal-sequence-retrieval.ask-timeout", 1.second) + ) } case class JournalSequenceRetrievalConfig( batchSize: Int, maxTries: Int, queryDelay: FiniteDuration, maxBackoffQueryDelay: FiniteDuration, - askTimeout: FiniteDuration) + askTimeout: FiniteDuration +) class ReadJournalConfig(config: Config) { val journalTableConfiguration = new JournalTableConfiguration(config) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/db/DbErrors.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/db/DbErrors.scala index 94011020..70c32e9d 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/db/DbErrors.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/db/DbErrors.scala @@ -1,11 +1,10 @@ package org.apache.pekko.persistence.postgres.db -import java.sql.SQLException - import org.slf4j.Logger +import java.sql.SQLException import scala.concurrent.ExecutionContext -import scala.util.{ Failure, Success } +import scala.util.{Failure, Success} object DbErrors { @@ -14,8 +13,9 @@ object DbErrors { val PgDuplicateTable: String = "42P07" val PgUniqueViolation: String = "23505" - def withHandledPartitionErrors(logger: Logger, partitionDetails: String)(dbio: DBIOAction[_, NoStream, Effect])( - implicit ec: ExecutionContext): DBIOAction[Unit, NoStream, Effect] = + def withHandledPartitionErrors(logger: Logger, partitionDetails: String)( + dbio: DBIOAction[_, NoStream, Effect] + )(implicit ec: ExecutionContext): DBIOAction[Unit, NoStream, Effect] = dbio.asTry.flatMap { case Failure(ex: SQLException) if ex.getSQLState == PgDuplicateTable => logger.debug(s"Partition for $partitionDetails already exists") @@ -28,8 +28,9 @@ object DbErrors { DBIO.successful(()) } - def withHandledIndexErrors(logger: Logger, indexDetails: String)(dbio: DBIOAction[_, NoStream, Effect])( - implicit ec: ExecutionContext): DBIOAction[Unit, NoStream, Effect] = + def withHandledIndexErrors(logger: Logger, indexDetails: String)( + dbio: DBIOAction[_, NoStream, Effect] + )(implicit ec: ExecutionContext): DBIOAction[Unit, NoStream, Effect] = dbio.asTry.flatMap { case Failure(ex: SQLException) if ex.getSQLState == PgUniqueViolation => logger.debug(s"Index $indexDetails already exists") diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/db/SlickDatabase.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/db/SlickDatabase.scala index e353a495..b5fa6154 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/db/SlickDatabase.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/db/SlickDatabase.scala @@ -5,42 +5,38 @@ package org.apache.pekko.persistence.postgres.db +import com.typesafe.config.Config import org.apache.pekko.actor.ActorSystem -import javax.naming.InitialContext import org.apache.pekko.persistence.postgres.config.SlickConfiguration -import com.typesafe.config.Config import slick.basic.DatabaseConfig -import slick.jdbc.JdbcProfile import slick.jdbc.JdbcBackend._ +import slick.jdbc.JdbcProfile -/** - * INTERNAL API - */ +import javax.naming.InitialContext + +/** INTERNAL API + */ @deprecated(message = "Internal API, will be removed in 4.0.0", since = "3.4.0") object SlickDriver { - /** - * INTERNAL API - */ + /** INTERNAL API + */ @deprecated(message = "Internal API, will be removed in 4.0.0", since = "3.4.0") def forDriverName(config: Config): JdbcProfile = SlickDatabase.profile(config, "slick") } -/** - * INTERNAL API - */ +/** INTERNAL API + */ object SlickDatabase { - /** - * INTERNAL API - */ + /** INTERNAL API + */ private[postgres] def profile(config: Config, path: String): JdbcProfile = DatabaseConfig.forConfig[JdbcProfile](path, config).profile - /** - * INTERNAL API - */ + /** INTERNAL API + */ private[postgres] def database(config: Config, slickConfiguration: SlickConfiguration, path: String): Database = { slickConfiguration.jndiName .map(Database.forName(_, None)) @@ -50,13 +46,13 @@ object SlickDatabase { .getOrElse(Database.forConfig(path, config)) } - /** - * INTERNAL API - */ + /** INTERNAL API + */ private[postgres] def initializeEagerly( config: Config, slickConfiguration: SlickConfiguration, - path: String): SlickDatabase = { + path: String + ): SlickDatabase = { val dbPath = if (path.isEmpty) "db" else s"$path.db" EagerSlickDatabase(database(config, slickConfiguration, dbPath), profile(config, path)) } @@ -66,11 +62,9 @@ trait SlickDatabase { def database: Database def profile: JdbcProfile - /** - * If true, the requesting side usually a (read/write/snapshot journal) - * should shutdown the database when it closes. If false, it should leave - * the database connection pool open, since it might still be used elsewhere. - */ + /** If true, the requesting side usually a (read/write/snapshot journal) should shutdown the database when it closes. + * If false, it should leave the database connection pool open, since it might still be used elsewhere. + */ def allowShutdown: Boolean } @@ -78,10 +72,10 @@ case class EagerSlickDatabase(database: Database, profile: JdbcProfile) extends override def allowShutdown: Boolean = true } -/** - * A LazySlickDatabase lazily initializes a database, it also manages the shutdown of the database - * @param config The configuration used to create the database - */ +/** A LazySlickDatabase lazily initializes a database, it also manages the shutdown of the database + * @param config + * The configuration used to create the database + */ class LazySlickDatabase(config: Config, system: ActorSystem) extends SlickDatabase { val profile: JdbcProfile = SlickDatabase.profile(config, path = "") diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/db/SlickExtension.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/db/SlickExtension.scala index 1efa6f52..90f7da21 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/db/SlickExtension.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/db/SlickExtension.scala @@ -5,13 +5,13 @@ package org.apache.pekko.persistence.postgres.db -import org.apache.pekko.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider } -import org.apache.pekko.persistence.postgres.config.{ ConfigKeys, SlickConfiguration } +import com.typesafe.config.{Config, ConfigObject} +import org.apache.pekko.actor.{ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider} +import org.apache.pekko.persistence.postgres.config.{ConfigKeys, SlickConfiguration} import org.apache.pekko.persistence.postgres.util.ConfigOps._ -import com.typesafe.config.{ Config, ConfigObject } import scala.jdk.CollectionConverters._ -import scala.util.{ Failure, Success } +import scala.util.{Failure, Success} object SlickExtension extends ExtensionId[SlickExtensionImpl] with ExtensionIdProvider { override def lookup: SlickExtension.type = SlickExtension @@ -32,23 +32,20 @@ class SlickExtensionImpl(system: ExtendedActorSystem) extends Extension { def database(config: Config): SlickDatabase = dbProvider.database(config) } -/** - * User overridable database provider. - * Since this provider is called from an pekko extension it must be thread safe! - * - * A SlickDatabaseProvider is loaded using reflection, - * The instance is created using the following: - * - The fully qualified class name as configured in `postgres-journal.database-provider-fqcn`. - * - The constructor with one argument of type [[org.apache.pekko.actor.ActorSystem]] is used to create the instance. - * Therefore the class must have such a constructor. - */ +/** User overridable database provider. Since this provider is called from an pekko extension it must be thread safe! + * + * A SlickDatabaseProvider is loaded using reflection, The instance is created using the following: + * - The fully qualified class name as configured in `postgres-journal.database-provider-fqcn`. + * - The constructor with one argument of type [[org.apache.pekko.actor.ActorSystem]] is used to create the instance. + * Therefore the class must have such a constructor. + */ trait SlickDatabaseProvider { - /** - * Create or retrieve the database - * @param config The configuration which may be used to create the database. If the database is shared - * then the SlickDatabaseProvider implementation may choose to ignore this parameter. - */ + /** Create or retrieve the database + * @param config + * The configuration which may be used to create the database. If the database is shared then the + * SlickDatabaseProvider implementation may choose to ignore this parameter. + */ def database(config: Config): SlickDatabase } @@ -66,7 +63,8 @@ class DefaultSlickDatabaseProvider(system: ActorSystem) extends SlickDatabasePro case (key, notAnObject) => throw new RuntimeException( s"""Expected "pekko-persistence-postgres.shared-databases.$key" to be a config ConfigObject, but got ${notAnObject - .valueType()} (${notAnObject.getClass})""") + .valueType()} (${notAnObject.getClass})""" + ) } .toMap @@ -74,7 +72,9 @@ class DefaultSlickDatabaseProvider(system: ActorSystem) extends SlickDatabasePro sharedDatabases.getOrElse( sharedDbName, throw new RuntimeException( - s"No shared database is configured under pekko-persistence-postgres.shared-databases.$sharedDbName")) + s"No shared database is configured under pekko-persistence-postgres.shared-databases.$sharedDbName" + ) + ) def database(config: Config): SlickDatabase = { config.asOptionalNonEmptyString(ConfigKeys.useSharedDb) match { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/PostgresAsyncWriteJournal.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/PostgresAsyncWriteJournal.scala index a58ae54a..fe0ce6d7 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/PostgresAsyncWriteJournal.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/PostgresAsyncWriteJournal.scala @@ -5,36 +5,34 @@ package org.apache.pekko.persistence.postgres.journal -import java.util.{ HashMap => JHMap, Map => JMap } - +import com.typesafe.config.Config import org.apache.pekko.Done -import org.apache.pekko.actor.{ ActorSystem, ExtendedActorSystem } +import org.apache.pekko.actor.{ActorSystem, ExtendedActorSystem} import org.apache.pekko.pattern.pipe -import org.apache.pekko.persistence.postgres.config.JournalConfig -import org.apache.pekko.persistence.postgres.db.{ SlickDatabase, SlickExtension } -import org.apache.pekko.persistence.postgres.journal.PostgresAsyncWriteJournal.{ InPlaceUpdateEvent, WriteFinished } -import org.apache.pekko.persistence.postgres.journal.dao.{ JournalDao, JournalDaoWithUpdates } +import org.apache.pekko.persistence.{AtomicWrite, PersistentRepr} import org.apache.pekko.persistence.journal.AsyncWriteJournal -import org.apache.pekko.persistence.{ AtomicWrite, PersistentRepr } -import org.apache.pekko.serialization.{ Serialization, SerializationExtension } -import org.apache.pekko.stream.{ Materializer, SystemMaterializer } -import com.typesafe.config.Config +import org.apache.pekko.persistence.postgres.config.JournalConfig +import org.apache.pekko.persistence.postgres.db.{SlickDatabase, SlickExtension} +import org.apache.pekko.persistence.postgres.journal.PostgresAsyncWriteJournal.{InPlaceUpdateEvent, WriteFinished} +import org.apache.pekko.persistence.postgres.journal.dao.{JournalDao, JournalDaoWithUpdates} +import org.apache.pekko.serialization.{Serialization, SerializationExtension} +import org.apache.pekko.stream.{Materializer, SystemMaterializer} import slick.jdbc.JdbcBackend._ +import java.util.{HashMap => JHMap, Map => JMap} import scala.collection.immutable._ -import scala.concurrent.{ ExecutionContext, Future } -import scala.util.{ Failure, Success, Try } +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success, Try} object PostgresAsyncWriteJournal { private case class WriteFinished(pid: String, f: Future[_]) - /** - * Extra Plugin API: May be used to issue in-place updates for events. - * To be used only for data migrations such as "encrypt all events" and similar operations. - * - * The write payload may be wrapped in a [[org.apache.pekko.persistence.journal.Tagged]], - * in which case the new tags will be skipped and the old tags remain unchanged. - */ + /** Extra Plugin API: May be used to issue in-place updates for events. To be used only for data migrations such as + * "encrypt all events" and similar operations. + * + * The write payload may be wrapped in a [[org.apache.pekko.persistence.journal.Tagged]], in which case the new tags + * will be skipped and the old tags remain unchanged. + */ final case class InPlaceUpdateEvent(persistenceId: String, seqNr: Long, write: AnyRef) } @@ -55,7 +53,8 @@ class PostgresAsyncWriteJournal(config: Config) extends AsyncWriteJournal { (classOf[JournalConfig], journalConfig), (classOf[Serialization], SerializationExtension(system)), (classOf[ExecutionContext], ec), - (classOf[Materializer], mat)) + (classOf[Materializer], mat) + ) system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[JournalDao](fqcn, args) match { case Success(dao) => dao case Failure(cause) => throw cause @@ -66,8 +65,10 @@ class PostgresAsyncWriteJournal(config: Config) extends AsyncWriteJournal { journalDao match { case upgraded: JournalDaoWithUpdates => upgraded case _ => - throw new IllegalStateException(s"The ${journalDao.getClass} does NOT implement [JournalDaoWithUpdates], " + - s"which is required to perform updates of events! Please configure a valid update capable DAO (e.g. the default [FlatJournalDao].") + throw new IllegalStateException( + s"The ${journalDao.getClass} does NOT implement [JournalDaoWithUpdates], " + + s"which is required to perform updates of events! Please configure a valid update capable DAO (e.g. the default [FlatJournalDao]." + ) } // readHighestSequence must be performed after pending write for a persistenceId @@ -111,7 +112,8 @@ class PostgresAsyncWriteJournal(config: Config) extends AsyncWriteJournal { } override def asyncReplayMessages(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, max: Long)( - recoveryCallback: PersistentRepr => Unit): Future[Unit] = + recoveryCallback: PersistentRepr => Unit + ): Future[Unit] = journalDao .messagesWithBatch(persistenceId, fromSequenceNr, toSequenceNr, journalConfig.daoConfig.replayBatchSize, None) .take(max) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/BaseByteArrayJournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/BaseByteArrayJournalDao.scala index f4c58d97..7a2eb84d 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/BaseByteArrayJournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/BaseByteArrayJournalDao.scala @@ -6,26 +6,25 @@ package org.apache.pekko.persistence.postgres package journal.dao +import org.apache.pekko.{Done, NotUsed} import org.apache.pekko.actor.Scheduler +import org.apache.pekko.persistence.{AtomicWrite, PersistentRepr} import org.apache.pekko.persistence.postgres.config.JournalConfig import org.apache.pekko.persistence.postgres.serialization.FlowPersistentReprSerializer import org.apache.pekko.persistence.postgres.tag.TagIdResolver -import org.apache.pekko.persistence.{ AtomicWrite, PersistentRepr } -import org.apache.pekko.stream.scaladsl.{ Keep, Sink, Source } -import org.apache.pekko.stream.{ Materializer, OverflowStrategy, QueueOfferResult } -import org.apache.pekko.{ Done, NotUsed } -import org.slf4j.{ Logger, LoggerFactory } +import org.apache.pekko.stream.{Materializer, OverflowStrategy, QueueOfferResult} +import org.apache.pekko.stream.scaladsl.{Keep, Sink, Source} +import org.slf4j.{Logger, LoggerFactory} import slick.dbio.DBIOAction import slick.jdbc.JdbcBackend._ import scala.collection.immutable._ +import scala.concurrent.{ExecutionContext, Future, Promise} import scala.concurrent.duration.FiniteDuration -import scala.concurrent.{ ExecutionContext, Future, Promise } -import scala.util.{ Failure, Success, Try } +import scala.util.{Failure, Success, Try} -/** - * The DefaultJournalDao contains all the knowledge to persist and load serialized journal entries - */ +/** The DefaultJournalDao contains all the knowledge to persist and load serialized journal entries + */ trait BaseByteArrayJournalDao extends JournalDaoWithUpdates with BaseJournalDaoWithReadMessages { val db: Database val queries: JournalQueries @@ -35,13 +34,14 @@ trait BaseByteArrayJournalDao extends JournalDaoWithUpdates with BaseJournalDaoW implicit val ec: ExecutionContext implicit val mat: Materializer + import journalConfig.daoConfig.{batchSize, bufferSize, logicalDelete, parallelism} import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile.api._ - import journalConfig.daoConfig.{ batchSize, bufferSize, logicalDelete, parallelism } val logger: Logger = LoggerFactory.getLogger(this.getClass) lazy val metadataQueries: JournalMetadataQueries = new JournalMetadataQueries( - JournalMetadataTable(journalConfig.journalMetadataTableConfiguration)) + JournalMetadataTable(journalConfig.journalMetadataTableConfiguration) + ) // This logging may block since we don't control how the user will configure logback // We can't use a Pekko logging neither because we don't have an ActorSystem in scope and @@ -50,7 +50,8 @@ trait BaseByteArrayJournalDao extends JournalDaoWithUpdates with BaseJournalDaoW lazy val logWarnAboutLogicalDeletionDeprecation: Unit = { logger.warn( "Logical deletion of events is deprecated and will be removed. " + - "To disable it in this current version you must set the property 'pekko-persistence-postgres.logicalDeletion.enable' to false.") + "To disable it in this current version you must set the property 'pekko-persistence-postgres.logicalDeletion.enable' to false." + ) } private val writeQueue = Source @@ -74,8 +75,11 @@ trait BaseByteArrayJournalDao extends JournalDaoWithUpdates with BaseJournalDaoW case QueueOfferResult.Failure(t) => Future.failed(new Exception("Failed to write journal row batch", t)) case QueueOfferResult.Dropped => - Future.failed(new Exception( - s"Failed to enqueue journal row batch write, the queue buffer was full ($bufferSize elements) please check the postgres-journal.bufferSize setting")) + Future.failed( + new Exception( + s"Failed to enqueue journal row batch write, the queue buffer was full ($bufferSize elements) please check the postgres-journal.bufferSize setting" + ) + ) case QueueOfferResult.QueueClosed => Future.failed(new Exception("Failed to enqueue journal row batch write, the queue was closed")) } @@ -86,9 +90,9 @@ trait BaseByteArrayJournalDao extends JournalDaoWithUpdates with BaseJournalDaoW db.run(queries.writeJournalRows(xs).transactionally).map(_ => ()) } - /** - * @see [[org.apache.pekko.persistence.journal.AsyncWriteJournal.asyncWriteMessages(messages)]] - */ + /** @see + * [[org.apache.pekko.persistence.journal.AsyncWriteJournal.asyncWriteMessages(messages)]] + */ def asyncWriteMessages(messages: Seq[AtomicWrite]): Future[Seq[Try[Unit]]] = Future .sequence { @@ -134,7 +138,8 @@ trait BaseByteArrayJournalDao extends JournalDaoWithUpdates with BaseJournalDaoW case Success(t) => db.run(queries.update(persistenceId, sequenceNr, t.message, t.metadata).map(_ => Done)) case Failure(_) => throw new IllegalArgumentException( - s"Failed to serialize ${write.getClass} for update of [$persistenceId] @ [$sequenceNr]") + s"Failed to serialize ${write.getClass} for update of [$persistenceId] @ [$sequenceNr]" + ) } } @@ -162,7 +167,8 @@ trait BaseByteArrayJournalDao extends JournalDaoWithUpdates with BaseJournalDaoW persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] = + max: Long + ): Source[Try[(PersistentRepr, Long)], NotUsed] = Source .fromPublisher(db.stream(queries.messagesQuery(persistenceId, fromSequenceNr, toSequenceNr, max).result)) .via(serializer.deserializeFlow) @@ -179,7 +185,8 @@ trait BaseJournalDaoWithReadMessages extends JournalDaoWithReadMessages { fromSequenceNr: Long, toSequenceNr: Long, batchSize: Int, - refreshInterval: Option[(FiniteDuration, Scheduler)]): Source[Try[(PersistentRepr, Long)], NotUsed] = { + refreshInterval: Option[(FiniteDuration, Scheduler)] + ): Source[Try[(PersistentRepr, Long)], NotUsed] = { Source .unfoldAsync[(Long, FlowControl), Seq[Try[(PersistentRepr, Long)]]]((Math.max(1, fromSequenceNr), Continue)) { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/ByteArrayJournalSerializer.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/ByteArrayJournalSerializer.scala index c1cd9a62..8f339c52 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/ByteArrayJournalSerializer.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/ByteArrayJournalSerializer.scala @@ -6,20 +6,20 @@ package org.apache.pekko.persistence.postgres package journal.dao +import io.circe.{Decoder, Encoder} import org.apache.pekko.persistence.PersistentRepr import org.apache.pekko.persistence.postgres.journal.dao.ByteArrayJournalSerializer.Metadata import org.apache.pekko.persistence.postgres.serialization.FlowPersistentReprSerializer import org.apache.pekko.persistence.postgres.tag.TagIdResolver -import org.apache.pekko.serialization.{ Serialization, Serializers } -import io.circe.{ Decoder, Encoder } +import org.apache.pekko.serialization.{Serialization, Serializers} import scala.collection.immutable._ -import scala.concurrent.{ ExecutionContext, Future } +import scala.concurrent.{ExecutionContext, Future} import scala.util.Try -class ByteArrayJournalSerializer(serialization: Serialization, tagConverter: TagIdResolver)( - implicit val executionContext: ExecutionContext) - extends FlowPersistentReprSerializer[JournalRow] { +class ByteArrayJournalSerializer(serialization: Serialization, tagConverter: TagIdResolver)(implicit + val executionContext: ExecutionContext +) extends FlowPersistentReprSerializer[JournalRow] { override def serialize(persistentRepr: PersistentRepr, tags: Set[String]): Future[JournalRow] = { import io.circe.syntax._ @@ -42,7 +42,8 @@ class ByteArrayJournalSerializer(serialization: Serialization, tagConverter: Tag Option(serManifest).filterNot(_.trim.isEmpty), Option(persistentRepr.manifest).filterNot(_.trim.isEmpty), persistentRepr.writerUuid, - persistentRepr.timestamp) + persistentRepr.timestamp + ) JournalRow( Long.MinValue, persistentRepr.deleted, @@ -50,7 +51,8 @@ class ByteArrayJournalSerializer(serialization: Serialization, tagConverter: Tag persistentRepr.sequenceNr, serializedEvent, convertedTags.toList, - meta.asJson) + meta.asJson + ) } } @@ -69,8 +71,10 @@ class ByteArrayJournalSerializer(serialization: Serialization, tagConverter: Tag deleted = false, // not used, marked as deprecated (https://github.com/akka/akka/issues/27278 sender = null, - metadata.writerUuid).withTimestamp(metadata.timestamp), - journalRow.ordering) + metadata.writerUuid + ).withTimestamp(metadata.timestamp), + journalRow.ordering + ) } } @@ -81,7 +85,8 @@ object ByteArrayJournalSerializer { serManifest: Option[String], eventManifest: Option[String], writerUuid: String, - timestamp: Long) + timestamp: Long + ) object Metadata { implicit val encoder: Encoder[Metadata] = Encoder diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/FlatJournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/FlatJournalDao.scala index 51491977..8331de10 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/FlatJournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/FlatJournalDao.scala @@ -2,17 +2,17 @@ package org.apache.pekko.persistence.postgres package journal.dao import org.apache.pekko.persistence.postgres.config.JournalConfig -import org.apache.pekko.persistence.postgres.tag.{ CachedTagIdResolver, SimpleTagDao } +import org.apache.pekko.persistence.postgres.tag.{CachedTagIdResolver, SimpleTagDao} import org.apache.pekko.serialization.Serialization import org.apache.pekko.stream.Materializer import slick.jdbc.JdbcBackend._ import scala.concurrent.ExecutionContext -class FlatJournalDao(val db: Database, val journalConfig: JournalConfig, serialization: Serialization)( - implicit val ec: ExecutionContext, - val mat: Materializer) - extends BaseByteArrayJournalDao { +class FlatJournalDao(val db: Database, val journalConfig: JournalConfig, serialization: Serialization)(implicit + val ec: ExecutionContext, + val mat: Materializer +) extends BaseByteArrayJournalDao { val queries = new JournalQueries(FlatJournalTable(journalConfig.journalTableConfiguration)) val tagDao = new SimpleTagDao(db, journalConfig.tagsTableConfiguration) val eventTagConverter = new CachedTagIdResolver(tagDao, journalConfig.tagsConfig) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/FlowControl.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/FlowControl.scala index b5d53455..cfae530d 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/FlowControl.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/FlowControl.scala @@ -5,17 +5,15 @@ package org.apache.pekko.persistence.postgres.journal.dao -private[postgres] sealed trait FlowControl +sealed private[postgres] trait FlowControl private[postgres] object FlowControl { /** Keep querying - used when we are sure that there is more events to fetch */ case object Continue extends FlowControl - /** - * Keep querying with delay - used when we have consumed all events, - * but want to poll for future events - */ + /** Keep querying with delay - used when we have consumed all events, but want to poll for future events + */ case object ContinueDelayed extends FlowControl /** Stop querying - used when we reach the desired offset */ diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDao.scala index bf6b24c3..6db4bc7e 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDao.scala @@ -6,25 +6,24 @@ package org.apache.pekko.persistence.postgres.journal.dao import org.apache.pekko.persistence.AtomicWrite + import scala.collection.immutable.Seq import scala.concurrent.Future import scala.util.Try trait JournalDao extends JournalDaoWithReadMessages { - /** - * Deletes all persistent messages up to toSequenceNr (inclusive) for the persistenceId - */ + /** Deletes all persistent messages up to toSequenceNr (inclusive) for the persistenceId + */ def delete(persistenceId: String, toSequenceNr: Long): Future[Unit] - /** - * Returns the highest sequence number for the events that are stored for that `persistenceId`. When no events are - * found for the `persistenceId`, 0L will be the highest sequence number - */ + /** Returns the highest sequence number for the events that are stored for that `persistenceId`. When no events are + * found for the `persistenceId`, 0L will be the highest sequence number + */ def highestSequenceNr(persistenceId: String, fromSequenceNr: Long): Future[Long] - /** - * @see [[org.apache.pekko.persistence.journal.AsyncWriteJournal.asyncWriteMessages(messages)]] - */ + /** @see + * [[org.apache.pekko.persistence.journal.AsyncWriteJournal.asyncWriteMessages(messages)]] + */ def asyncWriteMessages(messages: Seq[AtomicWrite]): Future[Seq[Try[Unit]]] } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDaoWithReadMessages.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDaoWithReadMessages.scala index 42b5e593..28c0cc97 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDaoWithReadMessages.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDaoWithReadMessages.scala @@ -5,37 +5,35 @@ package org.apache.pekko.persistence.postgres.journal.dao -import scala.concurrent.duration.FiniteDuration -import scala.util.Try - import org.apache.pekko.NotUsed import org.apache.pekko.actor.Scheduler import org.apache.pekko.persistence.PersistentRepr import org.apache.pekko.stream.scaladsl.Source +import scala.concurrent.duration.FiniteDuration +import scala.util.Try + trait JournalDaoWithReadMessages { - /** - * Returns a Source of PersistentRepr and ordering number for a certain persistenceId. - * It includes the events with sequenceNr between `fromSequenceNr` (inclusive) and - * `toSequenceNr` (inclusive). - */ + /** Returns a Source of PersistentRepr and ordering number for a certain persistenceId. It includes the events with + * sequenceNr between `fromSequenceNr` (inclusive) and `toSequenceNr` (inclusive). + */ def messages( persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] + max: Long + ): Source[Try[(PersistentRepr, Long)], NotUsed] - /** - * Returns a Source of PersistentRepr and ordering number for a certain persistenceId. - * It includes the events with sequenceNr between `fromSequenceNr` (inclusive) and - * `toSequenceNr` (inclusive). - */ + /** Returns a Source of PersistentRepr and ordering number for a certain persistenceId. It includes the events with + * sequenceNr between `fromSequenceNr` (inclusive) and `toSequenceNr` (inclusive). + */ def messagesWithBatch( persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, batchSize: Int, - refreshInterval: Option[(FiniteDuration, Scheduler)]): Source[Try[(PersistentRepr, Long)], NotUsed] + refreshInterval: Option[(FiniteDuration, Scheduler)] + ): Source[Try[(PersistentRepr, Long)], NotUsed] } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDaoWithUpdates.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDaoWithUpdates.scala index a646d15e..95a11997 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDaoWithUpdates.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalDaoWithUpdates.scala @@ -9,14 +9,12 @@ import org.apache.pekko.Done import scala.concurrent.Future -/** - * A [[JournalDao]] with extended capabilities, such as updating payloads and tags of existing events. - * These operations should be used sparingly, for example for migrating data from un-encrypted to encrypted formats - */ +/** A [[JournalDao]] with extended capabilities, such as updating payloads and tags of existing events. These operations + * should be used sparingly, for example for migrating data from un-encrypted to encrypted formats + */ trait JournalDaoWithUpdates extends JournalDao { - /** - * Update (!) an existing event with the passed in data. - */ + /** Update (!) an existing event with the passed in data. + */ def update(persistenceId: String, sequenceNr: Long, payload: AnyRef): Future[Done] } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalMetadataQueries.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalMetadataQueries.scala index 56091576..6b38f2c6 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalMetadataQueries.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalMetadataQueries.scala @@ -12,7 +12,8 @@ class JournalMetadataQueries(journalMetadataTable: TableQuery[JournalMetadataTab val highestSequenceNrForPersistenceId = Compiled(_highestSequenceNrForPersistenceId _) private def _minAndMaxOrderingForPersistenceId( - persistenceId: Rep[String]): Query[(Rep[Long], Rep[Long]), (Long, Long), Seq] = + persistenceId: Rep[String] + ): Query[(Rep[Long], Rep[Long]), (Long, Long), Seq] = journalMetadataTable.filter(_.persistenceId === persistenceId).take(1).map(r => (r.minOrdering, r.maxOrdering)) val minAndMaxOrderingForPersistenceId = Compiled(_minAndMaxOrderingForPersistenceId _) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalQueries.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalQueries.scala index 7f57890e..f01483a3 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalQueries.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalQueries.scala @@ -23,15 +23,15 @@ class JournalQueries(journalTable: TableQuery[JournalTable]) { journalTable.filter(_.persistenceId === persistenceId).filter(_.sequenceNumber <= toSequenceNr).delete } - /** - * Updates (!) a payload stored in a specific events row. - * Intended to be used sparingly, e.g. moving all events to their encrypted counterparts. - */ + /** Updates (!) a payload stored in a specific events row. Intended to be used sparingly, e.g. moving all events to + * their encrypted counterparts. + */ def update( persistenceId: String, seqNr: Long, replacement: Array[Byte], - metadata: Json): FixedSqlAction[Int, NoStream, Effect.Write] = { + metadata: Json + ): FixedSqlAction[Int, NoStream, Effect.Write] = { val baseQuery = journalTable.filter(_.persistenceId === persistenceId).filter(_.sequenceNumber === seqNr) baseQuery.map(r => (r.message, r.metadata)).update((replacement, metadata)) @@ -59,7 +59,8 @@ class JournalQueries(journalTable: TableQuery[JournalTable]) { persistenceId: Rep[String], fromSequenceNr: Rep[Long], toSequenceNr: Rep[Long], - max: ConstColumn[Long]) = + max: ConstColumn[Long] + ) = journalTable .filter(_.persistenceId === persistenceId) .filter(_.deleted === false) @@ -73,7 +74,8 @@ class JournalQueries(journalTable: TableQuery[JournalTable]) { fromSequenceNr: Rep[Long], toSequenceNr: Rep[Long], max: ConstColumn[Long], - minOrdering: Rep[Long]): Query[JournalTable, JournalRow, Seq] = + minOrdering: Rep[Long] + ): Query[JournalTable, JournalRow, Seq] = journalTable .filter(_.persistenceId === persistenceId) .filter(_.deleted === false) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalTables.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalTables.scala index 583ea6e7..d4598d87 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalTables.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalTables.scala @@ -6,9 +6,9 @@ package org.apache.pekko.persistence.postgres package journal.dao -import org.apache.pekko.persistence.postgres.config.{ JournalMetadataTableConfiguration, JournalTableConfiguration } -import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile.api._ import io.circe.Json +import org.apache.pekko.persistence.postgres.config.{JournalMetadataTableConfiguration, JournalTableConfiguration} +import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile.api._ trait JournalTable extends Table[JournalRow] { def ordering: Rep[Long] @@ -24,7 +24,8 @@ abstract class BaseJournalTable(_tableTag: Tag, journalTableCfg: JournalTableCon extends Table[JournalRow]( _tableTag, _schemaName = journalTableCfg.schemaName, - _tableName = journalTableCfg.tableName) + _tableName = journalTableCfg.tableName + ) with JournalTable class FlatJournalTable private[dao] (_tableTag: Tag, journalTableCfg: JournalTableConfiguration) @@ -36,7 +37,8 @@ class FlatJournalTable private[dao] (_tableTag: Tag, journalTableCfg: JournalTab sequenceNumber, message, tags, - metadata) <> (JournalRow.tupled, JournalRow.unapply) + metadata + ) <> (JournalRow.tupled, JournalRow.unapply) val ordering: Rep[Long] = column[Long](journalTableCfg.columnNames.ordering, O.AutoInc) val persistenceId: Rep[String] = @@ -66,7 +68,8 @@ class PartitionedJournalTable private (_tableTag: Tag, journalTableCfg: JournalT sequenceNumber, message, tags, - metadata) <> (JournalRow.tupled, JournalRow.unapply) + metadata + ) <> (JournalRow.tupled, JournalRow.unapply) val ordering: Rep[Long] = column[Long](journalTableCfg.columnNames.ordering) val persistenceId: Rep[String] = @@ -95,13 +98,15 @@ class JournalMetadataTable(_tableTag: Tag, journalMetadataTableCfg: JournalMetad extends Table[JournalMetadataRow]( _tableTag, _schemaName = journalMetadataTableCfg.schemaName, - _tableName = journalMetadataTableCfg.tableName) { + _tableName = journalMetadataTableCfg.tableName + ) { override def * = ( id, persistenceId, maxSequenceNumber, minOrdering, - maxOrdering) <> (JournalMetadataRow.tupled, JournalMetadataRow.unapply) + maxOrdering + ) <> (JournalMetadataRow.tupled, JournalMetadataRow.unapply) val id: Rep[Long] = column[Long](journalMetadataTableCfg.columnNames.id) val persistenceId: Rep[String] = @@ -114,7 +119,6 @@ class JournalMetadataTable(_tableTag: Tag, journalMetadataTableCfg: JournalMetad } object JournalMetadataTable { - def apply( - journalMetadataTableCfg: JournalMetadataTableConfiguration): TableQuery[JournalMetadataTable] = + def apply(journalMetadataTableCfg: JournalMetadataTableConfiguration): TableQuery[JournalMetadataTable] = TableQuery(tag => new JournalMetadataTable(tag, journalMetadataTableCfg)) } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/NestedPartitionsJournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/NestedPartitionsJournalDao.scala index 6cb47aa0..b7901ad4 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/NestedPartitionsJournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/NestedPartitionsJournalDao.scala @@ -1,7 +1,5 @@ package org.apache.pekko.persistence.postgres.journal.dao -import java.util.concurrent.ConcurrentHashMap - import org.apache.pekko.persistence.postgres.JournalRow import org.apache.pekko.persistence.postgres.config.JournalConfig import org.apache.pekko.persistence.postgres.db.DbErrors.withHandledPartitionErrors @@ -10,13 +8,14 @@ import org.apache.pekko.serialization.Serialization import org.apache.pekko.stream.Materializer import slick.jdbc.JdbcBackend.Database -import scala.collection.immutable.{ List, Nil, Seq } -import scala.concurrent.{ ExecutionContext, Future } +import java.util.concurrent.ConcurrentHashMap +import scala.collection.immutable.{List, Nil, Seq} +import scala.concurrent.{ExecutionContext, Future} -class NestedPartitionsJournalDao(db: Database, journalConfig: JournalConfig, serialization: Serialization)( - implicit ec: ExecutionContext, - mat: Materializer) - extends FlatJournalDao(db, journalConfig, serialization) { +class NestedPartitionsJournalDao(db: Database, journalConfig: JournalConfig, serialization: Serialization)(implicit + ec: ExecutionContext, + mat: Materializer +) extends FlatJournalDao(db, journalConfig, serialization) { override val queries = new JournalQueries(NestedPartitionsJournalTable(journalConfig.journalTableConfiguration)) private val journalTableCfg = journalConfig.journalTableConfiguration private val partitionSize = journalConfig.partitionsConfig.size diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/PartitionedJournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/PartitionedJournalDao.scala index 73a411fd..851064e1 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/PartitionedJournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/journal/dao/PartitionedJournalDao.scala @@ -4,21 +4,21 @@ import org.apache.pekko.NotUsed import org.apache.pekko.persistence.PersistentRepr import org.apache.pekko.persistence.postgres.JournalRow import org.apache.pekko.persistence.postgres.config.JournalConfig -import org.apache.pekko.persistence.postgres.db.DbErrors.{ withHandledIndexErrors, withHandledPartitionErrors } +import org.apache.pekko.persistence.postgres.db.DbErrors.{withHandledIndexErrors, withHandledPartitionErrors} import org.apache.pekko.serialization.Serialization import org.apache.pekko.stream.Materializer import org.apache.pekko.stream.scaladsl.Source import slick.jdbc.JdbcBackend.Database import java.util.concurrent.atomic.AtomicReference -import scala.collection.immutable.{ Nil, Seq } -import scala.concurrent.{ ExecutionContext, Future } +import scala.collection.immutable.{Nil, Seq} +import scala.concurrent.{ExecutionContext, Future} import scala.util.Try -class PartitionedJournalDao(db: Database, journalConfig: JournalConfig, serialization: Serialization)( - implicit ec: ExecutionContext, - mat: Materializer) - extends FlatJournalDao(db, journalConfig, serialization) { +class PartitionedJournalDao(db: Database, journalConfig: JournalConfig, serialization: Serialization)(implicit + ec: ExecutionContext, + mat: Materializer +) extends FlatJournalDao(db, journalConfig, serialization) { override val queries = new JournalQueries(PartitionedJournalTable(journalConfig.journalTableConfiguration)) private val journalTableCfg = journalConfig.journalTableConfiguration private val partitionSize = journalConfig.partitionsConfig.size @@ -93,7 +93,8 @@ class PartitionedJournalDao(db: Database, journalConfig: JournalConfig, serializ persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] = { + max: Long + ): Source[Try[(PersistentRepr, Long)], NotUsed] = { // This behaviour override is only applied here, because it is only useful on the PartitionedJournal strategy. val query = if (journalConfig.useJournalMetadata) { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/package.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/package.scala index 5a87a844..d77d1fc4 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/package.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/package.scala @@ -15,12 +15,14 @@ package object postgres { sequenceNumber: Long, message: Array[Byte], tags: List[Int], - metadata: Json) + metadata: Json + ) final case class JournalMetadataRow( id: Long, persistenceId: String, maxSequenceNumber: Long, minOrdering: Long, - maxOrdering: Long) + maxOrdering: Long + ) } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/JournalSequenceActor.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/JournalSequenceActor.scala index c0673e2e..bc21dede 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/JournalSequenceActor.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/JournalSequenceActor.scala @@ -6,10 +6,10 @@ package org.apache.pekko.persistence.postgres package query -import org.apache.pekko.actor.{ Actor, ActorLogging, Props, Status, Timers } -import org.apache.pekko.persistence.postgres.query.dao.ReadJournalDao +import org.apache.pekko.actor.{Actor, ActorLogging, Props, Status, Timers} import org.apache.pekko.pattern.pipe import org.apache.pekko.persistence.postgres.config.JournalSequenceRetrievalConfig +import org.apache.pekko.persistence.postgres.query.dao.ReadJournalDao import org.apache.pekko.stream.Materializer import org.apache.pekko.stream.scaladsl.Sink @@ -17,8 +17,9 @@ import scala.collection.immutable.NumericRange import scala.concurrent.duration.FiniteDuration object JournalSequenceActor { - def props(readJournalDao: ReadJournalDao, config: JournalSequenceRetrievalConfig)( - implicit materializer: Materializer): Props = Props(new JournalSequenceActor(readJournalDao, config)) + def props(readJournalDao: ReadJournalDao, config: JournalSequenceRetrievalConfig)(implicit + materializer: Materializer + ): Props = Props(new JournalSequenceActor(readJournalDao, config)) private case object QueryOrderingIds private case class NewOrderingIds(originalOffset: Long, elements: Seq[OrderingId]) @@ -34,10 +35,8 @@ object JournalSequenceActor { private type OrderingId = Long - /** - * Efficient representation of missing elements using NumericRanges. - * It can be seen as a collection of OrderingIds - */ + /** Efficient representation of missing elements using NumericRanges. It can be seen as a collection of OrderingIds + */ private case class MissingElements(elements: Seq[NumericRange[OrderingId]]) { def addRange(from: OrderingId, until: OrderingId): MissingElements = { val newRange = from.until(until) @@ -51,19 +50,18 @@ object JournalSequenceActor { } } -/** - * To support the EventsByTag query, this actor keeps track of which rows are visible in the database. - * This is required to guarantee the EventByTag does not skip any rows in case rows with a higher (ordering) id are - * visible in the database before rows with a lower (ordering) id. - */ -class JournalSequenceActor(readJournalDao: ReadJournalDao, config: JournalSequenceRetrievalConfig)( - implicit materializer: Materializer) - extends Actor +/** To support the EventsByTag query, this actor keeps track of which rows are visible in the database. This is required + * to guarantee the EventByTag does not skip any rows in case rows with a higher (ordering) id are visible in the + * database before rows with a lower (ordering) id. + */ +class JournalSequenceActor(readJournalDao: ReadJournalDao, config: JournalSequenceRetrievalConfig)(implicit + materializer: Materializer +) extends Actor with ActorLogging with Timers { import JournalSequenceActor._ + import config.{batchSize, maxBackoffQueryDelay, maxTries, queryDelay} import context.dispatcher - import config.{ batchSize, maxBackoffQueryDelay, maxTries, queryDelay } override def receive: Receive = receive(0L, Map.empty, 0) @@ -77,18 +75,22 @@ class JournalSequenceActor(readJournalDao: ReadJournalDao, config: JournalSequen } } - /** - * @param currentMaxOrdering The highest ordering value for which it is known that no missing elements exist - * @param missingByCounter A map with missing orderingIds. The key of the map is the count at which the missing elements - * can be assumed to be "skipped ids" (they are no longer assumed missing). - * @param moduloCounter A counter which is incremented every time a new query have been executed, modulo `maxTries` - * @param previousDelay The last used delay (may change in case failures occur) - */ + /** @param currentMaxOrdering + * The highest ordering value for which it is known that no missing elements exist + * @param missingByCounter + * A map with missing orderingIds. The key of the map is the count at which the missing elements can be assumed to + * be "skipped ids" (they are no longer assumed missing). + * @param moduloCounter + * A counter which is incremented every time a new query have been executed, modulo `maxTries` + * @param previousDelay + * The last used delay (may change in case failures occur) + */ def receive( currentMaxOrdering: OrderingId, missingByCounter: Map[Int, MissingElements], moduloCounter: Int, - previousDelay: FiniteDuration = queryDelay): Receive = { + previousDelay: FiniteDuration = queryDelay + ): Receive = { case ScheduleAssumeMaxOrderingId(max) => // All elements smaller than max can be assumed missing after this delay val delay = queryDelay * maxTries @@ -126,14 +128,14 @@ class JournalSequenceActor(readJournalDao: ReadJournalDao, config: JournalSequen context.become(receive(currentMaxOrdering, missingByCounter, moduloCounter, newDelay)) } - /** - * This method that implements the "find gaps" algo. It's the meat and main purpose of this actor. - */ + /** This method that implements the "find gaps" algo. It's the meat and main purpose of this actor. + */ def findGaps( elements: Seq[OrderingId], currentMaxOrdering: OrderingId, missingByCounter: Map[Int, MissingElements], - moduloCounter: Int): Unit = { + moduloCounter: Int + ): Unit = { // list of elements that will be considered as genuine gaps. // `givenUp` is either empty or is was filled on a previous iteration val givenUp = missingByCounter.getOrElse(moduloCounter, MissingElements.empty) @@ -143,7 +145,8 @@ class JournalSequenceActor(readJournalDao: ReadJournalDao, config: JournalSequen elements.foldLeft[(OrderingId, OrderingId, MissingElements)]( currentMaxOrdering, currentMaxOrdering, - MissingElements.empty) { case ((currentMax, previousElement, missing), currentElement) => + MissingElements.empty + ) { case ((currentMax, previousElement, missing), currentElement) => // we must decide if we move the cursor forward val newMax = if ((currentMax + 1).until(currentElement).forall(givenUp.contains)) { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/PostgresReadJournalProvider.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/PostgresReadJournalProvider.scala index 0372e248..fa4a2afc 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/PostgresReadJournalProvider.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/PostgresReadJournalProvider.scala @@ -5,9 +5,9 @@ package org.apache.pekko.persistence.postgres.query +import com.typesafe.config.Config import org.apache.pekko.actor.ExtendedActorSystem import org.apache.pekko.persistence.query.ReadJournalProvider -import com.typesafe.config.Config class PostgresReadJournalProvider(system: ExtendedActorSystem, config: Config, configPath: String) extends ReadJournalProvider { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/BaseByteArrayReadJournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/BaseByteArrayReadJournalDao.scala index 17a29c36..62502101 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/BaseByteArrayReadJournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/BaseByteArrayReadJournalDao.scala @@ -15,7 +15,7 @@ import org.apache.pekko.persistence.postgres.journal.dao.{ JournalMetadataTable } import org.apache.pekko.persistence.postgres.serialization.FlowPersistentReprSerializer -import org.apache.pekko.persistence.postgres.tag.{ CachedTagIdResolver, SimpleTagDao, TagIdResolver } +import org.apache.pekko.persistence.postgres.tag.{CachedTagIdResolver, SimpleTagDao, TagIdResolver} import org.apache.pekko.serialization.Serialization import org.apache.pekko.stream.Materializer import org.apache.pekko.stream.scaladsl.Source @@ -23,7 +23,7 @@ import slick.basic.DatabasePublisher import slick.jdbc.JdbcBackend._ import scala.collection.immutable._ -import scala.concurrent.{ ExecutionContext, Future } +import scala.concurrent.{ExecutionContext, Future} import scala.util.Try trait BaseByteArrayReadJournalDao extends ReadJournalDao with BaseJournalDaoWithReadMessages { @@ -42,7 +42,8 @@ trait BaseByteArrayReadJournalDao extends ReadJournalDao with BaseJournalDaoWith tag: String, offset: Long, maxOffset: Long, - max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] = { + max: Long + ): Source[Try[(PersistentRepr, Long)], NotUsed] = { val publisher: Int => DatabasePublisher[JournalRow] = tagId => db.stream(queries.eventsByTag(List(tagId), offset, maxOffset).result) Source @@ -55,7 +56,8 @@ trait BaseByteArrayReadJournalDao extends ReadJournalDao with BaseJournalDaoWith persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] = + max: Long + ): Source[Try[(PersistentRepr, Long)], NotUsed] = Source .fromPublisher(db.stream(queries.messagesQuery(persistenceId, fromSequenceNr, toSequenceNr, max).result)) .via(serializer.deserializeFlow) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/FlatReadJournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/FlatReadJournalDao.scala index 21b3aba5..11036a43 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/FlatReadJournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/FlatReadJournalDao.scala @@ -1,8 +1,8 @@ package org.apache.pekko.persistence.postgres.query.dao import org.apache.pekko.persistence.postgres.config.ReadJournalConfig -import org.apache.pekko.persistence.postgres.journal.dao.{ ByteArrayJournalSerializer, FlatJournalTable } -import org.apache.pekko.persistence.postgres.tag.{ CachedTagIdResolver, SimpleTagDao, TagIdResolver } +import org.apache.pekko.persistence.postgres.journal.dao.{ByteArrayJournalSerializer, FlatJournalTable} +import org.apache.pekko.persistence.postgres.tag.{CachedTagIdResolver, SimpleTagDao, TagIdResolver} import org.apache.pekko.serialization.Serialization import org.apache.pekko.stream.Materializer import slick.jdbc.JdbcBackend.Database @@ -13,14 +13,18 @@ class FlatReadJournalDao( val db: Database, val readJournalConfig: ReadJournalConfig, serialization: Serialization, - val tagIdResolver: TagIdResolver)(implicit val ec: ExecutionContext, val mat: Materializer) + val tagIdResolver: TagIdResolver +)(implicit val ec: ExecutionContext, val mat: Materializer) extends BaseByteArrayReadJournalDao { val queries = new ReadJournalQueries( FlatJournalTable(readJournalConfig.journalTableConfiguration), - readJournalConfig.includeDeleted) + readJournalConfig.includeDeleted + ) val serializer = new ByteArrayJournalSerializer( serialization, new CachedTagIdResolver( new SimpleTagDao(db, readJournalConfig.tagsTableConfiguration), - readJournalConfig.tagsConfig)) + readJournalConfig.tagsConfig + ) + ) } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/PartitionedReadJournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/PartitionedReadJournalDao.scala index 7dd424da..d1380115 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/PartitionedReadJournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/PartitionedReadJournalDao.scala @@ -8,7 +8,7 @@ import org.apache.pekko.persistence.postgres.journal.dao.{ JournalMetadataTable, PartitionedJournalTable } -import org.apache.pekko.persistence.postgres.tag.{ CachedTagIdResolver, SimpleTagDao, TagIdResolver } +import org.apache.pekko.persistence.postgres.tag.{CachedTagIdResolver, SimpleTagDao, TagIdResolver} import org.apache.pekko.serialization.Serialization import org.apache.pekko.stream.Materializer import org.apache.pekko.stream.scaladsl.Source @@ -21,28 +21,34 @@ class PartitionedReadJournalDao( val db: Database, val readJournalConfig: ReadJournalConfig, serialization: Serialization, - val tagIdResolver: TagIdResolver)(implicit val ec: ExecutionContext, val mat: Materializer) + val tagIdResolver: TagIdResolver +)(implicit val ec: ExecutionContext, val mat: Materializer) extends BaseByteArrayReadJournalDao { import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile.api._ val queries = new ReadJournalQueries( PartitionedJournalTable(readJournalConfig.journalTableConfiguration), - readJournalConfig.includeDeleted) + readJournalConfig.includeDeleted + ) private val metadataQueries: ReadJournalMetadataQueries = new ReadJournalMetadataQueries( - JournalMetadataTable(readJournalConfig.journalMetadataTableConfiguration)) + JournalMetadataTable(readJournalConfig.journalMetadataTableConfiguration) + ) val serializer = new ByteArrayJournalSerializer( serialization, new CachedTagIdResolver( new SimpleTagDao(db, readJournalConfig.tagsTableConfiguration), - readJournalConfig.tagsConfig)) + readJournalConfig.tagsConfig + ) + ) override def messages( persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] = { + max: Long + ): Source[Try[(PersistentRepr, Long)], NotUsed] = { // This behaviour override is only applied here, because it is only useful on the PartitionedJournal strategy. val query = if (readJournalConfig.useJournalMetadata) { metadataQueries.minAndMaxOrderingForPersistenceId(persistenceId).result.headOption.flatMap { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalDao.scala index 9a6a43b9..c7350948 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalDao.scala @@ -16,27 +16,27 @@ import scala.util.Try trait ReadJournalDao extends JournalDaoWithReadMessages { - /** - * Returns distinct stream of persistenceIds - */ + /** Returns distinct stream of persistenceIds + */ def allPersistenceIdsSource(max: Long): Source[String, NotUsed] - /** - * Returns a Source of deserialized data for certain tag from an offset. The result is sorted by - * the global ordering of the events. - * Each element with be a try with a PersistentRepr, set of tags, and a Long representing the global ordering of events - */ + /** Returns a Source of deserialized data for certain tag from an offset. The result is sorted by the global ordering + * of the events. Each element with be a try with a PersistentRepr, set of tags, and a Long representing the global + * ordering of events + */ def eventsByTag(tag: String, offset: Long, maxOffset: Long, max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] - /** - * @param offset Minimum value to retrieve - * @param limit Maximum number of values to retrieve - * @return A Source of journal event sequence numbers (corresponding to the Ordering column) - */ + /** @param offset + * Minimum value to retrieve + * @param limit + * Maximum number of values to retrieve + * @return + * A Source of journal event sequence numbers (corresponding to the Ordering column) + */ def journalSequence(offset: Long, limit: Long): Source[Long, NotUsed] - /** - * @return The value of the maximum (ordering) id in the journal - */ + /** @return + * The value of the maximum (ordering) id in the journal + */ def maxJournalSequence(): Future[Long] } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalMetadataQueries.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalMetadataQueries.scala index 0d89525f..f424ecf0 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalMetadataQueries.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalMetadataQueries.scala @@ -8,7 +8,8 @@ class ReadJournalMetadataQueries(journalMetadataTable: TableQuery[JournalMetadat import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile.api._ private def _minAndMaxOrderingForPersistenceId( - persistenceId: Rep[String]): Query[(Rep[Long], Rep[Long]), (Long, Long), Seq] = + persistenceId: Rep[String] + ): Query[(Rep[Long], Rep[Long]), (Long, Long), Seq] = journalMetadataTable.filter(_.persistenceId === persistenceId).take(1).map(r => (r.minOrdering, r.maxOrdering)) val minAndMaxOrderingForPersistenceId = Compiled(_minAndMaxOrderingForPersistenceId _) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalQueries.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalQueries.scala index b3680f41..2ce5d074 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalQueries.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalQueries.scala @@ -25,7 +25,8 @@ class ReadJournalQueries(journalTable: TableQuery[JournalTable], includeDeleted: persistenceId: Rep[String], fromSequenceNr: Rep[Long], toSequenceNr: Rep[Long], - max: ConstColumn[Long]): Query[JournalTable, JournalRow, Seq] = + max: ConstColumn[Long] + ): Query[JournalTable, JournalRow, Seq] = baseTableQuery() .filter(_.persistenceId === persistenceId) .filter(_.sequenceNumber >= fromSequenceNr) @@ -38,7 +39,8 @@ class ReadJournalQueries(journalTable: TableQuery[JournalTable], includeDeleted: fromSequenceNr: Rep[Long], toSequenceNr: Rep[Long], max: ConstColumn[Long], - minOrdering: Rep[Long]): Query[JournalTable, JournalRow, Seq] = + minOrdering: Rep[Long] + ): Query[JournalTable, JournalRow, Seq] = baseTableQuery() .filter(_.persistenceId === persistenceId) .filter(_.sequenceNumber >= fromSequenceNr) @@ -54,7 +56,8 @@ class ReadJournalQueries(journalTable: TableQuery[JournalTable], includeDeleted: protected def _eventsByTag( tag: Rep[List[Int]], offset: ConstColumn[Long], - maxOffset: ConstColumn[Long]): Query[JournalTable, JournalRow, Seq] = { + maxOffset: ConstColumn[Long] + ): Query[JournalTable, JournalRow, Seq] = { baseTableQuery() .filter(_.tags @> tag) .sortBy(_.ordering.asc) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/javadsl/PostgresReadJournal.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/javadsl/PostgresReadJournal.scala index 57ab471d..d174b685 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/javadsl/PostgresReadJournal.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/javadsl/PostgresReadJournal.scala @@ -6,8 +6,8 @@ package org.apache.pekko.persistence.postgres.query.javadsl import org.apache.pekko.NotUsed -import org.apache.pekko.persistence.postgres.query.scaladsl.{ PostgresReadJournal => ScalaPostgresReadJournal } -import org.apache.pekko.persistence.query.{ EventEnvelope, Offset } +import org.apache.pekko.persistence.postgres.query.scaladsl.{PostgresReadJournal => ScalaPostgresReadJournal} +import org.apache.pekko.persistence.query.{EventEnvelope, Offset} import org.apache.pekko.persistence.query.javadsl._ import org.apache.pekko.stream.javadsl.Source @@ -24,102 +24,87 @@ class PostgresReadJournal(journal: ScalaPostgresReadJournal) with CurrentEventsByTagQuery with EventsByTagQuery { - /** - * Same type of query as `persistenceIds` but the event stream - * is completed immediately when it reaches the end of the "result set". Events that are - * stored after the query is completed are not included in the event stream. - */ + /** Same type of query as `persistenceIds` but the event stream is completed immediately when it reaches the end of + * the "result set". Events that are stored after the query is completed are not included in the event stream. + */ override def currentPersistenceIds(): Source[String, NotUsed] = journal.currentPersistenceIds().asJava - /** - * `persistenceIds` is used to retrieve a stream of all `persistenceId`s as strings. - * - * The stream guarantees that a `persistenceId` is only emitted once and there are no duplicates. - * Order is not defined. Multiple executions of the same stream (even bounded) may emit different - * sequence of `persistenceId`s. - * - * The stream is not completed when it reaches the end of the currently known `persistenceId`s, - * but it continues to push new `persistenceId`s when new events are persisted. - * Corresponding query that is completed when it reaches the end of the currently - * known `persistenceId`s is provided by `currentPersistenceIds`. - */ + /** `persistenceIds` is used to retrieve a stream of all `persistenceId`s as strings. + * + * The stream guarantees that a `persistenceId` is only emitted once and there are no duplicates. Order is not + * defined. Multiple executions of the same stream (even bounded) may emit different sequence of `persistenceId`s. + * + * The stream is not completed when it reaches the end of the currently known `persistenceId`s, but it continues to + * push new `persistenceId`s when new events are persisted. Corresponding query that is completed when it reaches the + * end of the currently known `persistenceId`s is provided by `currentPersistenceIds`. + */ override def persistenceIds(): Source[String, NotUsed] = journal.persistenceIds().asJava - /** - * Same type of query as `eventsByPersistenceId` but the event stream - * is completed immediately when it reaches the end of the "result set". Events that are - * stored after the query is completed are not included in the event stream. - */ + /** Same type of query as `eventsByPersistenceId` but the event stream is completed immediately when it reaches the + * end of the "result set". Events that are stored after the query is completed are not included in the event stream. + */ override def currentEventsByPersistenceId( persistenceId: String, fromSequenceNr: Long, - toSequenceNr: Long): Source[EventEnvelope, NotUsed] = + toSequenceNr: Long + ): Source[EventEnvelope, NotUsed] = journal.currentEventsByPersistenceId(persistenceId, fromSequenceNr, toSequenceNr).asJava - /** - * `eventsByPersistenceId` is used to retrieve a stream of events for a particular persistenceId. - * - * The `EventEnvelope` contains the event and provides `persistenceId` and `sequenceNr` - * for each event. The `sequenceNr` is the sequence number for the persistent actor with the - * `persistenceId` that persisted the event. The `persistenceId` + `sequenceNr` is an unique - * identifier for the event. - * - * `fromSequenceNr` and `toSequenceNr` can be specified to limit the set of returned events. - * The `fromSequenceNr` and `toSequenceNr` are inclusive. - * - * The `EventEnvelope` also provides the `offset` that corresponds to the `ordering` column in - * the Journal table. The `ordering` is a sequential id number that uniquely identifies the - * position of each event, also across different `persistenceId`. The `Offset` type is - * `org.apache.pekko.persistence.query.Sequence` with the `ordering` as the offset value. This is the - * same `ordering` number as is used in the offset of the `eventsByTag` query. - * - * The returned event stream is ordered by `sequenceNr`. - * - * Causality is guaranteed (`sequenceNr`s of events for a particular `persistenceId` are always ordered - * in a sequence monotonically increasing by one). Multiple executions of the same bounded stream are - * guaranteed to emit exactly the same stream of events. - * - * The stream is not completed when it reaches the end of the currently stored events, - * but it continues to push new events when new events are persisted. - * Corresponding query that is completed when it reaches the end of the currently - * stored events is provided by `currentEventsByPersistenceId`. - */ + /** `eventsByPersistenceId` is used to retrieve a stream of events for a particular persistenceId. + * + * The `EventEnvelope` contains the event and provides `persistenceId` and `sequenceNr` for each event. The + * `sequenceNr` is the sequence number for the persistent actor with the `persistenceId` that persisted the event. + * The `persistenceId` + `sequenceNr` is an unique identifier for the event. + * + * `fromSequenceNr` and `toSequenceNr` can be specified to limit the set of returned events. The `fromSequenceNr` and + * `toSequenceNr` are inclusive. + * + * The `EventEnvelope` also provides the `offset` that corresponds to the `ordering` column in the Journal table. The + * `ordering` is a sequential id number that uniquely identifies the position of each event, also across different + * `persistenceId`. The `Offset` type is `org.apache.pekko.persistence.query.Sequence` with the `ordering` as the + * offset value. This is the same `ordering` number as is used in the offset of the `eventsByTag` query. + * + * The returned event stream is ordered by `sequenceNr`. + * + * Causality is guaranteed (`sequenceNr`s of events for a particular `persistenceId` are always ordered in a sequence + * monotonically increasing by one). Multiple executions of the same bounded stream are guaranteed to emit exactly + * the same stream of events. + * + * The stream is not completed when it reaches the end of the currently stored events, but it continues to push new + * events when new events are persisted. Corresponding query that is completed when it reaches the end of the + * currently stored events is provided by `currentEventsByPersistenceId`. + */ override def eventsByPersistenceId( persistenceId: String, fromSequenceNr: Long, - toSequenceNr: Long): Source[EventEnvelope, NotUsed] = + toSequenceNr: Long + ): Source[EventEnvelope, NotUsed] = journal.eventsByPersistenceId(persistenceId, fromSequenceNr, toSequenceNr).asJava - /** - * Same type of query as `eventsByTag` but the event stream - * is completed immediately when it reaches the end of the "result set". Events that are - * stored after the query is completed are not included in the event stream. - */ + /** Same type of query as `eventsByTag` but the event stream is completed immediately when it reaches the end of the + * "result set". Events that are stored after the query is completed are not included in the event stream. + */ override def currentEventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] = journal.currentEventsByTag(tag, offset).asJava - /** - * Query events that have a specific tag. - * - * The consumer can keep track of its current position in the event stream by storing the - * `offset` and restart the query from a given `offset` after a crash/restart. - * The offset is exclusive, i.e. the event corresponding to the given `offset` parameter is not - * included in the stream. - * - * For pekko-persistence-postgres the `offset` corresponds to the `ordering` column in the Journal table. - * The `ordering` is a sequential id number that uniquely identifies the position of each event within - * the event stream. The `Offset` type is `org.apache.pekko.persistence.query.Sequence` with the `ordering` as the - * offset value. - * - * The returned event stream is ordered by `offset`. - * - * The stream is not completed when it reaches the end of the currently stored events, - * but it continues to push new events when new events are persisted. - * Corresponding query that is completed when it reaches the end of the currently - * stored events is provided by [[CurrentEventsByTagQuery#currentEventsByTag]]. - */ + /** Query events that have a specific tag. + * + * The consumer can keep track of its current position in the event stream by storing the `offset` and restart the + * query from a given `offset` after a crash/restart. The offset is exclusive, i.e. the event corresponding to the + * given `offset` parameter is not included in the stream. + * + * For pekko-persistence-postgres the `offset` corresponds to the `ordering` column in the Journal table. The + * `ordering` is a sequential id number that uniquely identifies the position of each event within the event stream. + * The `Offset` type is `org.apache.pekko.persistence.query.Sequence` with the `ordering` as the offset value. + * + * The returned event stream is ordered by `offset`. + * + * The stream is not completed when it reaches the end of the currently stored events, but it continues to push new + * events when new events are persisted. Corresponding query that is completed when it reaches the end of the + * currently stored events is provided by [[CurrentEventsByTagQuery#currentEventsByTag]]. + */ override def eventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] = journal.eventsByTag(tag, offset).asJava } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/package.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/package.scala index cc4ea206..a768b16e 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/package.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/package.scala @@ -8,6 +8,7 @@ package org.apache.pekko.persistence.postgres import org.apache.pekko.NotUsed import org.apache.pekko.persistence.query._ import org.apache.pekko.stream.scaladsl.Source + import scala.language.implicitConversions package object query { @@ -17,7 +18,8 @@ package object query { case NoOffset => 0L case _ => throw new IllegalArgumentException( - "pekko-persistence-postgres does not support " + that.getClass.getName + " offsets") + "pekko-persistence-postgres does not support " + that.getClass.getName + " offsets" + ) } } } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/scaladsl/PostgresReadJournal.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/scaladsl/PostgresReadJournal.scala index 482dce99..e2f82bae 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/query/scaladsl/PostgresReadJournal.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/query/scaladsl/PostgresReadJournal.scala @@ -6,29 +6,29 @@ package org.apache.pekko.persistence.postgres.query package scaladsl +import com.typesafe.config.Config import org.apache.pekko.NotUsed -import org.apache.pekko.actor.{ ExtendedActorSystem, Scheduler } +import org.apache.pekko.actor.{ExtendedActorSystem, Scheduler} +import org.apache.pekko.persistence.{Persistence, PersistentRepr} import org.apache.pekko.persistence.postgres.config.ReadJournalConfig import org.apache.pekko.persistence.postgres.db.SlickExtension import org.apache.pekko.persistence.postgres.journal.dao.FlowControl -import org.apache.pekko.persistence.postgres.query.JournalSequenceActor.{ GetMaxOrderingId, MaxOrderingId } +import org.apache.pekko.persistence.postgres.query.JournalSequenceActor.{GetMaxOrderingId, MaxOrderingId} import org.apache.pekko.persistence.postgres.query.dao.ReadJournalDao -import org.apache.pekko.persistence.postgres.tag.{ CachedTagIdResolver, SimpleTagDao, TagIdResolver } +import org.apache.pekko.persistence.postgres.tag.{CachedTagIdResolver, SimpleTagDao, TagIdResolver} +import org.apache.pekko.persistence.query.{EventEnvelope, Offset, Sequence} import org.apache.pekko.persistence.query.scaladsl._ -import org.apache.pekko.persistence.query.{ EventEnvelope, Offset, Sequence } -import org.apache.pekko.persistence.{ Persistence, PersistentRepr } -import org.apache.pekko.serialization.{ Serialization, SerializationExtension } -import org.apache.pekko.stream.scaladsl.{ Sink, Source } -import org.apache.pekko.stream.{ Materializer, SystemMaterializer } +import org.apache.pekko.serialization.{Serialization, SerializationExtension} +import org.apache.pekko.stream.{Materializer, SystemMaterializer} +import org.apache.pekko.stream.scaladsl.{Sink, Source} import org.apache.pekko.util.Timeout -import com.typesafe.config.Config import org.slf4j.LoggerFactory import slick.jdbc.JdbcBackend._ import scala.collection.immutable._ +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ -import scala.concurrent.{ ExecutionContext, Future } -import scala.util.{ Failure, Success } +import scala.util.{Failure, Success} object PostgresReadJournal { final val Identifier = "postgres-read-journal" @@ -59,7 +59,8 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste val db = slickDb.database val tagIdResolver = new CachedTagIdResolver( new SimpleTagDao(db, readJournalConfig.tagsTableConfiguration), - readJournalConfig.tagsConfig) + readJournalConfig.tagsConfig + ) if (readJournalConfig.addShutdownHook && slickDb.allowShutdown) { system.registerOnTermination { db.close() @@ -72,7 +73,8 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste (classOf[Serialization], SerializationExtension(system)), (classOf[TagIdResolver], tagIdResolver), (classOf[ExecutionContext], ec), - (classOf[Materializer], mat)) + (classOf[Materializer], mat) + ) system.dynamicAccess.createInstanceFor[ReadJournalDao](fqcn, args) match { case Success(dao) => dao case Failure(cause) => throw cause @@ -82,30 +84,26 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste // Started lazily to prevent the actor for querying the db if no eventsByTag queries are used private[query] lazy val journalSequenceActor = system.systemActorOf( JournalSequenceActor.props(readJournalDao, readJournalConfig.journalSequenceRetrievalConfiguration), - s"$configPath.pekko-persistence-postgres-journal-sequence-actor") + s"$configPath.pekko-persistence-postgres-journal-sequence-actor" + ) private val delaySource = Source.tick(readJournalConfig.refreshInterval, 0.seconds, 0).take(1) - /** - * Same type of query as `persistenceIds` but the event stream - * is completed immediately when it reaches the end of the "result set". Events that are - * stored after the query is completed are not included in the event stream. - */ + /** Same type of query as `persistenceIds` but the event stream is completed immediately when it reaches the end of + * the "result set". Events that are stored after the query is completed are not included in the event stream. + */ override def currentPersistenceIds(): Source[String, NotUsed] = readJournalDao.allPersistenceIdsSource(Long.MaxValue) - /** - * `persistenceIds` is used to retrieve a stream of all `persistenceId`s as strings. - * - * The stream guarantees that a `persistenceId` is only emitted once and there are no duplicates. - * Order is not defined. Multiple executions of the same stream (even bounded) may emit different - * sequence of `persistenceId`s. - * - * The stream is not completed when it reaches the end of the currently known `persistenceId`s, - * but it continues to push new `persistenceId`s when new events are persisted. - * Corresponding query that is completed when it reaches the end of the currently - * known `persistenceId`s is provided by `currentPersistenceIds`. - */ + /** `persistenceIds` is used to retrieve a stream of all `persistenceId`s as strings. + * + * The stream guarantees that a `persistenceId` is only emitted once and there are no duplicates. Order is not + * defined. Multiple executions of the same stream (even bounded) may emit different sequence of `persistenceId`s. + * + * The stream is not completed when it reaches the end of the currently known `persistenceId`s, but it continues to + * push new `persistenceId`s when new events are persisted. Corresponding query that is completed when it reaches the + * end of the currently known `persistenceId`s is provided by `currentPersistenceIds`. + */ override def persistenceIds(): Source[String, NotUsed] = Source .repeat(0) @@ -125,60 +123,58 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste adapter.fromJournal(repr.payload, repr.manifest).events.map(repr.withPayload) } - /** - * Same type of query as `eventsByPersistenceId` but the event stream - * is completed immediately when it reaches the end of the "result set". Events that are - * stored after the query is completed are not included in the event stream. - */ + /** Same type of query as `eventsByPersistenceId` but the event stream is completed immediately when it reaches the + * end of the "result set". Events that are stored after the query is completed are not included in the event stream. + */ override def currentEventsByPersistenceId( persistenceId: String, fromSequenceNr: Long, - toSequenceNr: Long): Source[EventEnvelope, NotUsed] = + toSequenceNr: Long + ): Source[EventEnvelope, NotUsed] = eventsByPersistenceIdSource(persistenceId, fromSequenceNr, toSequenceNr, None) - /** - * `eventsByPersistenceId` is used to retrieve a stream of events for a particular persistenceId. - * - * The `EventEnvelope` contains the event and provides `persistenceId` and `sequenceNr` - * for each event. The `sequenceNr` is the sequence number for the persistent actor with the - * `persistenceId` that persisted the event. The `persistenceId` + `sequenceNr` is an unique - * identifier for the event. - * - * `fromSequenceNr` and `toSequenceNr` can be specified to limit the set of returned events. - * The `fromSequenceNr` and `toSequenceNr` are inclusive. - * - * The `EventEnvelope` also provides the `offset` that corresponds to the `ordering` column in - * the Journal table. The `ordering` is a sequential id number that uniquely identifies the - * position of each event, also across different `persistenceId`. The `Offset` type is - * `org.apache.pekko.persistence.query.Sequence` with the `ordering` as the offset value. This is the - * same `ordering` number as is used in the offset of the `eventsByTag` query. - * - * The returned event stream is ordered by `sequenceNr`. - * - * Causality is guaranteed (`sequenceNr`s of events for a particular `persistenceId` are always ordered - * in a sequence monotonically increasing by one). Multiple executions of the same bounded stream are - * guaranteed to emit exactly the same stream of events. - * - * The stream is not completed when it reaches the end of the currently stored events, - * but it continues to push new events when new events are persisted. - * Corresponding query that is completed when it reaches the end of the currently - * stored events is provided by `currentEventsByPersistenceId`. - */ + /** `eventsByPersistenceId` is used to retrieve a stream of events for a particular persistenceId. + * + * The `EventEnvelope` contains the event and provides `persistenceId` and `sequenceNr` for each event. The + * `sequenceNr` is the sequence number for the persistent actor with the `persistenceId` that persisted the event. + * The `persistenceId` + `sequenceNr` is an unique identifier for the event. + * + * `fromSequenceNr` and `toSequenceNr` can be specified to limit the set of returned events. The `fromSequenceNr` and + * `toSequenceNr` are inclusive. + * + * The `EventEnvelope` also provides the `offset` that corresponds to the `ordering` column in the Journal table. The + * `ordering` is a sequential id number that uniquely identifies the position of each event, also across different + * `persistenceId`. The `Offset` type is `org.apache.pekko.persistence.query.Sequence` with the `ordering` as the + * offset value. This is the same `ordering` number as is used in the offset of the `eventsByTag` query. + * + * The returned event stream is ordered by `sequenceNr`. + * + * Causality is guaranteed (`sequenceNr`s of events for a particular `persistenceId` are always ordered in a sequence + * monotonically increasing by one). Multiple executions of the same bounded stream are guaranteed to emit exactly + * the same stream of events. + * + * The stream is not completed when it reaches the end of the currently stored events, but it continues to push new + * events when new events are persisted. Corresponding query that is completed when it reaches the end of the + * currently stored events is provided by `currentEventsByPersistenceId`. + */ override def eventsByPersistenceId( persistenceId: String, fromSequenceNr: Long, - toSequenceNr: Long): Source[EventEnvelope, NotUsed] = + toSequenceNr: Long + ): Source[EventEnvelope, NotUsed] = eventsByPersistenceIdSource( persistenceId, fromSequenceNr, toSequenceNr, - Some(readJournalConfig.refreshInterval -> system.scheduler)) + Some(readJournalConfig.refreshInterval -> system.scheduler) + ) private def eventsByPersistenceIdSource( persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - refreshInterval: Option[(FiniteDuration, Scheduler)]): Source[EventEnvelope, NotUsed] = { + refreshInterval: Option[(FiniteDuration, Scheduler)] + ): Source[EventEnvelope, NotUsed] = { val batchSize = readJournalConfig.maxBufferSize readJournalDao .messagesWithBatch(persistenceId, fromSequenceNr, toSequenceNr, batchSize, refreshInterval) @@ -191,11 +187,9 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste } } - /** - * Same type of query as `eventsByTag` but the event stream - * is completed immediately when it reaches the end of the "result set". Events that are - * stored after the query is completed are not included in the event stream. - */ + /** Same type of query as `eventsByTag` but the event stream is completed immediately when it reaches the end of the + * "result set". Events that are stored after the query is completed are not included in the event stream. + */ override def currentEventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] = currentEventsByTag(tag, offset.value) @@ -203,7 +197,8 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste tag: String, offset: Long, max: Long, - latestOrdering: MaxOrderingId): Source[EventEnvelope, NotUsed] = { + latestOrdering: MaxOrderingId + ): Source[EventEnvelope, NotUsed] = { if (latestOrdering.maxOrdering < offset) Source.empty else { readJournalDao @@ -211,21 +206,22 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste .mapAsync(1)(Future.fromTry) .mapConcat { case (repr, ordering) => adaptEvents(repr).map(r => - EventEnvelope(Sequence(ordering), r.persistenceId, r.sequenceNr, r.payload, r.timestamp)) + EventEnvelope(Sequence(ordering), r.persistenceId, r.sequenceNr, r.payload, r.timestamp) + ) } } } - /** - * @param terminateAfterOffset If None, the stream never completes. If a Some, then the stream will complete once a - * query has been executed which might return an event with this offset (or a higher offset). - * The stream may include offsets higher than the value in terminateAfterOffset, since the last batch - * will be returned completely. - */ + /** @param terminateAfterOffset + * If None, the stream never completes. If a Some, then the stream will complete once a query has been executed + * which might return an event with this offset (or a higher offset). The stream may include offsets higher than + * the value in terminateAfterOffset, since the last batch will be returned completely. + */ private def eventsByTag( tag: String, offset: Long, - terminateAfterOffset: Option[Long]): Source[EventEnvelope, NotUsed] = { + terminateAfterOffset: Option[Long] + ): Source[EventEnvelope, NotUsed] = { import FlowControl._ import org.apache.pekko.pattern.ask implicit val askTimeout: Timeout = Timeout(readJournalConfig.journalSequenceRetrievalConfiguration.askTimeout) @@ -281,7 +277,8 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste } log.trace( - s"tag = $tag => ($nextStartingOffset, $nextControl), [highestOffset = $highestOffset, maxOrdering = ${queryUntil.maxOrdering}, hasMoreEvents = $hasMoreEvents, results = ${xs.size}, from = $from]") + s"tag = $tag => ($nextStartingOffset, $nextControl), [highestOffset = $highestOffset, maxOrdering = ${queryUntil.maxOrdering}, hasMoreEvents = $hasMoreEvents, results = ${xs.size}, from = $from]" + ) Some((nextStartingOffset, nextControl), xs) } } @@ -304,31 +301,26 @@ class PostgresReadJournal(config: Config, configPath: String)(implicit val syste .mapMaterializedValue(_ => NotUsed) } - /** - * Query events that have a specific tag. - * - * The consumer can keep track of its current position in the event stream by storing the - * `offset` and restart the query from a given `offset` after a crash/restart. - * The offset is exclusive, i.e. the event corresponding to the given `offset` parameter is not - * included in the stream. - * - * For pekko-persistence-postgres the `offset` corresponds to the `ordering` column in the Journal table. - * The `ordering` is a sequential id number that uniquely identifies the position of each event within - * the event stream. The `Offset` type is `org.apache.pekko.persistence.query.Sequence` with the `ordering` as the - * offset value. - * - * The returned event stream is ordered by `offset`. - * - * In addition to the `offset` the `EventEnvelope` also provides `persistenceId` and `sequenceNr` - * for each event. The `sequenceNr` is the sequence number for the persistent actor with the - * `persistenceId` that persisted the event. The `persistenceId` + `sequenceNr` is an unique - * identifier for the event. - * - * The stream is not completed when it reaches the end of the currently stored events, - * but it continues to push new events when new events are persisted. - * Corresponding query that is completed when it reaches the end of the currently - * stored events is provided by [[CurrentEventsByTagQuery#currentEventsByTag]]. - */ + /** Query events that have a specific tag. + * + * The consumer can keep track of its current position in the event stream by storing the `offset` and restart the + * query from a given `offset` after a crash/restart. The offset is exclusive, i.e. the event corresponding to the + * given `offset` parameter is not included in the stream. + * + * For pekko-persistence-postgres the `offset` corresponds to the `ordering` column in the Journal table. The + * `ordering` is a sequential id number that uniquely identifies the position of each event within the event stream. + * The `Offset` type is `org.apache.pekko.persistence.query.Sequence` with the `ordering` as the offset value. + * + * The returned event stream is ordered by `offset`. + * + * In addition to the `offset` the `EventEnvelope` also provides `persistenceId` and `sequenceNr` for each event. The + * `sequenceNr` is the sequence number for the persistent actor with the `persistenceId` that persisted the event. + * The `persistenceId` + `sequenceNr` is an unique identifier for the event. + * + * The stream is not completed when it reaches the end of the currently stored events, but it continues to push new + * events when new events are persisted. Corresponding query that is completed when it reaches the end of the + * currently stored events is provided by [[CurrentEventsByTagQuery#currentEventsByTag]]. + */ override def eventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] = eventsByTag(tag, offset.value) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/serialization/PersistentReprSerializer.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/serialization/PersistentReprSerializer.scala index 232596b9..34eafad5 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/serialization/PersistentReprSerializer.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/serialization/PersistentReprSerializer.scala @@ -6,24 +6,22 @@ package org.apache.pekko.persistence.postgres.serialization import org.apache.pekko.NotUsed +import org.apache.pekko.persistence.{AtomicWrite, PersistentRepr} import org.apache.pekko.persistence.journal.Tagged -import org.apache.pekko.persistence.{ AtomicWrite, PersistentRepr } import org.apache.pekko.stream.scaladsl.Flow import scala.collection.immutable._ -import scala.concurrent.{ ExecutionContext, Future } +import scala.concurrent.{ExecutionContext, Future} import scala.util.Try trait PersistentReprSerializer[T] { implicit def executionContext: ExecutionContext - /** - * An org.apache.pekko.persistence.AtomicWrite contains a Sequence of events (with metadata, the PersistentRepr) - * that must all be persisted or all fail, what makes the operation atomic. The function converts - * each AtomicWrite to a Future[Seq[T]]. - * The Try denotes whether there was a problem with the AtomicWrite or not. - */ + /** An org.apache.pekko.persistence.AtomicWrite contains a Sequence of events (with metadata, the PersistentRepr) that + * must all be persisted or all fail, what makes the operation atomic. The function converts each AtomicWrite to a + * Future[Seq[T]]. The Try denotes whether there was a problem with the AtomicWrite or not. + */ def serialize(messages: Seq[AtomicWrite]): Seq[Future[Seq[T]]] = { messages.map { atomicWrite => val serialized = atomicWrite.payload.map(serialize) @@ -39,18 +37,16 @@ trait PersistentReprSerializer[T] { def serialize(persistentRepr: PersistentRepr, tags: Set[String]): Future[T] - /** - * deserialize into a PersistentRepr, a set of tags and a Long representing the global ordering of events - */ + /** deserialize into a PersistentRepr, a set of tags and a Long representing the global ordering of events + */ def deserialize(t: T): Try[(PersistentRepr, Long)] } trait FlowPersistentReprSerializer[T] extends PersistentReprSerializer[T] { - /** - * A flow which deserializes each element into a PersistentRepr, - * a set of tags and a Long representing the global ordering of events - */ + /** A flow which deserializes each element into a PersistentRepr, a set of tags and a Long representing the global + * ordering of events + */ def deserializeFlow: Flow[T, Try[(PersistentRepr, Long)], NotUsed] = { Flow[T].map(deserialize) } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/PostgresSnapshotStore.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/PostgresSnapshotStore.scala index eee878cd..c81a18d7 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/PostgresSnapshotStore.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/PostgresSnapshotStore.scala @@ -5,20 +5,20 @@ package org.apache.pekko.persistence.postgres.snapshot -import org.apache.pekko.actor.{ ActorSystem, ExtendedActorSystem } +import com.typesafe.config.Config +import org.apache.pekko.actor.{ActorSystem, ExtendedActorSystem} +import org.apache.pekko.persistence.{SelectedSnapshot, SnapshotMetadata, SnapshotSelectionCriteria} import org.apache.pekko.persistence.postgres.config.SnapshotConfig -import org.apache.pekko.persistence.postgres.db.{ SlickDatabase, SlickExtension } +import org.apache.pekko.persistence.postgres.db.{SlickDatabase, SlickExtension} import org.apache.pekko.persistence.postgres.snapshot.dao.SnapshotDao import org.apache.pekko.persistence.snapshot.SnapshotStore -import org.apache.pekko.persistence.{ SelectedSnapshot, SnapshotMetadata, SnapshotSelectionCriteria } -import org.apache.pekko.serialization.{ Serialization, SerializationExtension } -import org.apache.pekko.stream.{ Materializer, SystemMaterializer } -import com.typesafe.config.Config +import org.apache.pekko.serialization.{Serialization, SerializationExtension} +import org.apache.pekko.stream.{Materializer, SystemMaterializer} import slick.jdbc.JdbcBackend._ import scala.collection.immutable._ -import scala.concurrent.{ ExecutionContext, Future } -import scala.util.{ Failure, Success } +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} object PostgresSnapshotStore { def toSelectedSnapshot(tupled: (SnapshotMetadata, Any)): SelectedSnapshot = tupled match { @@ -44,7 +44,8 @@ class PostgresSnapshotStore(config: Config) extends SnapshotStore { (classOf[SnapshotConfig], snapshotConfig), (classOf[Serialization], SerializationExtension(system)), (classOf[ExecutionContext], ec), - (classOf[Materializer], mat)) + (classOf[Materializer], mat) + ) system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[SnapshotDao](fqcn, args) match { case Success(dao) => dao case Failure(cause) => throw cause @@ -53,7 +54,8 @@ class PostgresSnapshotStore(config: Config) extends SnapshotStore { override def loadAsync( persistenceId: String, - criteria: SnapshotSelectionCriteria): Future[Option[SelectedSnapshot]] = { + criteria: SnapshotSelectionCriteria + ): Future[Option[SelectedSnapshot]] = { val result = criteria match { case SnapshotSelectionCriteria(Long.MaxValue, Long.MaxValue, _, _) => snapshotDao.latestSnapshot(persistenceId) diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotDao.scala index 7f0dc201..2cd3879c 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotDao.scala @@ -10,16 +10,17 @@ import org.apache.pekko.persistence.postgres.config.SnapshotConfig import org.apache.pekko.persistence.postgres.snapshot.dao.SnapshotTables.SnapshotRow import org.apache.pekko.serialization.Serialization import org.apache.pekko.stream.Materializer -import slick.jdbc.JdbcProfile import slick.jdbc.JdbcBackend +import slick.jdbc.JdbcProfile -import scala.concurrent.{ ExecutionContext, Future } -import scala.util.{ Failure, Success } +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} class ByteArraySnapshotDao(db: JdbcBackend#Database, snapshotConfig: SnapshotConfig, serialization: Serialization)( - implicit ec: ExecutionContext, - val mat: Materializer) - extends SnapshotDao { + implicit + ec: ExecutionContext, + val mat: Materializer +) extends SnapshotDao { import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile.api._ val queries = new SnapshotQueries(snapshotConfig.snapshotTableConfiguration) @@ -39,14 +40,16 @@ class ByteArraySnapshotDao(db: JdbcBackend#Database, snapshotConfig: SnapshotCon override def snapshotForMaxTimestamp( persistenceId: String, - maxTimestamp: Long): Future[Option[(SnapshotMetadata, Any)]] = + maxTimestamp: Long + ): Future[Option[(SnapshotMetadata, Any)]] = for { rows <- db.run(queries.selectOneByPersistenceIdAndMaxTimestamp(persistenceId, maxTimestamp).result) } yield rows.headOption.map(toSnapshotData) override def snapshotForMaxSequenceNr( persistenceId: String, - maxSequenceNr: Long): Future[Option[(SnapshotMetadata, Any)]] = + maxSequenceNr: Long + ): Future[Option[(SnapshotMetadata, Any)]] = for { rows <- db.run(queries.selectOneByPersistenceIdAndMaxSequenceNr(persistenceId, maxSequenceNr).result) } yield rows.headOption.map(toSnapshotData) @@ -54,12 +57,14 @@ class ByteArraySnapshotDao(db: JdbcBackend#Database, snapshotConfig: SnapshotCon override def snapshotForMaxSequenceNrAndMaxTimestamp( persistenceId: String, maxSequenceNr: Long, - maxTimestamp: Long): Future[Option[(SnapshotMetadata, Any)]] = + maxTimestamp: Long + ): Future[Option[(SnapshotMetadata, Any)]] = for { rows <- db.run( queries .selectOneByPersistenceIdAndMaxSequenceNrAndMaxTimestamp(persistenceId, maxSequenceNr, maxTimestamp) - .result) + .result + ) } yield rows.headOption.map(toSnapshotData) override def save(snapshotMetadata: SnapshotMetadata, snapshot: Any): Future[Unit] = { @@ -90,11 +95,13 @@ class ByteArraySnapshotDao(db: JdbcBackend#Database, snapshotConfig: SnapshotCon override def deleteUpToMaxSequenceNrAndMaxTimestamp( persistenceId: String, maxSequenceNr: Long, - maxTimestamp: Long): Future[Unit] = + maxTimestamp: Long + ): Future[Unit] = for { _ <- db.run( queries .selectByPersistenceIdUpToMaxSequenceNrAndMaxTimestamp(persistenceId, maxSequenceNr, maxTimestamp) - .delete) + .delete + ) } yield () } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotSerializer.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotSerializer.scala index 6f513ea4..17c0dd83 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotSerializer.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotSerializer.scala @@ -5,12 +5,12 @@ package org.apache.pekko.persistence.postgres.snapshot.dao +import io.circe.{Decoder, Encoder} import org.apache.pekko.persistence.SnapshotMetadata import org.apache.pekko.persistence.postgres.serialization.SnapshotSerializer import org.apache.pekko.persistence.postgres.snapshot.dao.ByteArraySnapshotSerializer.Metadata import org.apache.pekko.persistence.postgres.snapshot.dao.SnapshotTables.SnapshotRow -import org.apache.pekko.serialization.{ Serialization, Serializers } -import io.circe.{ Decoder, Encoder } +import org.apache.pekko.serialization.{Serialization, Serializers} import scala.util.Try @@ -30,7 +30,8 @@ class ByteArraySnapshotSerializer(serialization: Serialization) extends Snapshot metadata.sequenceNr, metadata.timestamp, serializedSnapshot, - metadataJson.asJson) + metadataJson.asJson + ) } } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotDao.scala index c5f576fa..252845ac 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotDao.scala @@ -19,7 +19,8 @@ trait SnapshotDao { def deleteUpToMaxSequenceNrAndMaxTimestamp( persistenceId: String, maxSequenceNr: Long, - maxTimestamp: Long): Future[Unit] + maxTimestamp: Long + ): Future[Unit] def latestSnapshot(persistenceId: String): Future[Option[(SnapshotMetadata, Any)]] @@ -30,7 +31,8 @@ trait SnapshotDao { def snapshotForMaxSequenceNrAndMaxTimestamp( persistenceId: String, sequenceNr: Long, - timestamp: Long): Future[Option[(SnapshotMetadata, Any)]] + timestamp: Long + ): Future[Option[(SnapshotMetadata, Any)]] def delete(persistenceId: String, sequenceNr: Long): Future[Unit] diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotQueries.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotQueries.scala index 155b5f0e..de182c6f 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotQueries.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotQueries.scala @@ -39,10 +39,12 @@ class SnapshotQueries(override val snapshotTableCfg: SnapshotTableConfiguration) private def _selectByPersistenceIdUpToMaxSequenceNrAndMaxTimestamp( persistenceId: Rep[String], maxSequenceNr: Rep[Long], - maxTimestamp: Rep[Long]) = + maxTimestamp: Rep[Long] + ) = _selectByPersistenceIdUpToMaxSequenceNr(persistenceId, maxSequenceNr).filter(_.created <= maxTimestamp) val selectByPersistenceIdUpToMaxSequenceNrAndMaxTimestamp = Compiled( - _selectByPersistenceIdUpToMaxSequenceNrAndMaxTimestamp _) + _selectByPersistenceIdUpToMaxSequenceNrAndMaxTimestamp _ + ) private def _selectOneByPersistenceIdAndMaxTimestamp(persistenceId: Rep[String], maxTimestamp: Rep[Long]) = _selectAllByPersistenceId(persistenceId).filter(_.created <= maxTimestamp).take(1) @@ -55,8 +57,10 @@ class SnapshotQueries(override val snapshotTableCfg: SnapshotTableConfiguration) private def _selectOneByPersistenceIdAndMaxSequenceNrAndMaxTimestamp( persistenceId: Rep[String], maxSequenceNr: Rep[Long], - maxTimestamp: Rep[Long]) = + maxTimestamp: Rep[Long] + ) = _selectByPersistenceIdUpToMaxSequenceNr(persistenceId, maxSequenceNr).filter(_.created <= maxTimestamp).take(1) val selectOneByPersistenceIdAndMaxSequenceNrAndMaxTimestamp = Compiled( - _selectOneByPersistenceIdAndMaxSequenceNrAndMaxTimestamp _) + _selectOneByPersistenceIdAndMaxSequenceNrAndMaxTimestamp _ + ) } diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotTables.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotTables.scala index 22355748..49c4f7f3 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotTables.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotTables.scala @@ -5,9 +5,9 @@ package org.apache.pekko.persistence.postgres.snapshot.dao +import io.circe.Json import org.apache.pekko.persistence.postgres.config.SnapshotTableConfiguration import org.apache.pekko.persistence.postgres.snapshot.dao.SnapshotTables._ -import io.circe.Json object SnapshotTables { case class SnapshotRow( @@ -15,7 +15,8 @@ object SnapshotTables { sequenceNumber: Long, created: Long, snapshot: Array[Byte], - metadata: Json) + metadata: Json + ) } trait SnapshotTables { @@ -27,7 +28,8 @@ trait SnapshotTables { extends Table[SnapshotRow]( _tableTag, _schemaName = snapshotTableCfg.schemaName, - _tableName = snapshotTableCfg.tableName) { + _tableName = snapshotTableCfg.tableName + ) { def * = (persistenceId, sequenceNumber, created, snapshot, metadata) <> (SnapshotRow.tupled, SnapshotRow.unapply) val persistenceId: Rep[String] = diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/tag/TagDao.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/tag/TagDao.scala index 705c18d4..59f28423 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/tag/TagDao.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/tag/TagDao.scala @@ -3,7 +3,7 @@ package org.apache.pekko.persistence.postgres.tag import org.apache.pekko.persistence.postgres.config.TagsTableConfiguration import slick.jdbc.JdbcBackend.Database -import scala.concurrent.{ ExecutionContext, Future } +import scala.concurrent.{ExecutionContext, Future} trait TagDao { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/tag/TagIdResolver.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/tag/TagIdResolver.scala index 8f7b4fbb..052a6282 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/tag/TagIdResolver.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/tag/TagIdResolver.scala @@ -1,9 +1,9 @@ package org.apache.pekko.persistence.postgres.tag +import com.github.blemale.scaffeine.{AsyncLoadingCache, Scaffeine} import org.apache.pekko.persistence.postgres.config.TagsConfig -import com.github.blemale.scaffeine.{ AsyncLoadingCache, Scaffeine } -import scala.concurrent.{ ExecutionContext, Future } +import scala.concurrent.{ExecutionContext, Future} import scala.util.Success trait TagIdResolver { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/util/BlockingOps.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/util/BlockingOps.scala index ec8d5fc6..dccc178f 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/util/BlockingOps.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/util/BlockingOps.scala @@ -5,8 +5,8 @@ package org.apache.pekko.persistence.postgres.util -import scala.concurrent.duration.{ FiniteDuration, _ } -import scala.concurrent.{ Await, Future } +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.{FiniteDuration, _} object BlockingOps { implicit class BlockingFutureImplicits[T](val that: Future[T]) extends AnyVal { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/util/ByteArrayOps.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/util/ByteArrayOps.scala index f02eb3bc..07eab62c 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/util/ByteArrayOps.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/util/ByteArrayOps.scala @@ -5,7 +5,7 @@ package org.apache.pekko.persistence.postgres.util -import java.io.{ ByteArrayInputStream, InputStream } +import java.io.{ByteArrayInputStream, InputStream} import java.util.Base64 object ByteArrayOps { diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/util/ConfigOps.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/util/ConfigOps.scala index 2ca2f7a0..1b05560d 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/util/ConfigOps.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/util/ConfigOps.scala @@ -5,12 +5,11 @@ package org.apache.pekko.persistence.postgres.util +import com.typesafe.config.{Config, ConfigFactory} + import java.util.Locale import java.util.concurrent.TimeUnit - -import com.typesafe.config.{ Config, ConfigFactory } - -import scala.concurrent.duration.{ Duration, FiniteDuration } +import scala.concurrent.duration.{Duration, FiniteDuration} import scala.language.implicitConversions import scala.util.Try @@ -62,19 +61,18 @@ object ConfigOps { implicit def TryToOption[A](t: Try[A]): Option[A] = t.toOption - final implicit class TryOps[A](val t: Try[A]) extends AnyVal { + implicit final class TryOps[A](val t: Try[A]) extends AnyVal { def ?:(default: A): A = t.getOrElse(default) } - final implicit class StringTryOps(val t: Try[String]) extends AnyVal { + implicit final class StringTryOps(val t: Try[String]) extends AnyVal { - /** - * Trim the String content, when empty, return None - */ + /** Trim the String content, when empty, return None + */ def trim: Option[String] = t.map(_.trim).filter(_.nonEmpty) } - final implicit class Requiring[A](val value: A) extends AnyVal { + implicit final class Requiring[A](val value: A) extends AnyVal { @inline def requiring(cond: Boolean, msg: => Any): A = { require(cond, msg) value diff --git a/core/src/main/scala/org/apache/pekko/persistence/postgres/util/InputStreamOps.scala b/core/src/main/scala/org/apache/pekko/persistence/postgres/util/InputStreamOps.scala index 19b33b0a..d8edd0b4 100644 --- a/core/src/main/scala/org/apache/pekko/persistence/postgres/util/InputStreamOps.scala +++ b/core/src/main/scala/org/apache/pekko/persistence/postgres/util/InputStreamOps.scala @@ -5,8 +5,7 @@ package org.apache.pekko.persistence.postgres.util -import java.io.{ ByteArrayOutputStream, InputStream } - +import java.io.{ByteArrayOutputStream, InputStream} import scala.concurrent.blocking object InputStreamOps { diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/SharedActorSystemTestSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/SharedActorSystemTestSpec.scala index 51f8d7d2..968a351f 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/SharedActorSystemTestSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/SharedActorSystemTestSpec.scala @@ -5,15 +5,15 @@ package org.apache.pekko.persistence.postgres +import com.typesafe.config.{Config, ConfigFactory, ConfigValue} import org.apache.pekko.actor.ActorSystem -import org.apache.pekko.persistence.postgres.config.{ JournalConfig, ReadJournalConfig } +import org.apache.pekko.persistence.postgres.config.{JournalConfig, ReadJournalConfig} import org.apache.pekko.persistence.postgres.db.SlickExtension import org.apache.pekko.persistence.postgres.query.javadsl.PostgresReadJournal import org.apache.pekko.persistence.postgres.util.DropCreate import org.apache.pekko.serialization.SerializationExtension -import org.apache.pekko.stream.{ Materializer, SystemMaterializer } +import org.apache.pekko.stream.{Materializer, SystemMaterializer} import org.apache.pekko.util.Timeout -import com.typesafe.config.{ Config, ConfigFactory, ConfigValue } import org.scalatest.BeforeAndAfterAll import scala.concurrent.ExecutionContext diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/SimpleSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/SimpleSpec.scala index 244206b4..125f029f 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/SimpleSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/SimpleSpec.scala @@ -5,11 +5,11 @@ package org.apache.pekko.persistence.postgres -import org.apache.pekko.actor.{ ActorRef, ActorSystem } +import org.apache.pekko.actor.{ActorRef, ActorSystem} import org.apache.pekko.persistence.postgres.util.ClasspathResources import org.apache.pekko.testkit.TestProbe import org.scalatest._ -import org.scalatest.concurrent.{ Eventually, ScalaFutures } +import org.scalatest.concurrent.{Eventually, ScalaFutures} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers @@ -25,9 +25,8 @@ trait SimpleSpec with BeforeAndAfterEach with GivenWhenThen { - /** - * Sends the PoisonPill command to an actor and waits for it to die - */ + /** Sends the PoisonPill command to an actor and waits for it to die + */ def killActors(actors: ActorRef*)(implicit system: ActorSystem): Unit = { val tp = TestProbe() actors.foreach { (actor: ActorRef) => diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/SingleActorSystemPerTestSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/SingleActorSystemPerTestSpec.scala index 6695c678..3d890bea 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/SingleActorSystemPerTestSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/SingleActorSystemPerTestSpec.scala @@ -5,13 +5,13 @@ package org.apache.pekko.persistence.postgres +import com.typesafe.config.{Config, ConfigFactory, ConfigValue} import org.apache.pekko.actor.ActorSystem -import org.apache.pekko.persistence.postgres.config.{ JournalConfig, ReadJournalConfig, SlickConfiguration } +import org.apache.pekko.persistence.postgres.config.{JournalConfig, ReadJournalConfig, SlickConfiguration} import org.apache.pekko.persistence.postgres.db.SlickDatabase import org.apache.pekko.persistence.postgres.query.javadsl.PostgresReadJournal import org.apache.pekko.persistence.postgres.util.DropCreate import org.apache.pekko.util.Timeout -import com.typesafe.config.{ Config, ConfigFactory, ConfigValue } import org.scalatest.BeforeAndAfterEach import slick.jdbc.JdbcBackend.Database @@ -43,7 +43,8 @@ abstract class SingleActorSystemPerTestSpec(val config: Config) SlickDatabase.database( config, new SlickConfiguration(config.getConfig("pekko-persistence-postgres.shared-databases.slick")), - "pekko-persistence-postgres.shared-databases.slick.db") + "pekko-persistence-postgres.shared-databases.slick.db" + ) dbOpt = Some(newDb) newDb diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/TablesTestSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/TablesTestSpec.scala index b95e2e23..239dccb9 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/TablesTestSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/TablesTestSpec.scala @@ -5,8 +5,8 @@ package org.apache.pekko.persistence.postgres -import org.apache.pekko.persistence.postgres.config._ import com.typesafe.config.ConfigFactory +import org.apache.pekko.persistence.postgres.config._ import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/ConfigOpsTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/ConfigOpsTest.scala index 1ea0cf3f..5cff9f07 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/ConfigOpsTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/ConfigOpsTest.scala @@ -5,12 +5,13 @@ package org.apache.pekko.persistence.postgres.configuration -import org.apache.pekko.persistence.postgres.SimpleSpec -import org.apache.pekko.persistence.postgres.util.ConfigOps -import ConfigOps._ import com.typesafe.config.ConfigFactory +import org.apache.pekko.persistence.postgres.SimpleSpec class ConfigOpsTest extends SimpleSpec { + + import org.apache.pekko.persistence.postgres.util.ConfigOps._ + it should "parse field values to Try[A]" in { val cfg = ConfigFactory.parseString(""" | person { diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/JNDIConfigTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/JNDIConfigTest.scala index 80ab20c7..db762699 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/JNDIConfigTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/JNDIConfigTest.scala @@ -5,10 +5,10 @@ package org.apache.pekko.persistence.postgres.configuration +import com.typesafe.config.ConfigFactory import org.apache.pekko.actor.ActorSystem import org.apache.pekko.persistence.postgres.SimpleSpec import org.apache.pekko.persistence.postgres.db.SlickExtension -import com.typesafe.config.ConfigFactory class JNDIConfigTest extends SimpleSpec { "JNDI config" should "read the config and throw NoInitialContextException in case the JNDI resource is not available" in { diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/PekkoPersistenceConfigTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/PekkoPersistenceConfigTest.scala index b5d2b4fe..976dc0dd 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/PekkoPersistenceConfigTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/configuration/PekkoPersistenceConfigTest.scala @@ -5,8 +5,8 @@ package org.apache.pekko.persistence.postgres.configuration +import com.typesafe.config.{Config, ConfigFactory} import org.apache.pekko.persistence.postgres.config._ -import com.typesafe.config.{ Config, ConfigFactory } import org.scalatest.OptionValues import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/JournalPartitioningSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/JournalPartitioningSpec.scala index 753b1250..88aa4ba6 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/JournalPartitioningSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/JournalPartitioningSpec.scala @@ -1,24 +1,23 @@ package org.apache.pekko.persistence.postgres.journal -import java.util.UUID - -import org.apache.pekko.actor.{ Actor, ActorRef, ActorSystem } +import com.typesafe.config.{Config, ConfigFactory} +import org.apache.pekko.actor.{Actor, ActorRef, ActorSystem} +import org.apache.pekko.persistence.{AtomicWrite, Persistence, PersistentImpl, PersistentRepr} import org.apache.pekko.persistence.JournalProtocol._ import org.apache.pekko.persistence.postgres.config._ import org.apache.pekko.persistence.postgres.db.SlickExtension import org.apache.pekko.persistence.postgres.journal.JournalPartitioningSpec.HugeBatchSmallPartitionConfig +import org.apache.pekko.persistence.postgres.util.{ClasspathResources, DropCreate} import org.apache.pekko.persistence.postgres.util.Schema._ -import org.apache.pekko.persistence.postgres.util.{ ClasspathResources, DropCreate } -import org.apache.pekko.persistence.{ AtomicWrite, Persistence, PersistentImpl, PersistentRepr } import org.apache.pekko.testkit.TestProbe -import com.typesafe.config.{ Config, ConfigFactory } +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.concurrent.ScalaFutures import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec -import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } +import java.util.UUID +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ -import scala.concurrent.{ ExecutionContext, Future } object JournalPartitioningSpec { val HugeBatchSmallPartitionConfig: Config = ConfigFactory.parseString { @@ -72,7 +71,8 @@ abstract class JournalPartitioningSpec(schemaType: SchemaType) def supportsAtomicPersistAllOfSeveralEvents: Boolean = true def writeMessages(fromSnr: Int, toSnr: Int, pid: String, sender: ActorRef, writerUuid: String)( - journal: ActorRef): Unit = { + journal: ActorRef + ): Unit = { def persistentRepr(sequenceNr: Long) = PersistentRepr( @@ -80,7 +80,8 @@ abstract class JournalPartitioningSpec(schemaType: SchemaType) sequenceNr = sequenceNr, persistenceId = pid, sender = sender, - writerUuid = writerUuid) + writerUuid = writerUuid + ) val messages = if (supportsAtomicPersistAllOfSeveralEvents) { @@ -149,7 +150,8 @@ abstract class JournalPartitioningSpec(schemaType: SchemaType) snr: Long, pid: String, writerUuid: String, - deleted: Boolean = false): ReplayedMessage = + deleted: Boolean = false + ): ReplayedMessage = ReplayedMessage(PersistentImpl(s"a-$snr", snr, pid, "", deleted, Actor.noSender, writerUuid, 0L, None)) } diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/PostgresJournalPerfSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/PostgresJournalPerfSpec.scala index 3a646b9c..5bdfd20c 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/PostgresJournalPerfSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/PostgresJournalPerfSpec.scala @@ -5,18 +5,18 @@ package org.apache.pekko.persistence.postgres.journal +import com.typesafe.config.ConfigFactory import org.apache.pekko.actor.Props import org.apache.pekko.persistence.CapabilityFlag import org.apache.pekko.persistence.journal.JournalPerfSpec -import org.apache.pekko.persistence.journal.JournalPerfSpec.{ BenchActor, Cmd, ResetCounter } +import org.apache.pekko.persistence.journal.JournalPerfSpec.{BenchActor, Cmd, ResetCounter} import org.apache.pekko.persistence.postgres.config._ import org.apache.pekko.persistence.postgres.db.SlickExtension +import org.apache.pekko.persistence.postgres.util.{ClasspathResources, DropCreate} import org.apache.pekko.persistence.postgres.util.Schema._ -import org.apache.pekko.persistence.postgres.util.{ ClasspathResources, DropCreate } import org.apache.pekko.testkit.TestProbe -import com.typesafe.config.ConfigFactory +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.concurrent.ScalaFutures -import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import scala.concurrent.ExecutionContextExecutor import scala.concurrent.duration._ diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/PostgresJournalSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/PostgresJournalSpec.scala index e0dd7fe2..98b21549 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/PostgresJournalSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/PostgresJournalSpec.scala @@ -5,26 +5,31 @@ package org.apache.pekko.persistence.postgres.journal -import org.apache.pekko.actor.{ Actor, ActorRef } -import org.apache.pekko.persistence.JournalProtocol.{ ReplayedMessage, WriteMessages, WriteMessagesFailed, WriteMessagesSuccessful } +import com.typesafe.config.{Config, ConfigFactory} +import org.apache.pekko.actor.{Actor, ActorRef} +import org.apache.pekko.persistence.{AtomicWrite, CapabilityFlag, PersistentImpl, PersistentRepr} +import org.apache.pekko.persistence.JournalProtocol.{ + ReplayedMessage, + WriteMessages, + WriteMessagesFailed, + WriteMessagesSuccessful +} import org.apache.pekko.persistence.journal.JournalSpec import org.apache.pekko.persistence.postgres.config._ import org.apache.pekko.persistence.postgres.db.SlickExtension import org.apache.pekko.persistence.postgres.journal.dao.JournalMetadataTable import org.apache.pekko.persistence.postgres.query.ScalaPostgresReadJournalOperations +import org.apache.pekko.persistence.postgres.util.{ClasspathResources, DropCreate} import org.apache.pekko.persistence.postgres.util.Schema._ -import org.apache.pekko.persistence.postgres.util.{ ClasspathResources, DropCreate } import org.apache.pekko.persistence.query.Sequence -import org.apache.pekko.persistence.{ AtomicWrite, CapabilityFlag, PersistentImpl, PersistentRepr } import org.apache.pekko.testkit.TestProbe -import com.typesafe.config.{ Config, ConfigFactory } +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.concurrent.ScalaFutures -import org.scalatest.time.{ Minute, Span } -import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } +import org.scalatest.time.{Minute, Span} +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ -import scala.concurrent.{ ExecutionContext, Future } abstract class PostgresJournalSpec(config: String, schemaType: SchemaType) extends JournalSpec(ConfigFactory.load(config)) @@ -62,7 +67,9 @@ abstract class PostgresJournalSpec(config: String, schemaType: SchemaType) sequenceNr = seqNr, persistenceId = pid, sender = sender, - writerUuid = writerUuid)) + writerUuid = writerUuid + ) + ) val probe = TestProbe() journal ! WriteMessages(List(msg), probe.ref, actorInstanceId) probe.expectMsg(WriteMessagesSuccessful) @@ -85,7 +92,9 @@ abstract class PostgresJournalSpec(config: String, schemaType: SchemaType) sequenceNr = repeatedSnr, persistenceId = perId, sender = sender.ref, - writerUuid = writerUuid)) + writerUuid = writerUuid + ) + ) val probe = TestProbe() journal ! WriteMessages(Seq(msg), probe.ref, actorInstanceId) diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/ByteArrayJournalSerializerTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/ByteArrayJournalSerializerTest.scala index b2c714b0..99f8ec94 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/ByteArrayJournalSerializerTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/ByteArrayJournalSerializerTest.scala @@ -6,19 +6,18 @@ package org.apache.pekko.persistence.postgres package journal.dao -import java.nio.charset.Charset -import java.time.{ LocalDateTime, ZoneOffset } -import java.util.UUID - +import io.circe.Json +import org.apache.pekko.persistence.{AtomicWrite, PersistentRepr} import org.apache.pekko.persistence.journal.Tagged import org.apache.pekko.persistence.postgres.journal.dao.FakeTagIdResolver.unwanted1 import org.apache.pekko.persistence.postgres.tag.TagIdResolver -import org.apache.pekko.persistence.{ AtomicWrite, PersistentRepr } -import org.apache.pekko.serialization.{ Serializer, Serializers } -import io.circe.Json +import org.apache.pekko.serialization.{Serializer, Serializers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.matchers.must.Matchers +import java.nio.charset.Charset +import java.time.{LocalDateTime, ZoneOffset} +import java.util.UUID import scala.collection.immutable._ import scala.concurrent.Future @@ -86,7 +85,8 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal "sid" -> Json.fromInt(payloadSer.identifier), "wid" -> Json.fromString(repr.writerUuid), "t" -> Json.fromLong(repr.timestamp), - "em" -> Json.fromString("customManifest")) + "em" -> Json.fromString("customManifest") + ) } } @@ -97,7 +97,8 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal Json.obj( "sid" -> Json.fromInt(payloadSer.identifier), "wid" -> Json.fromString(repr.writerUuid), - "t" -> Json.fromLong(repr.timestamp)) + "t" -> Json.fromLong(repr.timestamp) + ) } } @@ -109,7 +110,8 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal Json.obj( "sid" -> Json.fromInt(payloadSer.identifier), "wid" -> Json.fromString(repr.writerUuid), - "t" -> Json.fromLong(repr.timestamp)) + "t" -> Json.fromLong(repr.timestamp) + ) } } } @@ -143,7 +145,9 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal "serManifest" -> Json.fromString(serManifest), "eventManifest" -> Json.fromString(eventManifest), "writerUuid" -> Json.fromString(writerUuid), - "timestamp" -> Json.fromLong(timestamp))) + "timestamp" -> Json.fromLong(timestamp) + ) + ) val repr = deserialized(meta) repr should equal { PersistentRepr(payload, 2137L, "my-7", eventManifest, false, null, writerUuid) @@ -161,7 +165,9 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal "sm" -> Json.fromString(serManifest), "em" -> Json.fromString(eventManifest), "wid" -> Json.fromString(writerUuid), - "t" -> Json.fromLong(timestamp))) + "t" -> Json.fromLong(timestamp) + ) + ) val repr = deserialized(meta) repr should equal { PersistentRepr(payload, 2137L, "my-7", eventManifest, false, null, writerUuid) @@ -173,7 +179,8 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal val timestamp = LocalDateTime.now().toEpochSecond(ZoneOffset.UTC) val meta = Json.fromFields( - List("sid" -> Json.fromLong(serId), "wid" -> Json.fromString(writerUuid), "t" -> Json.fromLong(timestamp))) + List("sid" -> Json.fromLong(serId), "wid" -> Json.fromString(writerUuid), "t" -> Json.fromLong(timestamp)) + ) val repr = deserialized(meta) repr should equal { PersistentRepr(payload, 2137L, "my-7", "", false, null, writerUuid) @@ -189,7 +196,9 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal "sid" -> Json.fromLong(serId), "wid" -> Json.fromString(writerUuid), "em" -> Json.fromString(""), - "t" -> Json.fromLong(timestamp))) + "t" -> Json.fromLong(timestamp) + ) + ) val repr = deserialized(meta) repr should equal { PersistentRepr(payload, 2137L, "my-7", "", false, null, writerUuid) @@ -208,7 +217,9 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal "serManifest" -> Json.fromString("broken"), "eventManifest" -> Json.fromString("this should be skipped"), "wid" -> Json.fromString(writerUuid), - "t" -> Json.fromLong(timestamp))) + "t" -> Json.fromLong(timestamp) + ) + ) val repr = deserialized(meta) repr should equal { PersistentRepr(payload, 2137L, "my-7", "", false, null, writerUuid) @@ -221,8 +232,8 @@ class ByteArrayJournalSerializerTest extends SharedActorSystemTestSpec with Scal class FakeTagIdResolver( getOrAssignIdsForF: Set[String] => Future[Map[String, Int]] = unwanted1("getOrAssignIdFor"), - lookupIdForF: String => Future[Option[Int]] = unwanted1("lookupIdFor")) - extends TagIdResolver { + lookupIdForF: String => Future[Option[Int]] = unwanted1("lookupIdFor") +) extends TagIdResolver { override def getOrAssignIdsFor(tags: Set[String]): Future[Map[String, Int]] = getOrAssignIdsForF(tags) override def lookupIdFor(name: String): Future[Option[Int]] = lookupIdForF(name) diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalMetadataQueriesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalMetadataQueriesTest.scala index 876114ea..c864094e 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalMetadataQueriesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalMetadataQueriesTest.scala @@ -6,12 +6,14 @@ class JournalMetadataQueriesTest extends BaseQueryTest { it should "create SQL query for highestSequenceNrForPersistenceId" in withJournalMetadataQueries { queries => queries.highestSequenceNrForPersistenceId( - "aaa") shouldBeSQL """select "max_sequence_number" from "journal_metadata" where "persistence_id" = ? limit 1""" + "aaa" + ) shouldBeSQL """select "max_sequence_number" from "journal_metadata" where "persistence_id" = ? limit 1""" } it should "create SQL query for minAndMaxOrderingForPersistenceId" in withJournalMetadataQueries { queries => queries.minAndMaxOrderingForPersistenceId( - "aaa") shouldBeSQL """select "min_ordering", "max_ordering" from "journal_metadata" where "persistence_id" = ? limit 1""" + "aaa" + ) shouldBeSQL """select "min_ordering", "max_ordering" from "journal_metadata" where "persistence_id" = ? limit 1""" } private def withJournalMetadataQueries(f: JournalMetadataQueries => Unit): Unit = { diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalQueriesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalQueriesTest.scala index d6f37649..0d113bb5 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalQueriesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalQueriesTest.scala @@ -1,19 +1,21 @@ package org.apache.pekko.persistence.postgres.journal.dao +import io.circe.{Json, JsonObject} import org.apache.pekko.persistence.postgres.JournalRow import org.apache.pekko.persistence.postgres.util.BaseQueryTest -import io.circe.{ Json, JsonObject } class JournalQueriesTest extends BaseQueryTest { it should "create SQL query for highestMarkedSequenceNrForPersistenceId" in withJournalQueries { queries => queries.highestMarkedSequenceNrForPersistenceId( - "aaa") shouldBeSQL """select max("sequence_number") from "journal" where ("deleted" = true) and ("persistence_id" = ?)""" + "aaa" + ) shouldBeSQL """select max("sequence_number") from "journal" where ("deleted" = true) and ("persistence_id" = ?)""" } it should "create SQL query for highestSequenceNrForPersistenceId" in withJournalQueries { queries => queries.highestSequenceNrForPersistenceId( - "aaa") shouldBeSQL """select max("sequence_number") from "journal" where "persistence_id" = ?""" + "aaa" + ) shouldBeSQL """select max("sequence_number") from "journal" where "persistence_id" = ?""" } it should "create SQL query for messagesQuery" in withJournalQueries { queries => @@ -21,7 +23,8 @@ class JournalQueriesTest extends BaseQueryTest { "aaa", 11L, 11L, - 11L) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where ((("persistence_id" = ?) and ("deleted" = false)) and ("sequence_number" >= ?)) and ("sequence_number" <= ?) order by "sequence_number" limit ?""" + 11L + ) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where ((("persistence_id" = ?) and ("deleted" = false)) and ("sequence_number" >= ?)) and ("sequence_number" <= ?) order by "sequence_number" limit ?""" } it should "create SQL query for messagesMinOrderingBoundedQuery" in withJournalQueries { queries => @@ -30,13 +33,15 @@ class JournalQueriesTest extends BaseQueryTest { 11L, 11L, 11L, - 11L) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where (((("persistence_id" = ?) and ("deleted" = false)) and ("sequence_number" >= ?)) and ("sequence_number" <= ?)) and ("ordering" >= ?) order by "sequence_number" limit ?""" + 11L + ) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where (((("persistence_id" = ?) and ("deleted" = false)) and ("sequence_number" >= ?)) and ("sequence_number" <= ?)) and ("ordering" >= ?) order by "sequence_number" limit ?""" } it should "create SQL query for markJournalMessagesAsDeleted" in withJournalQueries { queries => queries.markJournalMessagesAsDeleted( "aaa", - 11L) shouldBeSQL """update "journal" set "deleted" = ? where (("journal"."persistence_id" = 'aaa') and ("journal"."sequence_number" <= 11)) and ("journal"."deleted" = false)""" + 11L + ) shouldBeSQL """update "journal" set "deleted" = ? where (("journal"."persistence_id" = 'aaa') and ("journal"."sequence_number" <= 11)) and ("journal"."deleted" = false)""" } it should "create SQL query for update" in withJournalQueries { queries => @@ -44,19 +49,22 @@ class JournalQueriesTest extends BaseQueryTest { "aaa", 11L, Array.ofDim(0), - emptyJson) shouldBeSQL """update "journal" set "message" = ?, "metadata" = ? where ("journal"."persistence_id" = 'aaa') and ("journal"."sequence_number" = 11)""" + emptyJson + ) shouldBeSQL """update "journal" set "message" = ?, "metadata" = ? where ("journal"."persistence_id" = 'aaa') and ("journal"."sequence_number" = 11)""" } it should "create SQL query for delete" in withJournalQueries { queries => queries.delete( "aaa", - 11L) shouldBeSQL """delete from "journal" where ("journal"."persistence_id" = 'aaa') and ("journal"."sequence_number" <= 11)""" + 11L + ) shouldBeSQL """delete from "journal" where ("journal"."persistence_id" = 'aaa') and ("journal"."sequence_number" <= 11)""" } it should "create SQL query for writeJournalRows" in withJournalQueries { queries => val row = JournalRow(1L, deleted = false, "p", 3L, Array.ofDim(0), List(1, 2, 3), emptyJson) queries.writeJournalRows( - Seq(row, row, row)) shouldBeSQL """insert into "journal" ("deleted","persistence_id","sequence_number","message","tags","metadata") values (?,?,?,?,?,?)""" + Seq(row, row, row) + ) shouldBeSQL """insert into "journal" ("deleted","persistence_id","sequence_number","message","tags","metadata") values (?,?,?,?,?,?)""" } private lazy val emptyJson = Json.fromJsonObject(JsonObject.empty) diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalTablesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalTablesTest.scala index c3c968f5..b4e80c77 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalTablesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/journal/dao/JournalTablesTest.scala @@ -14,7 +14,8 @@ class JournalTablesTest extends TablesTestSpec { (journalName, journalTable) <- List( ("FlatJournalTable", FlatJournalTable(journalTableConfiguration)), ("PartitionedJournalTable", PartitionedJournalTable(journalTableConfiguration)), - ("NestedPartitionsJournalTable", NestedPartitionsJournalTable(journalTableConfiguration))) + ("NestedPartitionsJournalTable", NestedPartitionsJournalTable(journalTableConfiguration)) + ) } { journalName should "be configured with a schema name" in { journalTable.baseTableRow.schemaName shouldBe journalTableConfiguration.schemaName @@ -28,10 +29,12 @@ class JournalTablesTest extends TablesTestSpec { it should "be configured with column names" in { val colName = toColumnName(journalTableConfiguration.tableName)(_) journalTable.baseTableRow.persistenceId.toString shouldBe colName( - journalTableConfiguration.columnNames.persistenceId) + journalTableConfiguration.columnNames.persistenceId + ) journalTable.baseTableRow.deleted.toString shouldBe colName(journalTableConfiguration.columnNames.deleted) journalTable.baseTableRow.sequenceNumber.toString shouldBe colName( - journalTableConfiguration.columnNames.sequenceNumber) + journalTableConfiguration.columnNames.sequenceNumber + ) journalTable.baseTableRow.tags.toString shouldBe colName(journalTableConfiguration.columnNames.tags) } } @@ -50,12 +53,16 @@ class JournalTablesTest extends TablesTestSpec { it should "be configured with column names" in { val colName = toColumnName(journalMetadataTableConfiguration.tableName)(_) journalMetadataTable.baseTableRow.persistenceId.toString shouldBe colName( - journalMetadataTableConfiguration.columnNames.persistenceId) + journalMetadataTableConfiguration.columnNames.persistenceId + ) journalMetadataTable.baseTableRow.maxSequenceNumber.toString shouldBe colName( - journalMetadataTableConfiguration.columnNames.maxSequenceNumber) + journalMetadataTableConfiguration.columnNames.maxSequenceNumber + ) journalMetadataTable.baseTableRow.maxOrdering.toString shouldBe colName( - journalMetadataTableConfiguration.columnNames.maxOrdering) + journalMetadataTableConfiguration.columnNames.maxOrdering + ) journalMetadataTable.baseTableRow.minOrdering.toString shouldBe colName( - journalMetadataTableConfiguration.columnNames.minOrdering) + journalMetadataTableConfiguration.columnNames.minOrdering + ) } } diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/AllPersistenceIdsTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/AllPersistenceIdsTest.scala index 8f89e35a..e1da08be 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/AllPersistenceIdsTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/AllPersistenceIdsTest.scala @@ -5,7 +5,7 @@ package org.apache.pekko.persistence.postgres.query -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} import scala.concurrent.duration._ diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByPersistenceIdTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByPersistenceIdTest.scala index 2f5b4c8d..d6ca0964 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByPersistenceIdTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByPersistenceIdTest.scala @@ -8,8 +8,8 @@ package org.apache.pekko.persistence.postgres.query import org.apache.pekko.Done import org.apache.pekko.persistence.Persistence import org.apache.pekko.persistence.postgres.journal.PostgresAsyncWriteJournal -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } -import org.apache.pekko.persistence.query.{ EventEnvelope, Offset, Sequence } +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} +import org.apache.pekko.persistence.query.{EventEnvelope, Offset, Sequence} import org.apache.pekko.testkit.TestProbe abstract class CurrentEventsByPersistenceIdTest(val schemaType: SchemaType) diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByTagTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByTagTest.scala index def761ab..1da5f7a3 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByTagTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByTagTest.scala @@ -5,13 +5,13 @@ package org.apache.pekko.persistence.postgres.query +import com.typesafe.config.{Config, ConfigFactory, ConfigValue, ConfigValueFactory} import org.apache.pekko.Done import org.apache.pekko.pattern.ask import org.apache.pekko.persistence.postgres.query.CurrentEventsByTagTest._ -import org.apache.pekko.persistence.postgres.query.EventAdapterTest.{ Event, TaggedAsyncEvent } -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } -import org.apache.pekko.persistence.query.{ EventEnvelope, NoOffset, Sequence } -import com.typesafe.config.{ Config, ConfigFactory, ConfigValue, ConfigValueFactory } +import org.apache.pekko.persistence.postgres.query.EventAdapterTest.{Event, TaggedAsyncEvent} +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} +import org.apache.pekko.persistence.query.{EventEnvelope, NoOffset, Sequence} import scala.concurrent.Future import scala.concurrent.duration._ @@ -22,7 +22,8 @@ object CurrentEventsByTagTest { val configOverrides: Map[String, ConfigValue] = Map( "postgres-read-journal.max-buffer-size" -> ConfigValueFactory.fromAnyRef(maxBufferSize.toString), - "postgres-read-journal.refresh-interval" -> ConfigValueFactory.fromAnyRef(refreshInterval.toString())) + "postgres-read-journal.refresh-interval" -> ConfigValueFactory.fromAnyRef(refreshInterval.toString()) + ) case class TestEvent(greetings: String) @@ -176,7 +177,8 @@ abstract class CurrentEventsByTagTest(val schemaType: SchemaType) import scala.jdk.CollectionConverters._ it should "complete without any gaps in case events are being persisted when the query is executed" in withActorSystem( - withMaxBufferSize(1000)) { implicit system => + withMaxBufferSize(1000) + ) { implicit system => val journalOps = new JavaDslPostgresReadJournalOperations(system) import system.dispatcher withTestActors(replyToMessages = true) { (actor1, actor2, actor3) => @@ -215,7 +217,8 @@ abstract class CurrentEventsByTagTest(val schemaType: SchemaType) } it should "complete without omitting any events in case events are being persisted when the query is executed" in withActorSystem( - withMaxBufferSize((batch1Size + batch2Size) * (numOfActors + 1))) { implicit system => + withMaxBufferSize((batch1Size + batch2Size) * (numOfActors + 1)) + ) { implicit system => val journalOps = new JavaDslPostgresReadJournalOperations(system) import system.dispatcher withTestActors(replyToMessages = true) { (actor1, actor2, actor3) => diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByTagWithGapsTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByTagWithGapsTest.scala index 3929af84..a697c711 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByTagWithGapsTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentEventsByTagWithGapsTest.scala @@ -1,15 +1,15 @@ package org.apache.pekko.persistence.postgres.query +import com.typesafe.config.{ConfigValue, ConfigValueFactory} import org.apache.pekko.actor.ActorSystem import org.apache.pekko.persistence.PersistentRepr -import org.apache.pekko.persistence.postgres.journal.dao.{ ByteArrayJournalSerializer, JournalQueries } -import org.apache.pekko.persistence.postgres.tag.{ CachedTagIdResolver, SimpleTagDao } +import org.apache.pekko.persistence.postgres.journal.dao.{ByteArrayJournalSerializer, JournalQueries} +import org.apache.pekko.persistence.postgres.tag.{CachedTagIdResolver, SimpleTagDao} import org.apache.pekko.persistence.postgres.util.Schema import org.apache.pekko.persistence.postgres.util.Schema.SchemaType import org.apache.pekko.persistence.query.NoOffset import org.apache.pekko.serialization.SerializationExtension -import org.apache.pekko.stream.scaladsl.{ Sink, Source } -import com.typesafe.config.{ ConfigValue, ConfigValueFactory } +import org.apache.pekko.stream.scaladsl.{Sink, Source} import scala.concurrent.duration._ @@ -19,13 +19,15 @@ object CurrentEventsByTagWithGapsTest { val configOverrides: Map[String, ConfigValue] = Map( "postgres-read-journal.max-buffer-size" -> ConfigValueFactory.fromAnyRef(maxBufferSize.toString), - "postgres-read-journal.refresh-interval" -> ConfigValueFactory.fromAnyRef(refreshInterval.toString())) + "postgres-read-journal.refresh-interval" -> ConfigValueFactory.fromAnyRef(refreshInterval.toString()) + ) } class CurrentEventsByTagWithGapsTest extends QueryTestSpec( s"${Schema.Partitioned.resourceNamePrefix}-shared-db-application.conf", - CurrentEventsByTagWithGapsTest.configOverrides) { + CurrentEventsByTagWithGapsTest.configOverrides + ) { // We are using Partitioned variant because it does not override values for an `ordering` field override val schemaType: SchemaType = Schema.Partitioned @@ -53,7 +55,8 @@ class CurrentEventsByTagWithGapsTest val tagDao = new SimpleTagDao(db, journalConfig.tagsTableConfiguration) val serializer = new ByteArrayJournalSerializer( SerializationExtension(system), - new CachedTagIdResolver(tagDao, journalConfig.tagsConfig)) + new CachedTagIdResolver(tagDao, journalConfig.tagsConfig) + ) val numElements = 1000 val gapSize = 10000 diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentPersistenceIdsTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentPersistenceIdsTest.scala index 57e42d09..9652ff53 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentPersistenceIdsTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/CurrentPersistenceIdsTest.scala @@ -5,7 +5,7 @@ package org.apache.pekko.persistence.postgres.query -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} abstract class CurrentPersistenceIdsTest(val schemaType: SchemaType) extends QueryTestSpec(s"${schemaType.resourceNamePrefix}-shared-db-application.conf") { diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventAdapterTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventAdapterTest.scala index 9859526c..e4908261 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventAdapterTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventAdapterTest.scala @@ -6,9 +6,9 @@ package org.apache.pekko.persistence.postgres.query import org.apache.pekko.pattern.ask -import org.apache.pekko.persistence.journal.{ EventSeq, ReadEventAdapter, Tagged, WriteEventAdapter } -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } -import org.apache.pekko.persistence.query.{ EventEnvelope, NoOffset, Sequence } +import org.apache.pekko.persistence.journal.{EventSeq, ReadEventAdapter, Tagged, WriteEventAdapter} +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} +import org.apache.pekko.persistence.query.{EventEnvelope, NoOffset, Sequence} import scala.concurrent.duration._ @@ -45,9 +45,8 @@ object EventAdapterTest { } } -/** - * Tests that check persistence queries when event adapter is configured for persisted event. - */ +/** Tests that check persistence queries when event adapter is configured for persisted event. + */ abstract class EventAdapterTest(val schemaType: SchemaType) extends QueryTestSpec(schemaType.configName) { import EventAdapterTest._ diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventsByPersistenceIdTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventsByPersistenceIdTest.scala index f84178c7..77b498be 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventsByPersistenceIdTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventsByPersistenceIdTest.scala @@ -7,9 +7,9 @@ package org.apache.pekko.persistence.postgres.query import org.apache.pekko.Done import org.apache.pekko.pattern.ask -import org.apache.pekko.persistence.postgres.query.EventAdapterTest.{ Event, TaggedAsyncEvent } -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } -import org.apache.pekko.persistence.query.{ EventEnvelope, Offset, Sequence } +import org.apache.pekko.persistence.postgres.query.EventAdapterTest.{Event, TaggedAsyncEvent} +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} +import org.apache.pekko.persistence.query.{EventEnvelope, Offset, Sequence} import scala.concurrent.Future import scala.concurrent.duration._ diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventsByTagTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventsByTagTest.scala index 64ec60d6..b5f963c7 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventsByTagTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/EventsByTagTest.scala @@ -5,13 +5,18 @@ package org.apache.pekko.persistence.postgres.query +import com.typesafe.config.{ConfigValue, ConfigValueFactory} import org.apache.pekko.Done import org.apache.pekko.pattern.ask -import org.apache.pekko.persistence.postgres.query.EventAdapterTest.{ Event, EventRestored, TaggedAsyncEvent, TaggedEvent } +import org.apache.pekko.persistence.postgres.query.EventAdapterTest.{ + Event, + EventRestored, + TaggedAsyncEvent, + TaggedEvent +} import org.apache.pekko.persistence.postgres.query.EventsByTagTest._ -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } -import org.apache.pekko.persistence.query.{ EventEnvelope, NoOffset, Sequence } -import com.typesafe.config.{ ConfigValue, ConfigValueFactory } +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} +import org.apache.pekko.persistence.query.{EventEnvelope, NoOffset, Sequence} import scala.concurrent.Future import scala.concurrent.duration._ @@ -22,7 +27,8 @@ object EventsByTagTest { val configOverrides: Map[String, ConfigValue] = Map( "postgres-read-journal.max-buffer-size" -> ConfigValueFactory.fromAnyRef(maxBufferSize.toString), - "postgres-read-journal.refresh-interval" -> ConfigValueFactory.fromAnyRef(refreshInterval.toString())) + "postgres-read-journal.refresh-interval" -> ConfigValueFactory.fromAnyRef(refreshInterval.toString()) + ) } abstract class EventsByTagTest(val schemaType: SchemaType) @@ -121,10 +127,8 @@ abstract class EventsByTagTest(val schemaType: SchemaType) } it should "deliver EventEnvelopes non-zero timestamps" in withActorSystem { implicit system => - val journalOps = new ScalaPostgresReadJournalOperations(system) withTestActors(replyToMessages = true) { (actor1, actor2, actor3) => - val testStartTime = System.currentTimeMillis() (actor1 ? withTags(1, "number")).futureValue diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/HardDeleteQueryTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/HardDeleteQueryTest.scala index d7d1b540..4e727dc8 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/HardDeleteQueryTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/HardDeleteQueryTest.scala @@ -6,7 +6,7 @@ package org.apache.pekko.persistence.postgres.query import org.apache.pekko.pattern._ -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} import org.apache.pekko.persistence.query.NoOffset import org.scalatest.matchers.should.Matchers diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/JournalDaoStreamMessagesMemoryTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/JournalDaoStreamMessagesMemoryTest.scala index 9d14ece6..a338baa9 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/JournalDaoStreamMessagesMemoryTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/JournalDaoStreamMessagesMemoryTest.scala @@ -5,34 +5,34 @@ package org.apache.pekko.persistence.postgres.query -import java.lang.management.{ ManagementFactory, MemoryMXBean } -import java.util.UUID - -import org.apache.pekko.actor.{ ActorSystem, ExtendedActorSystem } +import com.typesafe.config.{ConfigValue, ConfigValueFactory} +import org.apache.pekko.actor.{ActorSystem, ExtendedActorSystem} +import org.apache.pekko.persistence.{AtomicWrite, PersistentRepr} import org.apache.pekko.persistence.postgres.config.JournalConfig import org.apache.pekko.persistence.postgres.journal.dao.JournalDao -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } -import org.apache.pekko.persistence.{ AtomicWrite, PersistentRepr } -import org.apache.pekko.serialization.{ Serialization, SerializationExtension } -import org.apache.pekko.stream.scaladsl.{ Sink, Source } +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} +import org.apache.pekko.serialization.{Serialization, SerializationExtension} +import org.apache.pekko.stream.{Materializer, SystemMaterializer} +import org.apache.pekko.stream.scaladsl.{Sink, Source} import org.apache.pekko.stream.testkit.scaladsl.TestSink -import org.apache.pekko.stream.{ Materializer, SystemMaterializer } -import com.typesafe.config.{ ConfigValue, ConfigValueFactory } import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.matchers.should import org.slf4j.LoggerFactory import slick.jdbc.JdbcBackend.Database +import java.lang.management.{ManagementFactory, MemoryMXBean} +import java.util.UUID import scala.collection.immutable import scala.collection.immutable.Seq +import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} import scala.concurrent.duration._ -import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor } -import scala.util.{ Failure, Success } +import scala.util.{Failure, Success} object JournalDaoStreamMessagesMemoryTest { val configOverrides: Map[String, ConfigValue] = Map( - "postgres-journal.fetch-size" -> ConfigValueFactory.fromAnyRef("100")) + "postgres-journal.fetch-size" -> ConfigValueFactory.fromAnyRef("100") + ) val MB = 1024 * 1024 } @@ -69,7 +69,8 @@ abstract class JournalDaoStreamMessagesMemoryTest(val schemaType: SchemaType) (classOf[JournalConfig], journalConfig), (classOf[Serialization], SerializationExtension(system)), (classOf[ExecutionContext], ec), - (classOf[Materializer], mat)) + (classOf[Materializer], mat) + ) system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[JournalDao](fqcn, args) match { case Success(dao) => dao case Failure(cause) => throw cause @@ -88,7 +89,8 @@ abstract class JournalDaoStreamMessagesMemoryTest(val schemaType: SchemaType) val totalMessages = numberOfInsertBatches * eventsPerBatch val totalMessagePayload = totalMessages * payloadSize log.info( - s"batches: $numberOfInsertBatches (with $eventsPerBatch events), total messages: $totalMessages, total msgs size: $totalMessagePayload") + s"batches: $numberOfInsertBatches (with $eventsPerBatch events), total messages: $totalMessages, total msgs size: $totalMessagePayload" + ) // payload can be the same when inserting to avoid unnecessary memory usage val payload = Array.fill(payloadSize)('a'.toByte) diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/JournalSequenceActorTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/JournalSequenceActorTest.scala index 23752062..ade26d39 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/JournalSequenceActorTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/JournalSequenceActorTest.scala @@ -5,26 +5,30 @@ package org.apache.pekko.persistence.postgres.query -import java.util.concurrent.atomic.AtomicLong -import org.apache.pekko.actor.{ ActorRef, ActorSystem } +import io.circe.{Json, JsonObject} +import org.apache.pekko.actor.{ActorRef, ActorSystem} import org.apache.pekko.pattern.ask +import org.apache.pekko.persistence.postgres.{JournalRow, SharedActorSystemTestSpec} import org.apache.pekko.persistence.postgres.config.JournalSequenceRetrievalConfig import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile -import org.apache.pekko.persistence.postgres.query.JournalSequenceActor.{ GetMaxOrderingId, MaxOrderingId } -import org.apache.pekko.persistence.postgres.query.dao.{ FlatReadJournalDao, PartitionedReadJournalDao, TestProbeReadJournalDao } -import org.apache.pekko.persistence.postgres.tag.{ CachedTagIdResolver, SimpleTagDao } -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } -import org.apache.pekko.persistence.postgres.{ JournalRow, SharedActorSystemTestSpec } +import org.apache.pekko.persistence.postgres.query.JournalSequenceActor.{GetMaxOrderingId, MaxOrderingId} +import org.apache.pekko.persistence.postgres.query.dao.{ + FlatReadJournalDao, + PartitionedReadJournalDao, + TestProbeReadJournalDao +} +import org.apache.pekko.persistence.postgres.tag.{CachedTagIdResolver, SimpleTagDao} +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} import org.apache.pekko.serialization.SerializationExtension -import org.apache.pekko.stream.scaladsl.{ Sink, Source } -import org.apache.pekko.stream.{ Materializer, SystemMaterializer } +import org.apache.pekko.stream.{Materializer, SystemMaterializer} +import org.apache.pekko.stream.scaladsl.{Sink, Source} import org.apache.pekko.testkit.TestProbe -import io.circe.{ Json, JsonObject } import org.scalatest.time.Span import org.slf4j.LoggerFactory import slick.jdbc -import slick.jdbc.{ JdbcBackend, JdbcCapabilities } +import slick.jdbc.{JdbcBackend, JdbcCapabilities} +import java.util.concurrent.atomic.AtomicLong import scala.concurrent.Future import scala.concurrent.duration._ @@ -166,13 +170,13 @@ abstract class JournalSequenceActorTest(val schemaType: SchemaType) extends Quer } } - /** - * @param maxTries The number of tries before events are assumed missing - * (since the actor queries every second by default, - * this is effectively the number of seconds after which events are assumed missing) - */ - def withJournalSequenceActor(db: JdbcBackend.Database, maxTries: Int)(f: ActorRef => Unit)( - implicit system: ActorSystem): Unit = { + /** @param maxTries + * The number of tries before events are assumed missing (since the actor queries every second by default, this is + * effectively the number of seconds after which events are assumed missing) + */ + def withJournalSequenceActor(db: JdbcBackend.Database, maxTries: Int)( + f: ActorRef => Unit + )(implicit system: ActorSystem): Unit = { import system.dispatcher implicit val mat: Materializer = SystemMaterializer(system).materializer val readJournalDao = @@ -182,7 +186,9 @@ abstract class JournalSequenceActorTest(val schemaType: SchemaType) extends Quer SerializationExtension(system), new CachedTagIdResolver( new SimpleTagDao(db, readJournalConfig.tagsTableConfiguration), - readJournalConfig.tagsConfig)) + readJournalConfig.tagsConfig + ) + ) val actor = system.actorOf(JournalSequenceActor.props(readJournalDao, journalSequenceActorConfig.copy(maxTries = maxTries))) try f(actor) @@ -221,7 +227,8 @@ class MockDaoJournalSequenceActorTest extends SharedActorSystemTestSpec { daoProbe.reply(thirdBatch) withClue( s"when no more events are missing, but less that batchSize elemens have been received, " + - s"the actor should wait for $queryDelay before querying again") { + s"the actor should wait for $queryDelay before querying again" + ) { daoProbe.expectNoMessage(almostQueryDelay) daoProbe.expectMsg(almostQueryDelay, TestProbeReadJournalDao.JournalSequence(110, batchSize)) } @@ -230,7 +237,8 @@ class MockDaoJournalSequenceActorTest extends SharedActorSystemTestSpec { daoProbe.reply(fourthBatch) withClue( "When no more events are missing and the number of events received is equal to batchSize, " + - "the actor should query again immediately") { + "the actor should query again immediately" + ) { daoProbe.expectMsg(almostImmediately, TestProbeReadJournalDao.JournalSequence(210, batchSize)) } @@ -265,7 +273,8 @@ class MockDaoJournalSequenceActorTest extends SharedActorSystemTestSpec { retryResponse.last shouldBe 142 withClue( "The elements 41 and 42 should be assumed missing, " + - "the actor should query again immediately since a full batch has been received") { + "the actor should query again immediately since a full batch has been received" + ) { daoProbe.expectMsg(almostImmediately, TestProbeReadJournalDao.JournalSequence(142, batchSize)) fetchMaxOrderingId(actor).futureValue shouldBe 142 } @@ -277,14 +286,16 @@ class MockDaoJournalSequenceActorTest extends SharedActorSystemTestSpec { } def withTestProbeJournalSequenceActor(batchSize: Int, maxTries: Int, queryDelay: FiniteDuration)( - f: (TestProbe, ActorRef) => Unit)(implicit system: ActorSystem): Unit = { + f: (TestProbe, ActorRef) => Unit + )(implicit system: ActorSystem): Unit = { val testProbe = TestProbe() val config = JournalSequenceRetrievalConfig( batchSize = batchSize, maxTries = maxTries, queryDelay = queryDelay, maxBackoffQueryDelay = 4.seconds, - askTimeout = 100.millis) + askTimeout = 100.millis + ) val mockDao = new TestProbeReadJournalDao(testProbe) val actor = system.actorOf(JournalSequenceActor.props(mockDao, config)) try f(testProbe, actor) @@ -317,8 +328,9 @@ class PartitionedJournalSequenceActorTest extends JournalSequenceActorTest(Parti } } - override def withJournalSequenceActor(db: jdbc.JdbcBackend.Database, maxTries: Int)(f: ActorRef => Unit)( - implicit system: ActorSystem): Unit = { + override def withJournalSequenceActor(db: jdbc.JdbcBackend.Database, maxTries: Int)( + f: ActorRef => Unit + )(implicit system: ActorSystem): Unit = { import system.dispatcher implicit val mat: Materializer = SystemMaterializer(system).materializer val readJournalDao = @@ -328,11 +340,14 @@ class PartitionedJournalSequenceActorTest extends JournalSequenceActorTest(Parti SerializationExtension(system), new CachedTagIdResolver( new SimpleTagDao(db, readJournalConfig.tagsTableConfiguration), - readJournalConfig.tagsConfig)) + readJournalConfig.tagsConfig + ) + ) val actor = system.actorOf( JournalSequenceActor - .props(readJournalDao, readJournalConfig.journalSequenceRetrievalConfiguration.copy(maxTries = maxTries))) + .props(readJournalDao, readJournalConfig.journalSequenceRetrievalConfiguration.copy(maxTries = maxTries)) + ) try f(actor) finally system.stop(actor) } diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/LogicalDeleteQueryTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/LogicalDeleteQueryTest.scala index 183ceffd..85702f12 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/LogicalDeleteQueryTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/LogicalDeleteQueryTest.scala @@ -6,8 +6,8 @@ package org.apache.pekko.persistence.postgres.query import org.apache.pekko.pattern._ -import org.apache.pekko.persistence.postgres.util.Schema.{ NestedPartitions, Partitioned, Plain, SchemaType } -import org.apache.pekko.persistence.query.{ EventEnvelope, NoOffset, Sequence } +import org.apache.pekko.persistence.postgres.util.Schema.{NestedPartitions, Partitioned, Plain, SchemaType} +import org.apache.pekko.persistence.query.{EventEnvelope, NoOffset, Sequence} import scala.concurrent.duration._ diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/QueryTestSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/QueryTestSpec.scala index 7c67d0fa..459a325f 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/QueryTestSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/QueryTestSpec.scala @@ -5,53 +5,58 @@ package org.apache.pekko.persistence.postgres.query -import org.apache.pekko.actor.{ ActorRef, ActorSystem, Props, Stash, Status } +import com.typesafe.config.ConfigValue +import org.apache.pekko.actor.{ActorRef, ActorSystem, Props, Stash, Status} import org.apache.pekko.event.LoggingReceive +import org.apache.pekko.persistence.{DeleteMessagesFailure, DeleteMessagesSuccess, PersistentActor} import org.apache.pekko.persistence.journal.Tagged import org.apache.pekko.persistence.postgres.SingleActorSystemPerTestSpec -import org.apache.pekko.persistence.postgres.query.EventAdapterTest.{ Event, TaggedAsyncEvent, TaggedEvent } -import org.apache.pekko.persistence.postgres.query.javadsl.{ PostgresReadJournal => JavaPostgresReadJournal } +import org.apache.pekko.persistence.postgres.query.EventAdapterTest.{Event, TaggedAsyncEvent, TaggedEvent} +import org.apache.pekko.persistence.postgres.query.javadsl.{PostgresReadJournal => JavaPostgresReadJournal} import org.apache.pekko.persistence.postgres.query.scaladsl.PostgresReadJournal import org.apache.pekko.persistence.postgres.util.Schema.SchemaType -import org.apache.pekko.persistence.query.{ EventEnvelope, Offset, PersistenceQuery } -import org.apache.pekko.persistence.{ DeleteMessagesFailure, DeleteMessagesSuccess, PersistentActor } +import org.apache.pekko.persistence.query.{EventEnvelope, Offset, PersistenceQuery} +import org.apache.pekko.stream.{Materializer, SystemMaterializer} import org.apache.pekko.stream.scaladsl.Sink import org.apache.pekko.stream.testkit.TestSubscriber -import org.apache.pekko.stream.testkit.javadsl.{ TestSink => JavaSink } +import org.apache.pekko.stream.testkit.javadsl.{TestSink => JavaSink} import org.apache.pekko.stream.testkit.scaladsl.TestSink -import org.apache.pekko.stream.{ Materializer, SystemMaterializer } -import com.typesafe.config.ConfigValue import slick.jdbc.PostgresProfile.api._ import scala.concurrent.Future -import scala.concurrent.duration.{ FiniteDuration, _ } +import scala.concurrent.duration.{FiniteDuration, _} trait ReadJournalOperations { def withCurrentPersistenceIds(within: FiniteDuration = 60.second)(f: TestSubscriber.Probe[String] => Unit): Unit def withPersistenceIds(within: FiniteDuration = 60.second)(f: TestSubscriber.Probe[String] => Unit): Unit - def withCurrentEventsByPersistenceId(within: FiniteDuration = 60.second)( - persistenceId: String, - fromSequenceNr: Long = 0, - toSequenceNr: Long = Long.MaxValue)(f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit - def withEventsByPersistenceId(within: FiniteDuration = 60.second)( - persistenceId: String, - fromSequenceNr: Long = 0, - toSequenceNr: Long = Long.MaxValue)(f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit + def withCurrentEventsByPersistenceId( + within: FiniteDuration = 60.second + )(persistenceId: String, fromSequenceNr: Long = 0, toSequenceNr: Long = Long.MaxValue)( + f: TestSubscriber.Probe[EventEnvelope] => Unit + ): Unit + def withEventsByPersistenceId( + within: FiniteDuration = 60.second + )(persistenceId: String, fromSequenceNr: Long = 0, toSequenceNr: Long = Long.MaxValue)( + f: TestSubscriber.Probe[EventEnvelope] => Unit + ): Unit def withCurrentEventsByTag(within: FiniteDuration = 60.second)(tag: String, offset: Offset)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit + f: TestSubscriber.Probe[EventEnvelope] => Unit + ): Unit def withEventsByTag(within: FiniteDuration = 60.second)(tag: String, offset: Offset)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit + f: TestSubscriber.Probe[EventEnvelope] => Unit + ): Unit def countJournal: Future[Long] } -class ScalaPostgresReadJournalOperations(readJournal: PostgresReadJournal)( - implicit system: ActorSystem, - mat: Materializer) - extends ReadJournalOperations { +class ScalaPostgresReadJournalOperations(readJournal: PostgresReadJournal)(implicit + system: ActorSystem, + mat: Materializer +) extends ReadJournalOperations { def this(system: ActorSystem) = this(PersistenceQuery(system).readJournalFor[PostgresReadJournal](PostgresReadJournal.Identifier))( system, - SystemMaterializer(system).materializer) + SystemMaterializer(system).materializer + ) import system.dispatcher @@ -66,8 +71,10 @@ class ScalaPostgresReadJournalOperations(readJournal: PostgresReadJournal)( } def withCurrentEventsByPersistenceId( - within: FiniteDuration)(persistenceId: String, fromSequenceNr: Long = 0, toSequenceNr: Long = Long.MaxValue)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { + within: FiniteDuration + )(persistenceId: String, fromSequenceNr: Long = 0, toSequenceNr: Long = Long.MaxValue)( + f: TestSubscriber.Probe[EventEnvelope] => Unit + ): Unit = { val tp = readJournal .currentEventsByPersistenceId(persistenceId, fromSequenceNr, toSequenceNr) .runWith(TestSink.probe[EventEnvelope]) @@ -75,22 +82,26 @@ class ScalaPostgresReadJournalOperations(readJournal: PostgresReadJournal)( } def withEventsByPersistenceId( - within: FiniteDuration)(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { + within: FiniteDuration + )(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long)( + f: TestSubscriber.Probe[EventEnvelope] => Unit + ): Unit = { val tp = readJournal .eventsByPersistenceId(persistenceId, fromSequenceNr, toSequenceNr) .runWith(TestSink.probe[EventEnvelope]) tp.within(within)(f(tp)) } - def withCurrentEventsByTag(within: FiniteDuration)(tag: String, offset: Offset)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { + def withCurrentEventsByTag( + within: FiniteDuration + )(tag: String, offset: Offset)(f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { val tp = readJournal.currentEventsByTag(tag, offset).runWith(TestSink.probe[EventEnvelope]) tp.within(within)(f(tp)) } - def withEventsByTag(within: FiniteDuration)(tag: String, offset: Offset)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { + def withEventsByTag( + within: FiniteDuration + )(tag: String, offset: Offset)(f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { val tp = readJournal.eventsByTag(tag, offset).runWith(TestSink.probe[EventEnvelope]) tp.within(within)(f(tp)) } @@ -106,17 +117,16 @@ class ScalaPostgresReadJournalOperations(readJournal: PostgresReadJournal)( .map(_.sum) } -class JavaDslPostgresReadJournalOperations(readJournal: javadsl.PostgresReadJournal)( - implicit system: ActorSystem, - mat: Materializer) - extends ReadJournalOperations { +class JavaDslPostgresReadJournalOperations(readJournal: javadsl.PostgresReadJournal)(implicit + system: ActorSystem, + mat: Materializer +) extends ReadJournalOperations { def this(system: ActorSystem) = this( PersistenceQuery .get(system) - .getReadJournalFor(classOf[javadsl.PostgresReadJournal], JavaPostgresReadJournal.Identifier))( - system, - SystemMaterializer(system).materializer) + .getReadJournalFor(classOf[javadsl.PostgresReadJournal], JavaPostgresReadJournal.Identifier) + )(system, SystemMaterializer(system).materializer) import system.dispatcher @@ -133,31 +143,41 @@ class JavaDslPostgresReadJournalOperations(readJournal: javadsl.PostgresReadJour } def withCurrentEventsByPersistenceId( - within: FiniteDuration)(persistenceId: String, fromSequenceNr: Long = 0, toSequenceNr: Long = Long.MaxValue)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { - val sink: org.apache.pekko.stream.javadsl.Sink[EventEnvelope, TestSubscriber.Probe[EventEnvelope]] = JavaSink.probe(system) + within: FiniteDuration + )(persistenceId: String, fromSequenceNr: Long = 0, toSequenceNr: Long = Long.MaxValue)( + f: TestSubscriber.Probe[EventEnvelope] => Unit + ): Unit = { + val sink: org.apache.pekko.stream.javadsl.Sink[EventEnvelope, TestSubscriber.Probe[EventEnvelope]] = + JavaSink.probe(system) val tp = readJournal.currentEventsByPersistenceId(persistenceId, fromSequenceNr, toSequenceNr).runWith(sink, mat) tp.within(within)(f(tp)) } def withEventsByPersistenceId( - within: FiniteDuration)(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { - val sink: org.apache.pekko.stream.javadsl.Sink[EventEnvelope, TestSubscriber.Probe[EventEnvelope]] = JavaSink.probe(system) + within: FiniteDuration + )(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long)( + f: TestSubscriber.Probe[EventEnvelope] => Unit + ): Unit = { + val sink: org.apache.pekko.stream.javadsl.Sink[EventEnvelope, TestSubscriber.Probe[EventEnvelope]] = + JavaSink.probe(system) val tp = readJournal.eventsByPersistenceId(persistenceId, fromSequenceNr, toSequenceNr).runWith(sink, mat) tp.within(within)(f(tp)) } - def withCurrentEventsByTag(within: FiniteDuration)(tag: String, offset: Offset)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { - val sink: org.apache.pekko.stream.javadsl.Sink[EventEnvelope, TestSubscriber.Probe[EventEnvelope]] = JavaSink.probe(system) + def withCurrentEventsByTag( + within: FiniteDuration + )(tag: String, offset: Offset)(f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { + val sink: org.apache.pekko.stream.javadsl.Sink[EventEnvelope, TestSubscriber.Probe[EventEnvelope]] = + JavaSink.probe(system) val tp = readJournal.currentEventsByTag(tag, offset).runWith(sink, mat) tp.within(within)(f(tp)) } - def withEventsByTag(within: FiniteDuration)(tag: String, offset: Offset)( - f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { - val sink: org.apache.pekko.stream.javadsl.Sink[EventEnvelope, TestSubscriber.Probe[EventEnvelope]] = JavaSink.probe(system) + def withEventsByTag( + within: FiniteDuration + )(tag: String, offset: Offset)(f: TestSubscriber.Probe[EventEnvelope] => Unit): Unit = { + val sink: org.apache.pekko.stream.javadsl.Sink[EventEnvelope, TestSubscriber.Probe[EventEnvelope]] = + JavaSink.probe(system) val tp = readJournal.eventsByTag(tag, offset).runWith(sink, mat) tp.within(within)(f(tp)) } @@ -184,7 +204,8 @@ object QueryTestSpec { def expectNextEventEnvelope( persistenceId: String, sequenceNr: Long, - event: Any): TestSubscriber.Probe[EventEnvelope] = { + event: Any + ): TestSubscriber.Probe[EventEnvelope] = { val env = probe.expectNext() assertEnvelope(env, persistenceId, sequenceNr, event) probe @@ -194,7 +215,8 @@ object QueryTestSpec { timeout: FiniteDuration, persistenceId: String, sequenceNr: Long, - event: Any): TestSubscriber.Probe[EventEnvelope] = { + event: Any + ): TestSubscriber.Probe[EventEnvelope] = { val env = probe.expectNext(timeout) assertEnvelope(env, persistenceId, sequenceNr, event) probe @@ -203,7 +225,8 @@ object QueryTestSpec { private def assertEnvelope(env: EventEnvelope, persistenceId: String, sequenceNr: Long, event: Any): Unit = { assert( env.persistenceId == persistenceId, - s"expected persistenceId $persistenceId, found ${env.persistenceId}, in $env") + s"expected persistenceId $persistenceId, found ${env.persistenceId}, in $env" + ) assert(env.sequenceNr == sequenceNr, s"expected sequenceNr $sequenceNr, found ${env.sequenceNr}, in $env") assert(env.event == event, s"expected event $event, found ${env.event}, in $env") } @@ -300,15 +323,17 @@ abstract class QueryTestSpec(config: String, configOverrides: Map[String, Config system.actorOf(Props(new TestActor(persistenceId, replyToMessages))) } - def withTestActors(seq: Int = 1, replyToMessages: Boolean = false)(f: (ActorRef, ActorRef, ActorRef) => Unit)( - implicit system: ActorSystem): Unit = { + def withTestActors(seq: Int = 1, replyToMessages: Boolean = false)( + f: (ActorRef, ActorRef, ActorRef) => Unit + )(implicit system: ActorSystem): Unit = { val refs = (seq until seq + 3).map(setupEmpty(_, replyToMessages)).toList try f(refs.head, refs.drop(1).head, refs.drop(2).head) finally killActors(refs: _*) } - def withManyTestActors(amount: Int, seq: Int = 1, replyToMessages: Boolean = false)(f: Seq[ActorRef] => Unit)( - implicit system: ActorSystem): Unit = { + def withManyTestActors(amount: Int, seq: Int = 1, replyToMessages: Boolean = false)( + f: Seq[ActorRef] => Unit + )(implicit system: ActorSystem): Unit = { val refs = (seq until seq + amount).map(setupEmpty(_, replyToMessages)).toList try f(refs) finally killActors(refs: _*) diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/TaggingEventAdapter.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/TaggingEventAdapter.scala index e2ca0d65..ff26d1cb 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/TaggingEventAdapter.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/TaggingEventAdapter.scala @@ -5,17 +5,15 @@ package org.apache.pekko.persistence.postgres.query +import org.apache.pekko.persistence.journal.{Tagged, WriteEventAdapter} import org.apache.pekko.persistence.postgres.query.TaggingEventAdapter.TagEvent -import org.apache.pekko.persistence.journal.{ Tagged, WriteEventAdapter } object TaggingEventAdapter { case class TagEvent(payload: Any, tags: Set[String]) } -/** - * The TaggingEventAdapter will instruct persistence - * to tag the received event. - */ +/** The TaggingEventAdapter will instruct persistence to tag the received event. + */ class TaggingEventAdapter extends WriteEventAdapter { override def manifest(event: Any): String = "" diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalMetadataQueriesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalMetadataQueriesTest.scala index 638f8e1c..a697b3b9 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalMetadataQueriesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalMetadataQueriesTest.scala @@ -6,7 +6,8 @@ import org.apache.pekko.persistence.postgres.util.BaseQueryTest class ReadJournalMetadataQueriesTest extends BaseQueryTest { it should "create SQL query for minAndMaxOrderingForPersistenceId" in withReadJournalMetadataQueries { queries => queries.minAndMaxOrderingForPersistenceId( - "aaa") shouldBeSQL """select "min_ordering", "max_ordering" from "journal_metadata" where "persistence_id" = ? limit 1""" + "aaa" + ) shouldBeSQL """select "min_ordering", "max_ordering" from "journal_metadata" where "persistence_id" = ? limit 1""" } private def withReadJournalMetadataQueries(f: ReadJournalMetadataQueries => Unit): Unit = { diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalQueriesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalQueriesTest.scala index 6945fc14..ad2181b2 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalQueriesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalQueriesTest.scala @@ -14,7 +14,8 @@ class ReadJournalQueriesTest extends BaseQueryTest { "p1", 1L, 4L, - 5L) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where (("persistence_id" = ?) and ("sequence_number" >= ?)) and ("sequence_number" <= ?) order by "sequence_number" limit ?""" + 5L + ) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where (("persistence_id" = ?) and ("sequence_number" >= ?)) and ("sequence_number" <= ?) order by "sequence_number" limit ?""" } it should "create SQL query for messagesMinOrderingBoundedQuery" in withReadJournalQueries { queries => @@ -23,20 +24,23 @@ class ReadJournalQueriesTest extends BaseQueryTest { 1L, 4L, 5L, - 1L) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where ((("persistence_id" = ?) and ("sequence_number" >= ?)) and ("sequence_number" <= ?)) and ("ordering" >= ?) order by "sequence_number" limit ?""" + 1L + ) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where ((("persistence_id" = ?) and ("sequence_number" >= ?)) and ("sequence_number" <= ?)) and ("ordering" >= ?) order by "sequence_number" limit ?""" } it should "create SQL query for eventsByTag" in withReadJournalQueries { queries => queries.eventsByTag( List(11), 23L, - 25L) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where ("tags" @> ?) and (("ordering" > ?) and ("ordering" <= ?)) order by "ordering"""" + 25L + ) shouldBeSQL """select "ordering", "deleted", "persistence_id", "sequence_number", "message", "tags", "metadata" from "journal" where ("tags" @> ?) and (("ordering" > ?) and ("ordering" <= ?)) order by "ordering"""" } it should "create SQL query for journalSequenceQuery" in withReadJournalQueries { queries => queries.orderingByOrdering( 11L, - 23L) shouldBeSQL """select "ordering" from "journal" where "ordering" > ? order by "ordering" limit ?""" + 23L + ) shouldBeSQL """select "ordering" from "journal" where "ordering" > ? order by "ordering" limit ?""" } it should "create SQL query for maxJournalSequenceQuery" in withReadJournalQueries { queries => @@ -48,7 +52,9 @@ class ReadJournalQueriesTest extends BaseQueryTest { f( new ReadJournalQueries( FlatJournalTable(readJournalConfig.journalTableConfiguration), - readJournalConfig.includeDeleted)) + readJournalConfig.includeDeleted + ) + ) } } } diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalTablesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalTablesTest.scala index 3cfc75d5..405fe04f 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalTablesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/ReadJournalTablesTest.scala @@ -6,7 +6,11 @@ package org.apache.pekko.persistence.postgres.query.dao import org.apache.pekko.persistence.postgres.TablesTestSpec -import org.apache.pekko.persistence.postgres.journal.dao.{ FlatJournalTable, NestedPartitionsJournalTable, PartitionedJournalTable } +import org.apache.pekko.persistence.postgres.journal.dao.{ + FlatJournalTable, + NestedPartitionsJournalTable, + PartitionedJournalTable +} class ReadJournalTablesTest extends TablesTestSpec { val readJournalTableConfiguration = readJournalConfig.journalTableConfiguration @@ -15,7 +19,8 @@ class ReadJournalTablesTest extends TablesTestSpec { (journalName, journalTable) <- List( ("FlatJournalTable", FlatJournalTable(readJournalTableConfiguration)), ("PartitionedJournalTable", PartitionedJournalTable(readJournalTableConfiguration)), - ("NestedPartitionsJournalTable", NestedPartitionsJournalTable(readJournalTableConfiguration))) + ("NestedPartitionsJournalTable", NestedPartitionsJournalTable(readJournalTableConfiguration)) + ) } { s"Read $journalName" should "be configured with a schema name" in { journalTable.baseTableRow.schemaName shouldBe readJournalTableConfiguration.schemaName @@ -28,9 +33,11 @@ class ReadJournalTablesTest extends TablesTestSpec { it should "be configured with column names" in { val colName = toColumnName(readJournalTableConfiguration.tableName)(_) journalTable.baseTableRow.persistenceId.toString shouldBe colName( - readJournalTableConfiguration.columnNames.persistenceId) + readJournalTableConfiguration.columnNames.persistenceId + ) journalTable.baseTableRow.sequenceNumber.toString shouldBe colName( - readJournalTableConfiguration.columnNames.sequenceNumber) + readJournalTableConfiguration.columnNames.sequenceNumber + ) journalTable.baseTableRow.tags.toString shouldBe colName(readJournalTableConfiguration.columnNames.tags) } } diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/TestProbeReadJournalDao.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/TestProbeReadJournalDao.scala index cd626a42..0144b577 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/TestProbeReadJournalDao.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/query/dao/TestProbeReadJournalDao.scala @@ -22,57 +22,58 @@ object TestProbeReadJournalDao { case class JournalSequence(offset: Long, limit: Long) } -/** - * Read journal dao where the journalSequence query is backed by a testprobe - */ +/** Read journal dao where the journalSequence query is backed by a testprobe + */ class TestProbeReadJournalDao(val probe: TestProbe) extends ReadJournalDao { // Since the testprobe is instrumented by the test, it should respond very fast implicit val askTimeout: Timeout = Timeout(100.millis) - /** - * Returns distinct stream of persistenceIds - */ + /** Returns distinct stream of persistenceIds + */ override def allPersistenceIdsSource(max: Long): Source[String, NotUsed] = ??? - /** - * Returns a Source of bytes for certain tag from an offset. The result is sorted by - * created time asc thus the offset is relative to the creation time - */ + /** Returns a Source of bytes for certain tag from an offset. The result is sorted by created time asc thus the offset + * is relative to the creation time + */ override def eventsByTag( tag: String, offset: Long, maxOffset: Long, - max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] = ??? + max: Long + ): Source[Try[(PersistentRepr, Long)], NotUsed] = ??? - /** - * Returns a Source of bytes for a certain persistenceId - */ + /** Returns a Source of bytes for a certain persistenceId + */ override def messages( persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - max: Long): Source[Try[(PersistentRepr, Long)], NotUsed] = ??? + max: Long + ): Source[Try[(PersistentRepr, Long)], NotUsed] = ??? override def messagesWithBatch( persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, batchSize: Int, - refreshInterval: Option[(FiniteDuration, Scheduler)]): Source[Try[(PersistentRepr, Long)], NotUsed] = ??? + refreshInterval: Option[(FiniteDuration, Scheduler)] + ): Source[Try[(PersistentRepr, Long)], NotUsed] = ??? - /** - * @param offset Minimum value to retrieve - * @param limit Maximum number of values to retrieve - * @return A Source of journal event sequence numbers (corresponding to the Ordering column) - */ + /** @param offset + * Minimum value to retrieve + * @param limit + * Maximum number of values to retrieve + * @return + * A Source of journal event sequence numbers (corresponding to the Ordering column) + */ override def journalSequence(offset: Long, limit: Long): Source[Long, NotUsed] = { val f = probe.ref.ask(JournalSequence(offset, limit)).mapTo[scala.collection.immutable.Seq[Long]] Source.future(f).mapConcat(identity) } - /** - * @return The value of the maximum (ordering) id in the journal - */ + /** @return + * The value of the maximum (ordering) id in the journal + */ override def maxJournalSequence(): Future[Long] = Future.successful(0) } diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/serialization/StoreOnlySerializableMessagesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/serialization/StoreOnlySerializableMessagesTest.scala index 748008ec..ef643415 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/serialization/StoreOnlySerializableMessagesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/serialization/StoreOnlySerializableMessagesTest.scala @@ -5,11 +5,11 @@ package org.apache.pekko.persistence.postgres.serialization -import org.apache.pekko.actor.{ ActorRef, Props } +import org.apache.pekko.actor.{ActorRef, Props} import org.apache.pekko.event.LoggingReceive +import org.apache.pekko.persistence.{PersistentActor, RecoveryCompleted} import org.apache.pekko.persistence.postgres.SharedActorSystemTestSpec import org.apache.pekko.persistence.postgres.util.Schema._ -import org.apache.pekko.persistence.{ PersistentActor, RecoveryCompleted } import org.apache.pekko.testkit.TestProbe import scala.concurrent.duration._ @@ -23,8 +23,8 @@ abstract class StoreOnlySerializableMessagesTest(schemaType: SchemaType) val persistenceId: String, recoverProbe: ActorRef, persistFailureProbe: ActorRef, - persistRejectedProbe: ActorRef) - extends PersistentActor { + persistRejectedProbe: ActorRef + ) extends PersistentActor { override val receiveRecover: Receive = LoggingReceive { case msg => recoverProbe ! msg } @@ -47,7 +47,8 @@ abstract class StoreOnlySerializableMessagesTest(schemaType: SchemaType) val persistFailureProbe = TestProbe() val persistRejectedProbe = TestProbe() val persistentActor = system.actorOf( - Props(new TestActor(s"my-$id", recoverProbe.ref, persistFailureProbe.ref, persistRejectedProbe.ref))) + Props(new TestActor(s"my-$id", recoverProbe.ref, persistFailureProbe.ref, persistRejectedProbe.ref)) + ) try f(persistentActor)(recoverProbe)(persistFailureProbe)(persistRejectedProbe) finally killActors(persistentActor) } diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/PostgresSnapshotStoreSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/PostgresSnapshotStoreSpec.scala index 1c32d210..d0727808 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/PostgresSnapshotStoreSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/PostgresSnapshotStoreSpec.scala @@ -5,12 +5,12 @@ package org.apache.pekko.persistence.postgres.snapshot +import com.typesafe.config.ConfigFactory import org.apache.pekko.persistence.postgres.config._ import org.apache.pekko.persistence.postgres.db.SlickDatabase -import org.apache.pekko.persistence.postgres.util.Schema._ import org.apache.pekko.persistence.postgres.util.{ClasspathResources, DropCreate} +import org.apache.pekko.persistence.postgres.util.Schema._ import org.apache.pekko.persistence.snapshot.SnapshotStoreSpec -import com.typesafe.config.ConfigFactory import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.ScalaFutures diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotSerializerTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotSerializerTest.scala index 1697c8ad..17a04521 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotSerializerTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/ByteArraySnapshotSerializerTest.scala @@ -1,14 +1,14 @@ package org.apache.pekko.persistence.postgres.snapshot.dao -import java.time.{ LocalDateTime, ZoneOffset } - +import io.circe.Json import org.apache.pekko.persistence.SnapshotMetadata import org.apache.pekko.persistence.postgres.SharedActorSystemTestSpec import org.apache.pekko.persistence.postgres.snapshot.dao.SnapshotTables.SnapshotRow import org.apache.pekko.serialization.Serializers -import io.circe.Json import org.scalatest.TryValues +import java.time.{LocalDateTime, ZoneOffset} + class ByteArraySnapshotSerializerTest extends SharedActorSystemTestSpec with TryValues { val serializer = new ByteArraySnapshotSerializer(serialization) @@ -27,7 +27,8 @@ class ByteArraySnapshotSerializerTest extends SharedActorSystemTestSpec with Try row.metadata should equal { Json.obj( // serialization manifest for String should be blank and omitted - "sid" -> Json.fromInt(serId)) + "sid" -> Json.fromInt(serId) + ) } } @@ -61,7 +62,8 @@ class ByteArraySnapshotSerializerTest extends SharedActorSystemTestSpec with Try val meta = Json.obj( "sid" -> Json.fromInt(serId), "serId" -> Json.fromInt(-1), - "serManifest" -> Json.fromString("this will be ignored")) + "serManifest" -> Json.fromString("this will be ignored") + ) val row = SnapshotRow("per-1", 42, timestamp, serializedPayload, meta) val (metadata, _) = serializer.deserialize(row).get metadata should equal(SnapshotMetadata("per-1", 42, timestamp)) diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotQueriesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotQueriesTest.scala index 62608d30..eac814e3 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotQueriesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotQueriesTest.scala @@ -1,9 +1,9 @@ package org.apache.pekko.persistence.postgres.snapshot.dao +import io.circe.{Json, JsonObject} import org.apache.pekko.persistence.postgres.config.SnapshotConfig import org.apache.pekko.persistence.postgres.snapshot.dao.SnapshotTables.SnapshotRow import org.apache.pekko.persistence.postgres.util.BaseQueryTest -import io.circe.{ Json, JsonObject } class SnapshotQueriesTest extends BaseQueryTest { import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile.api._ @@ -16,28 +16,27 @@ class SnapshotQueriesTest extends BaseQueryTest { it should "create SQL query for selectAllByPersistenceId" in withSnapshotQueries { queries => queries.selectAllByPersistenceId( - "p1") shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where "persistence_id" = ? order by "sequence_number" desc""" + "p1" + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where "persistence_id" = ? order by "sequence_number" desc""" } it should "create SQL query for insertOrUpdate" in withSnapshotQueries { queries => queries.insertOrUpdate( - SnapshotRow( - "p1", - 32L, - 1333L, - Array.ofDim(0), - emptyJson)) shouldBeSQL """insert into "snapshot" ("persistence_id","sequence_number","created","snapshot","metadata") values (?,?,?,?,?) on conflict ("persistence_id", "sequence_number") do update set "created"=EXCLUDED."created","snapshot"=EXCLUDED."snapshot","metadata"=EXCLUDED."metadata"""" + SnapshotRow("p1", 32L, 1333L, Array.ofDim(0), emptyJson) + ) shouldBeSQL """insert into "snapshot" ("persistence_id","sequence_number","created","snapshot","metadata") values (?,?,?,?,?) on conflict ("persistence_id", "sequence_number") do update set "created"=EXCLUDED."created","snapshot"=EXCLUDED."snapshot","metadata"=EXCLUDED."metadata"""" } it should "create SQL query for selectLatestByPersistenceId" in withSnapshotQueries { queries => queries.selectLatestByPersistenceId( - "p1") shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where "persistence_id" = ? order by "sequence_number" desc limit 1""" + "p1" + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where "persistence_id" = ? order by "sequence_number" desc limit 1""" } it should "create SQL query for selectByPersistenceIdAndSequenceNr" in withSnapshotQueries { queries => queries.selectByPersistenceIdAndSequenceNr( "p1", - 11L) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("sequence_number" = ?) order by "sequence_number" desc""" + 11L + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("sequence_number" = ?) order by "sequence_number" desc""" } it should "create SQL query for selectByPersistenceIdAndSequenceNr.delete" in withSnapshotQueries { queries => @@ -49,7 +48,8 @@ class SnapshotQueriesTest extends BaseQueryTest { it should "create SQL query for selectByPersistenceIdUpToMaxTimestamp" in withSnapshotQueries { queries => queries.selectByPersistenceIdUpToMaxTimestamp( "p1", - 11L) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("created" <= ?) order by "sequence_number" desc""" + 11L + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("created" <= ?) order by "sequence_number" desc""" } it should "create SQL query for selectByPersistenceIdUpToMaxTimestamp.delete" in withSnapshotQueries { queries => @@ -61,7 +61,8 @@ class SnapshotQueriesTest extends BaseQueryTest { it should "create SQL query for selectByPersistenceIdUpToMaxSequenceNr" in withSnapshotQueries { queries => queries.selectByPersistenceIdUpToMaxSequenceNr( "p1", - 11L) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("sequence_number" <= ?) order by "sequence_number" desc""" + 11L + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("sequence_number" <= ?) order by "sequence_number" desc""" } it should "create SQL query for selectByPersistenceIdUpToMaxSequenceNr.delete" in withSnapshotQueries { queries => @@ -75,7 +76,8 @@ class SnapshotQueriesTest extends BaseQueryTest { queries.selectByPersistenceIdUpToMaxSequenceNrAndMaxTimestamp( "p1", 11L, - 23L) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where (("persistence_id" = ?) and ("sequence_number" <= ?)) and ("created" <= ?) order by "sequence_number" desc""" + 23L + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where (("persistence_id" = ?) and ("sequence_number" <= ?)) and ("created" <= ?) order by "sequence_number" desc""" } it should "create SQL query for selectByPersistenceIdUpToMaxSequenceNrAndMaxTimestamp.delete" in withSnapshotQueries { @@ -88,13 +90,15 @@ class SnapshotQueriesTest extends BaseQueryTest { it should "create SQL query for selectOneByPersistenceIdAndMaxTimestamp" in withSnapshotQueries { queries => queries.selectOneByPersistenceIdAndMaxTimestamp( "p1", - 11L) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("created" <= ?) order by "sequence_number" desc limit 1""" + 11L + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("created" <= ?) order by "sequence_number" desc limit 1""" } it should "create SQL query for selectOneByPersistenceIdAndMaxSequenceNr" in withSnapshotQueries { queries => queries.selectOneByPersistenceIdAndMaxSequenceNr( "p1", - 23L) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("sequence_number" <= ?) order by "sequence_number" desc limit 1""" + 23L + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where ("persistence_id" = ?) and ("sequence_number" <= ?) order by "sequence_number" desc limit 1""" } it should "create SQL query for selectOneByPersistenceIdAndMaxSequenceNrAndMaxTimestamp" in withSnapshotQueries { @@ -102,7 +106,8 @@ class SnapshotQueriesTest extends BaseQueryTest { queries.selectOneByPersistenceIdAndMaxSequenceNrAndMaxTimestamp( "p1", 11L, - 23L) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where (("persistence_id" = ?) and ("sequence_number" <= ?)) and ("created" <= ?) order by "sequence_number" desc limit 1""" + 23L + ) shouldBeSQL """select "persistence_id", "sequence_number", "created", "snapshot", "metadata" from "snapshot" where (("persistence_id" = ?) and ("sequence_number" <= ?)) and ("created" <= ?) order by "sequence_number" desc limit 1""" } lazy val emptyJson: Json = Json.fromJsonObject(JsonObject.empty) diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotTablesTest.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotTablesTest.scala index 5eaa1675..f665f3a7 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotTablesTest.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/snapshot/dao/SnapshotTablesTest.scala @@ -24,12 +24,16 @@ class SnapshotTablesTest extends TablesTestSpec { it should "be configured with column names" in { val colName = toColumnName(snapshotTableConfiguration.tableName)(_) TestByteASnapshotTables.SnapshotTable.baseTableRow.persistenceId.toString shouldBe colName( - snapshotTableConfiguration.columnNames.persistenceId) + snapshotTableConfiguration.columnNames.persistenceId + ) TestByteASnapshotTables.SnapshotTable.baseTableRow.sequenceNumber.toString shouldBe colName( - snapshotTableConfiguration.columnNames.sequenceNumber) + snapshotTableConfiguration.columnNames.sequenceNumber + ) TestByteASnapshotTables.SnapshotTable.baseTableRow.created.toString shouldBe colName( - snapshotTableConfiguration.columnNames.created) + snapshotTableConfiguration.columnNames.created + ) TestByteASnapshotTables.SnapshotTable.baseTableRow.snapshot.toString shouldBe colName( - snapshotTableConfiguration.columnNames.snapshot) + snapshotTableConfiguration.columnNames.snapshot + ) } } diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/tag/CachedTagIdResolverSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/tag/CachedTagIdResolverSpec.scala index 70f456e3..5e1321e6 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/tag/CachedTagIdResolverSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/tag/CachedTagIdResolverSpec.scala @@ -1,18 +1,17 @@ package org.apache.pekko.persistence.postgres.tag -import java.util.concurrent.ThreadLocalRandom -import java.util.concurrent.atomic.{ AtomicBoolean, AtomicLong } - -import org.apache.pekko.persistence.postgres.config.TagsConfig import com.typesafe.config.ConfigFactory +import org.apache.pekko.persistence.postgres.config.TagsConfig +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, OptionValues} +import org.scalatest.concurrent.{IntegrationPatience, ScalaFutures} import org.scalatest.concurrent.Eventually.eventually -import org.scalatest.concurrent.{ IntegrationPatience, ScalaFutures } import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike -import org.scalatest.{ BeforeAndAfter, BeforeAndAfterAll, OptionValues } +import java.util.concurrent.ThreadLocalRandom +import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} import scala.collection.mutable -import scala.concurrent.{ ExecutionContext, Future } +import scala.concurrent.{ExecutionContext, Future} import scala.util.Random import scala.util.control.NoStackTrace @@ -25,7 +24,7 @@ class CachedTagIdResolverSpec with BeforeAndAfter with IntegrationPatience { - private implicit val global: ExecutionContext = ExecutionContext.global + implicit private val global: ExecutionContext = ExecutionContext.global "CachedTagIdResolver" when { "finding or adding tag name to id mapping" should { @@ -38,7 +37,8 @@ class CachedTagIdResolverSpec tagName should equal(fakeTagName) Future.successful(Some(fakeTagId)) }, - insertF = _ => fail("Unwanted interaction with DAO (insert)")) + insertF = _ => fail("Unwanted interaction with DAO (insert)") + ) val resolver = new CachedTagIdResolver(dao, config) // when @@ -57,7 +57,8 @@ class CachedTagIdResolverSpec insertF = tagName => { tagName should equal(fakeTagName) Future.successful(fakeTagId) - }) + } + ) val resolver = new CachedTagIdResolver(dao, config) // when @@ -83,8 +84,10 @@ class CachedTagIdResolverSpec else if (name == anotherTagName) Future.successful(anotherTagId) else fail( - s"Unwanted interaction with DAO (insert) for tagName = '$name' ($tagName, $anotherTagName, $existingTagName)") - }) + s"Unwanted interaction with DAO (insert) for tagName = '$name' ($tagName, $anotherTagName, $existingTagName)" + ) + } + ) val resolver = new CachedTagIdResolver(dao, config) // when @@ -94,7 +97,8 @@ class CachedTagIdResolverSpec returnedTagIds should contain theSameElementsAs Map( tagName -> tagId, anotherTagName -> anotherTagId, - existingTagName -> existingTagId) + existingTagName -> existingTagId + ) } "hit the DAO only once and then read from cache" in { @@ -108,7 +112,8 @@ class CachedTagIdResolverSpec insertF = tagName => { tagName should equal(fakeTagName) Future.successful(fakeTagId) - }) + } + ) val resolver = new CachedTagIdResolver(dao, config) // when @@ -129,7 +134,8 @@ class CachedTagIdResolverSpec insertF = _ => { attemptsCount.incrementAndGet() Future.failed(FakeException) - }) + } + ) val resolver = new CachedTagIdResolver(dao, config) // when @@ -145,7 +151,8 @@ class CachedTagIdResolverSpec val fakeTagId = Random.nextInt() val dao = new FakeTagDao( findF = _ => fail("Unwanted interaction with DAO (find)"), - insertF = _ => fail("Unwanted interaction with DAO (insert)")) + insertF = _ => fail("Unwanted interaction with DAO (insert)") + ) val resolver = new CachedTagIdResolver(dao, config) resolver.cache.synchronous().put(fakeTagName, fakeTagId) @@ -164,7 +171,8 @@ class CachedTagIdResolverSpec val listOfTagQueries = List.fill(300)(mapOfTags.keys.toList(Random.nextInt(mapOfTags.size))) val dao = new FakeTagDao( findF = tagName => Future.successful(if (Random.nextBoolean()) Some(mapOfTags(tagName)) else None), - insertF = tagName => Future.successful(mapOfTags(tagName))) + insertF = tagName => Future.successful(mapOfTags(tagName)) + ) val resolver = new CachedTagIdResolver(dao, config) // when @@ -186,7 +194,8 @@ class CachedTagIdResolverSpec tagName should equal(fakeTagName) Future.successful(Some(fakeTagId)) }, - insertF = _ => fail("Unwanted interaction with DAO (insert)")) + insertF = _ => fail("Unwanted interaction with DAO (insert)") + ) val resolver = new CachedTagIdResolver(dao, config) // when @@ -202,7 +211,8 @@ class CachedTagIdResolverSpec val fakeTagId = Random.nextInt() val dao = new FakeTagDao( findF = _ => fail("Unwanted interaction with DAO (find)"), - insertF = _ => fail("Unwanted interaction with DAO (insert)")) + insertF = _ => fail("Unwanted interaction with DAO (insert)") + ) val resolver = new CachedTagIdResolver(dao, config) resolver.cache.synchronous().put(fakeTagName, fakeTagId) @@ -218,7 +228,8 @@ class CachedTagIdResolverSpec val fakeTagName = generateTagName() val dao = new FakeTagDao( findF = _ => Future.successful(None), - insertF = _ => fail("Unwanted interaction with DAO (insert)")) + insertF = _ => fail("Unwanted interaction with DAO (insert)") + ) val resolver = new CachedTagIdResolver(dao, config) // when @@ -240,7 +251,8 @@ class CachedTagIdResolverSpec if (name == fakeTagName && lookupMissHappened.getAndSet(true)) Future.successful(Some(fakeTagId)) else Future.successful(None), - insertF = _ => fail("Unwanted interaction with DAO (insert)")) + insertF = _ => fail("Unwanted interaction with DAO (insert)") + ) val resolver = new CachedTagIdResolver(dao, config) @@ -260,7 +272,8 @@ class CachedTagIdResolverSpec val fakeTagId = Random.nextInt() val dao = new FakeTagDao( findF = _ => Future.successful(Some(fakeTagId)), - insertF = _ => fail("Unwanted interaction with DAO (insert)")) + insertF = _ => fail("Unwanted interaction with DAO (insert)") + ) val resolver = new CachedTagIdResolver(dao, config) // then @@ -277,7 +290,8 @@ class CachedTagIdResolverSpec val fakeTagName = generateTagName() val dao = new FakeTagDao( findF = _ => Future.successful(None), - insertF = _ => fail("Unwanted interaction with DAO (insert)")) + insertF = _ => fail("Unwanted interaction with DAO (insert)") + ) val resolver = new CachedTagIdResolver(dao, config) // then diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/tag/SimpleTagDaoSpec.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/tag/SimpleTagDaoSpec.scala index a6ec6a90..44031a6f 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/tag/SimpleTagDaoSpec.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/tag/SimpleTagDaoSpec.scala @@ -1,16 +1,15 @@ package org.apache.pekko.persistence.postgres.tag -import java.util.concurrent.ThreadLocalRandom - -import org.apache.pekko.persistence.postgres.config.{ SlickConfiguration, TagsTableConfiguration } +import com.typesafe.config.{Config, ConfigFactory} +import org.apache.pekko.persistence.postgres.config.{SlickConfiguration, TagsTableConfiguration} import org.apache.pekko.persistence.postgres.db.SlickDatabase -import com.typesafe.config.{ Config, ConfigFactory } -import org.scalatest.concurrent.{ IntegrationPatience, ScalaFutures } +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, OptionValues} +import org.scalatest.concurrent.{IntegrationPatience, ScalaFutures} import org.scalatest.flatspec.AnyFlatSpecLike import org.scalatest.matchers.should.Matchers -import org.scalatest.{ BeforeAndAfter, BeforeAndAfterAll, OptionValues } import slick.jdbc +import java.util.concurrent.ThreadLocalRandom import scala.concurrent.ExecutionContext class SimpleTagDaoSpec @@ -24,7 +23,7 @@ class SimpleTagDaoSpec import org.apache.pekko.persistence.postgres.db.ExtendedPostgresProfile.api._ - private implicit val global: ExecutionContext = ExecutionContext.global + implicit private val global: ExecutionContext = ExecutionContext.global before { withDB { db => diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/util/ClasspathResources.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/util/ClasspathResources.scala index ab8bae45..a9b737a7 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/util/ClasspathResources.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/util/ClasspathResources.scala @@ -6,8 +6,7 @@ package org.apache.pekko.persistence.postgres.util import java.io.InputStream - -import scala.io.{ Source => ScalaIOSource } +import scala.io.{Source => ScalaIOSource} object ClasspathResources extends ClasspathResources diff --git a/core/src/test/scala/org/apache/pekko/persistence/postgres/util/DropCreate.scala b/core/src/test/scala/org/apache/pekko/persistence/postgres/util/DropCreate.scala index e8272153..e574b78c 100644 --- a/core/src/test/scala/org/apache/pekko/persistence/postgres/util/DropCreate.scala +++ b/core/src/test/scala/org/apache/pekko/persistence/postgres/util/DropCreate.scala @@ -5,8 +5,7 @@ package org.apache.pekko.persistence.postgres.util -import java.sql.Statement -import org.apache.pekko.persistence.postgres.config.{ JournalMetadataTableConfiguration, JournalTableConfiguration } +import org.apache.pekko.persistence.postgres.config.{JournalMetadataTableConfiguration, JournalTableConfiguration} import org.apache.pekko.persistence.postgres.journal.dao.{ FlatJournalTable, JournalMetadataTable, @@ -15,9 +14,11 @@ import org.apache.pekko.persistence.postgres.journal.dao.{ PartitionedJournalTable } import org.apache.pekko.persistence.postgres.util.Schema.SchemaType -import slick.jdbc.JdbcBackend.{ Database, Session } +import slick.jdbc.JdbcBackend.{Database, Session} import slick.lifted.TableQuery +import java.sql.Statement + object Schema { sealed trait SchemaType { @@ -25,8 +26,8 @@ object Schema { lazy val schema: String = s"schema/postgres/$resourceNamePrefix-schema.sql" lazy val configName: String = s"${resourceNamePrefix}-application.conf" def table(journalTableCfg: JournalTableConfiguration): TableQuery[JournalTable] - def metadataTable(journalMetadataTableCfg: JournalMetadataTableConfiguration) - : TableQuery[JournalMetadataTable] = JournalMetadataTable.apply(journalMetadataTableCfg) + def metadataTable(journalMetadataTableCfg: JournalMetadataTableConfiguration): TableQuery[JournalMetadataTable] = + JournalMetadataTable.apply(journalMetadataTableCfg) } case object Plain extends SchemaType {