Configuring persistence plugins at runtime for EventSourcedBehavior (#1518)
* Add runtime journal plugin configuration to EventSourcedBehavior * Add runtime journal plugin configuration test for event journals * Add runtime journal plugin configuration test for snapshot store * Mark SnapshotStorageEmulatorExtension class as internal API * Add runtime journal plugin configuration test for event journals explicitly * Address review comments * Address review comments * Add binary compat exclusions * Fix test name * Add Java API * Use pekko option converters instead of scala jdk
This commit is contained in:
parent
0e10d291db
commit
4afe7cf331
13 changed files with 287 additions and 49 deletions
|
|
@ -141,9 +141,18 @@ object PersistenceTestKitPlugin {
|
||||||
* Persistence testkit plugin for snapshots.
|
* Persistence testkit plugin for snapshots.
|
||||||
*/
|
*/
|
||||||
@InternalApi
|
@InternalApi
|
||||||
class PersistenceTestKitSnapshotPlugin extends SnapshotStore {
|
class PersistenceTestKitSnapshotPlugin(
|
||||||
|
// providing this parameter in first position as unused
|
||||||
|
// because Persistence extension that instantiates the plugins
|
||||||
|
// does not support constructors without it
|
||||||
|
@unused cfg: Config,
|
||||||
|
cfgPath: String
|
||||||
|
) extends SnapshotStore {
|
||||||
|
|
||||||
private final val storage = SnapshotStorageEmulatorExtension(context.system)
|
private final val storage = {
|
||||||
|
log.debug("Using snapshot storage emulator extension [{}] for test kit snapshot storage", cfgPath)
|
||||||
|
SnapshotStorageEmulatorExtension(context.system).storageFor(cfgPath)
|
||||||
|
}
|
||||||
|
|
||||||
override def loadAsync(persistenceId: String, criteria: SnapshotSelectionCriteria): Future[Option[SelectedSnapshot]] =
|
override def loadAsync(persistenceId: String, criteria: SnapshotSelectionCriteria): Future[Option[SelectedSnapshot]] =
|
||||||
Future.fromTry(Try(storage.tryRead(persistenceId, criteria)))
|
Future.fromTry(Try(storage.tryRead(persistenceId, criteria)))
|
||||||
|
|
|
||||||
|
|
@ -13,6 +13,8 @@
|
||||||
|
|
||||||
package org.apache.pekko.persistence.testkit.internal
|
package org.apache.pekko.persistence.testkit.internal
|
||||||
|
|
||||||
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
|
|
||||||
import org.apache.pekko
|
import org.apache.pekko
|
||||||
import pekko.actor.Extension
|
import pekko.actor.Extension
|
||||||
import pekko.actor.{ ActorSystem, ExtendedActorSystem, ExtensionId, ExtensionIdProvider }
|
import pekko.actor.{ ActorSystem, ExtendedActorSystem, ExtensionId, ExtensionIdProvider }
|
||||||
|
|
@ -24,17 +26,34 @@ import pekko.persistence.testkit.scaladsl.SnapshotTestKit
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
@InternalApi
|
@InternalApi
|
||||||
private[testkit] object SnapshotStorageEmulatorExtension extends ExtensionId[SnapshotStorage] with ExtensionIdProvider {
|
private[testkit] object SnapshotStorageEmulatorExtension extends ExtensionId[SnapshotStorageEmulatorExtension]
|
||||||
|
with ExtensionIdProvider {
|
||||||
|
|
||||||
override def get(system: ActorSystem): SnapshotStorage = super.get(system)
|
override def get(system: ActorSystem): SnapshotStorageEmulatorExtension = super.get(system)
|
||||||
|
|
||||||
override def createExtension(system: ExtendedActorSystem): SnapshotStorage =
|
override def createExtension(system: ExtendedActorSystem): SnapshotStorageEmulatorExtension =
|
||||||
if (SnapshotTestKit.Settings(system).serialize) {
|
new SnapshotStorageEmulatorExtension(system)
|
||||||
new SerializedSnapshotStorageImpl(system)
|
|
||||||
} else {
|
|
||||||
new SimpleSnapshotStorageImpl
|
|
||||||
}
|
|
||||||
|
|
||||||
override def lookup: ExtensionId[_ <: Extension] =
|
override def lookup: ExtensionId[_ <: Extension] =
|
||||||
SnapshotStorageEmulatorExtension
|
SnapshotStorageEmulatorExtension
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*/
|
||||||
|
@InternalApi
|
||||||
|
final class SnapshotStorageEmulatorExtension(system: ExtendedActorSystem) extends Extension {
|
||||||
|
private val stores = new ConcurrentHashMap[String, SnapshotStorage]()
|
||||||
|
private lazy val shouldCreateSerializedSnapshotStorage = SnapshotTestKit.Settings(system).serialize
|
||||||
|
|
||||||
|
def storageFor(key: String): SnapshotStorage =
|
||||||
|
stores.computeIfAbsent(key,
|
||||||
|
_ => {
|
||||||
|
// we don't really care about the key here, we just want separate instances
|
||||||
|
if (shouldCreateSerializedSnapshotStorage) {
|
||||||
|
new SerializedSnapshotStorageImpl(system)
|
||||||
|
} else {
|
||||||
|
new SimpleSnapshotStorageImpl
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -336,7 +336,8 @@ class SnapshotTestKit(system: ActorSystem)
|
||||||
|
|
||||||
import SnapshotTestKit._
|
import SnapshotTestKit._
|
||||||
|
|
||||||
override protected val storage: SnapshotStorage = SnapshotStorageEmulatorExtension(system)
|
override protected val storage: SnapshotStorage =
|
||||||
|
SnapshotStorageEmulatorExtension(system).storageFor(PersistenceTestKitSnapshotPlugin.PluginId)
|
||||||
|
|
||||||
override def getItem(persistenceId: String, nextInd: Int): Option[Any] = {
|
override def getItem(persistenceId: String, nextInd: Int): Option[Any] = {
|
||||||
storage.firstInExpectNextQueue(persistenceId).map(reprToAny)
|
storage.firstInExpectNextQueue(persistenceId).map(reprToAny)
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,141 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.pekko.persistence.testkit.scaladsl
|
||||||
|
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
import org.apache.pekko
|
||||||
|
import pekko.Done
|
||||||
|
import pekko.actor.testkit.typed.scaladsl.LogCapturing
|
||||||
|
import pekko.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||||
|
import pekko.actor.typed.ActorRef
|
||||||
|
import pekko.actor.typed.Behavior
|
||||||
|
import pekko.actor.typed.scaladsl.adapter._
|
||||||
|
import pekko.persistence.JournalProtocol.RecoverySuccess
|
||||||
|
import pekko.persistence.JournalProtocol.ReplayMessages
|
||||||
|
import pekko.persistence.JournalProtocol.ReplayedMessage
|
||||||
|
import pekko.persistence.Persistence
|
||||||
|
import pekko.persistence.SelectedSnapshot
|
||||||
|
import pekko.persistence.SnapshotProtocol.LoadSnapshot
|
||||||
|
import pekko.persistence.SnapshotProtocol.LoadSnapshotResult
|
||||||
|
import pekko.persistence.SnapshotSelectionCriteria
|
||||||
|
import pekko.persistence.testkit.PersistenceTestKitPlugin
|
||||||
|
import pekko.persistence.testkit.PersistenceTestKitSnapshotPlugin
|
||||||
|
import pekko.persistence.typed.PersistenceId
|
||||||
|
import pekko.persistence.typed.scaladsl.Effect
|
||||||
|
import pekko.persistence.typed.scaladsl.EventSourcedBehavior
|
||||||
|
import pekko.persistence.typed.scaladsl.RetentionCriteria
|
||||||
|
import org.scalatest.Inside
|
||||||
|
import org.scalatest.wordspec.AnyWordSpecLike
|
||||||
|
|
||||||
|
object RuntimeJournalsSpec {
|
||||||
|
|
||||||
|
private object Actor {
|
||||||
|
sealed trait Command
|
||||||
|
case class Save(text: String, replyTo: ActorRef[Done]) extends Command
|
||||||
|
case class ShowMeWhatYouGot(replyTo: ActorRef[String]) extends Command
|
||||||
|
case object Stop extends Command
|
||||||
|
|
||||||
|
def apply(persistenceId: String, journal: String): Behavior[Command] =
|
||||||
|
EventSourcedBehavior[Command, String, String](
|
||||||
|
PersistenceId.ofUniqueId(persistenceId),
|
||||||
|
"",
|
||||||
|
(state, cmd) =>
|
||||||
|
cmd match {
|
||||||
|
case Save(text, replyTo) =>
|
||||||
|
Effect.persist(text).thenRun(_ => replyTo ! Done)
|
||||||
|
case ShowMeWhatYouGot(replyTo) =>
|
||||||
|
replyTo ! state
|
||||||
|
Effect.none
|
||||||
|
case Stop =>
|
||||||
|
Effect.stop()
|
||||||
|
},
|
||||||
|
(state, evt) => Seq(state, evt).filter(_.nonEmpty).mkString("|"))
|
||||||
|
.withRetention(RetentionCriteria.snapshotEvery(1, Int.MaxValue))
|
||||||
|
.withJournalPluginId(s"$journal.journal")
|
||||||
|
.withJournalPluginConfig(Some(config(journal)))
|
||||||
|
.withSnapshotPluginId(s"$journal.snapshot")
|
||||||
|
.withSnapshotPluginConfig(Some(config(journal)))
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private def config(journal: String) = {
|
||||||
|
ConfigFactory.parseString(s"""
|
||||||
|
$journal {
|
||||||
|
journal.class = "${classOf[PersistenceTestKitPlugin].getName}"
|
||||||
|
snapshot.class = "${classOf[PersistenceTestKitSnapshotPlugin].getName}"
|
||||||
|
}
|
||||||
|
""")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class RuntimeJournalsSpec
|
||||||
|
extends ScalaTestWithActorTestKit
|
||||||
|
with AnyWordSpecLike
|
||||||
|
with LogCapturing
|
||||||
|
with Inside {
|
||||||
|
|
||||||
|
import RuntimeJournalsSpec._
|
||||||
|
|
||||||
|
"The testkit journal and snapshot store plugins" must {
|
||||||
|
|
||||||
|
"be possible to configure at runtime and use in multiple isolated instances" in {
|
||||||
|
val probe = createTestProbe[Any]()
|
||||||
|
|
||||||
|
{
|
||||||
|
// one actor in each journal with same id
|
||||||
|
val j1 = spawn(Actor("id1", "journal1"))
|
||||||
|
val j2 = spawn(Actor("id1", "journal2"))
|
||||||
|
j1 ! Actor.Save("j1m1", probe.ref)
|
||||||
|
probe.receiveMessage()
|
||||||
|
j2 ! Actor.Save("j2m1", probe.ref)
|
||||||
|
probe.receiveMessage()
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
def assertJournal(journal: String, expectedEvent: String) = {
|
||||||
|
val ref = Persistence(system).journalFor(s"$journal.journal", config(journal))
|
||||||
|
ref.tell(ReplayMessages(0, Long.MaxValue, Long.MaxValue, "id1", probe.ref.toClassic), probe.ref.toClassic)
|
||||||
|
inside(probe.receiveMessage()) {
|
||||||
|
case ReplayedMessage(persistentRepr) =>
|
||||||
|
persistentRepr.persistenceId shouldBe "id1"
|
||||||
|
persistentRepr.payload shouldBe expectedEvent
|
||||||
|
}
|
||||||
|
probe.expectMessage(RecoverySuccess(1))
|
||||||
|
}
|
||||||
|
|
||||||
|
assertJournal("journal1", "j1m1")
|
||||||
|
assertJournal("journal2", "j2m1")
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
def assertSnapshot(journal: String, expectedShapshot: String) = {
|
||||||
|
val ref = Persistence(system).snapshotStoreFor(s"$journal.snapshot", config(journal))
|
||||||
|
ref.tell(LoadSnapshot("id1", SnapshotSelectionCriteria.Latest, Long.MaxValue),
|
||||||
|
probe.ref.toClassic)
|
||||||
|
inside(probe.receiveMessage()) {
|
||||||
|
case LoadSnapshotResult(Some(SelectedSnapshot(_, snapshot)), _) =>
|
||||||
|
snapshot shouldBe expectedShapshot
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
assertSnapshot("journal1", "j1m1")
|
||||||
|
assertSnapshot("journal2", "j2m1")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,21 @@
|
||||||
|
# Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
# or more contributor license agreements. See the NOTICE file
|
||||||
|
# distributed with this work for additional information
|
||||||
|
# regarding copyright ownership. The ASF licenses this file
|
||||||
|
# to you under the Apache License, Version 2.0 (the
|
||||||
|
# "License"); you may not use this file except in compliance
|
||||||
|
# with the License. You may obtain a copy of the License at
|
||||||
|
#
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
#
|
||||||
|
# Unless required by applicable law or agreed to in writing,
|
||||||
|
# software distributed under the License is distributed on an
|
||||||
|
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
# KIND, either express or implied. See the License for the
|
||||||
|
# specific language governing permissions and limitations
|
||||||
|
# under the License.
|
||||||
|
|
||||||
|
# Add EventSourcedBehavior.withJournalPluginConfig
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.pekko.persistence.typed.scaladsl.EventSourcedBehavior.withJournalPluginConfig")
|
||||||
|
# Add EventSourcedBehavior.withSnapshotPluginConfig
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.pekko.persistence.typed.scaladsl.EventSourcedBehavior.withSnapshotPluginConfig")
|
||||||
|
|
@ -14,9 +14,11 @@
|
||||||
package org.apache.pekko.persistence.typed.internal
|
package org.apache.pekko.persistence.typed.internal
|
||||||
|
|
||||||
import scala.concurrent.ExecutionContext
|
import scala.concurrent.ExecutionContext
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.apache.pekko
|
import org.apache.pekko
|
||||||
|
import pekko.util.Helpers.ConfigOps
|
||||||
import pekko.actor.Cancellable
|
import pekko.actor.Cancellable
|
||||||
import pekko.actor.typed.Signal
|
import pekko.actor.typed.Signal
|
||||||
import pekko.actor.typed.scaladsl.ActorContext
|
import pekko.actor.typed.scaladsl.ActorContext
|
||||||
|
|
@ -74,8 +76,10 @@ private[pekko] final class BehaviorSetup[C, E, S](
|
||||||
|
|
||||||
val persistence: Persistence = Persistence(context.system.toClassic)
|
val persistence: Persistence = Persistence(context.system.toClassic)
|
||||||
|
|
||||||
val journal: ClassicActorRef = persistence.journalFor(settings.journalPluginId)
|
val journal: ClassicActorRef = persistence
|
||||||
val snapshotStore: ClassicActorRef = persistence.snapshotStoreFor(settings.snapshotPluginId)
|
.journalFor(settings.journalPluginId, settings.journalPluginConfig.getOrElse(ConfigFactory.empty))
|
||||||
|
val snapshotStore: ClassicActorRef = persistence
|
||||||
|
.snapshotStoreFor(settings.snapshotPluginId, settings.snapshotPluginConfig.getOrElse(ConfigFactory.empty))
|
||||||
|
|
||||||
val isSnapshotOptional: Boolean =
|
val isSnapshotOptional: Boolean =
|
||||||
Persistence(context.system.classicSystem).configFor(snapshotStore).getBoolean("snapshot-is-optional")
|
Persistence(context.system.classicSystem).configFor(snapshotStore).getBoolean("snapshot-is-optional")
|
||||||
|
|
@ -107,14 +111,18 @@ private[pekko] final class BehaviorSetup[C, E, S](
|
||||||
|
|
||||||
private var recoveryTimer: OptionVal[Cancellable] = OptionVal.None
|
private var recoveryTimer: OptionVal[Cancellable] = OptionVal.None
|
||||||
|
|
||||||
|
val recoveryEventTimeout: FiniteDuration = persistence
|
||||||
|
.journalConfigFor(settings.journalPluginId, settings.journalPluginConfig.getOrElse(ConfigFactory.empty))
|
||||||
|
.getMillisDuration("recovery-event-timeout")
|
||||||
|
|
||||||
def startRecoveryTimer(snapshot: Boolean): Unit = {
|
def startRecoveryTimer(snapshot: Boolean): Unit = {
|
||||||
cancelRecoveryTimer()
|
cancelRecoveryTimer()
|
||||||
implicit val ec: ExecutionContext = context.executionContext
|
implicit val ec: ExecutionContext = context.executionContext
|
||||||
val timer =
|
val timer =
|
||||||
if (snapshot)
|
if (snapshot)
|
||||||
context.scheduleOnce(settings.recoveryEventTimeout, context.self, RecoveryTickEvent(snapshot = true))
|
context.scheduleOnce(recoveryEventTimeout, context.self, RecoveryTickEvent(snapshot = true))
|
||||||
else
|
else
|
||||||
context.system.scheduler.scheduleWithFixedDelay(settings.recoveryEventTimeout, settings.recoveryEventTimeout) {
|
context.system.scheduler.scheduleWithFixedDelay(recoveryEventTimeout, recoveryEventTimeout) {
|
||||||
() =>
|
() =>
|
||||||
context.self ! RecoveryTickEvent(snapshot = false)
|
context.self ! RecoveryTickEvent(snapshot = false)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,7 @@ import java.util.Optional
|
||||||
import java.util.UUID
|
import java.util.UUID
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
|
||||||
|
import com.typesafe.config.Config
|
||||||
import org.apache.pekko
|
import org.apache.pekko
|
||||||
import pekko.actor.typed
|
import pekko.actor.typed
|
||||||
import pekko.actor.typed.ActorRef
|
import pekko.actor.typed.ActorRef
|
||||||
|
|
@ -105,6 +106,8 @@ private[pekko] final case class EventSourcedBehaviorImpl[Command, Event, State](
|
||||||
loggerClass: Class[_],
|
loggerClass: Class[_],
|
||||||
journalPluginId: Option[String] = None,
|
journalPluginId: Option[String] = None,
|
||||||
snapshotPluginId: Option[String] = None,
|
snapshotPluginId: Option[String] = None,
|
||||||
|
journalPluginConfig: Option[Config] = None,
|
||||||
|
snapshotPluginConfig: Option[Config] = None,
|
||||||
tagger: Event => Set[String] = (_: Event) => Set.empty[String],
|
tagger: Event => Set[String] = (_: Event) => Set.empty[String],
|
||||||
eventAdapter: EventAdapter[Event, Any] = NoOpEventAdapter.instance[Event],
|
eventAdapter: EventAdapter[Event, Any] = NoOpEventAdapter.instance[Event],
|
||||||
snapshotAdapter: SnapshotAdapter[State] = NoOpSnapshotAdapter.instance[State],
|
snapshotAdapter: SnapshotAdapter[State] = NoOpSnapshotAdapter.instance[State],
|
||||||
|
|
@ -132,7 +135,8 @@ private[pekko] final case class EventSourcedBehaviorImpl[Command, Event, State](
|
||||||
case _ => false
|
case _ => false
|
||||||
}
|
}
|
||||||
if (!hasCustomLoggerName) ctx.setLoggerName(loggerClass)
|
if (!hasCustomLoggerName) ctx.setLoggerName(loggerClass)
|
||||||
val settings = EventSourcedSettings(ctx.system, journalPluginId.getOrElse(""), snapshotPluginId.getOrElse(""))
|
val settings = EventSourcedSettings(ctx.system, journalPluginId.getOrElse(""), snapshotPluginId.getOrElse(""),
|
||||||
|
journalPluginConfig, snapshotPluginConfig)
|
||||||
|
|
||||||
// stashState outside supervise because StashState should survive restarts due to persist failures
|
// stashState outside supervise because StashState should survive restarts due to persist failures
|
||||||
val stashState = new StashState(ctx.asInstanceOf[ActorContext[InternalProtocol]], settings)
|
val stashState = new StashState(ctx.asInstanceOf[ActorContext[InternalProtocol]], settings)
|
||||||
|
|
@ -261,6 +265,14 @@ private[pekko] final case class EventSourcedBehaviorImpl[Command, Event, State](
|
||||||
copy(snapshotPluginId = if (id != "") Some(id) else None)
|
copy(snapshotPluginId = if (id != "") Some(id) else None)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def withJournalPluginConfig(config: Option[Config]): EventSourcedBehavior[Command, Event, State] = {
|
||||||
|
copy(journalPluginConfig = config)
|
||||||
|
}
|
||||||
|
|
||||||
|
override def withSnapshotPluginConfig(config: Option[Config]): EventSourcedBehavior[Command, Event, State] = {
|
||||||
|
copy(snapshotPluginConfig = config)
|
||||||
|
}
|
||||||
|
|
||||||
override def withSnapshotSelectionCriteria(
|
override def withSnapshotSelectionCriteria(
|
||||||
selection: SnapshotSelectionCriteria): EventSourcedBehavior[Command, Event, State] = {
|
selection: SnapshotSelectionCriteria): EventSourcedBehavior[Command, Event, State] = {
|
||||||
copy(recovery = Recovery(selection.toClassic))
|
copy(recovery = Recovery(selection.toClassic))
|
||||||
|
|
|
||||||
|
|
@ -13,26 +13,40 @@
|
||||||
|
|
||||||
package org.apache.pekko.persistence.typed.internal
|
package org.apache.pekko.persistence.typed.internal
|
||||||
|
|
||||||
import java.util.concurrent.TimeUnit
|
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
|
||||||
import org.apache.pekko
|
import org.apache.pekko
|
||||||
import pekko.actor.typed.ActorSystem
|
import pekko.actor.typed.ActorSystem
|
||||||
import pekko.annotation.InternalApi
|
import pekko.annotation.InternalApi
|
||||||
import pekko.persistence.Persistence
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
@InternalApi private[pekko] object EventSourcedSettings {
|
@InternalApi private[pekko] object EventSourcedSettings {
|
||||||
|
|
||||||
def apply(system: ActorSystem[_], journalPluginId: String, snapshotPluginId: String): EventSourcedSettings =
|
def apply(
|
||||||
apply(system.settings.config, journalPluginId, snapshotPluginId)
|
system: ActorSystem[_],
|
||||||
|
journalPluginId: String,
|
||||||
|
snapshotPluginId: String
|
||||||
|
): EventSourcedSettings =
|
||||||
|
apply(system.settings.config, journalPluginId, snapshotPluginId, None, None)
|
||||||
|
|
||||||
def apply(config: Config, journalPluginId: String, snapshotPluginId: String): EventSourcedSettings = {
|
def apply(
|
||||||
|
system: ActorSystem[_],
|
||||||
|
journalPluginId: String,
|
||||||
|
snapshotPluginId: String,
|
||||||
|
journalPluginConfig: Option[Config],
|
||||||
|
snapshotPluginConfig: Option[Config]
|
||||||
|
): EventSourcedSettings =
|
||||||
|
apply(system.settings.config, journalPluginId, snapshotPluginId, journalPluginConfig, snapshotPluginConfig)
|
||||||
|
|
||||||
|
def apply(
|
||||||
|
config: Config,
|
||||||
|
journalPluginId: String,
|
||||||
|
snapshotPluginId: String,
|
||||||
|
journalPluginConfig: Option[Config],
|
||||||
|
snapshotPluginConfig: Option[Config]
|
||||||
|
): EventSourcedSettings = {
|
||||||
val typedConfig = config.getConfig("pekko.persistence.typed")
|
val typedConfig = config.getConfig("pekko.persistence.typed")
|
||||||
|
|
||||||
val stashOverflowStrategy = typedConfig.getString("stash-overflow-strategy").toLowerCase match {
|
val stashOverflowStrategy = typedConfig.getString("stash-overflow-strategy").toLowerCase match {
|
||||||
|
|
@ -47,36 +61,18 @@ import pekko.persistence.Persistence
|
||||||
|
|
||||||
val logOnStashing = typedConfig.getBoolean("log-stashing")
|
val logOnStashing = typedConfig.getBoolean("log-stashing")
|
||||||
|
|
||||||
val journalConfig = journalConfigFor(config, journalPluginId)
|
|
||||||
val recoveryEventTimeout: FiniteDuration =
|
|
||||||
journalConfig.getDuration("recovery-event-timeout", TimeUnit.MILLISECONDS).millis
|
|
||||||
|
|
||||||
val useContextLoggerForInternalLogging = typedConfig.getBoolean("use-context-logger-for-internal-logging")
|
val useContextLoggerForInternalLogging = typedConfig.getBoolean("use-context-logger-for-internal-logging")
|
||||||
|
|
||||||
Persistence.verifyPluginConfigExists(config, snapshotPluginId, "Snapshot store")
|
|
||||||
|
|
||||||
EventSourcedSettings(
|
EventSourcedSettings(
|
||||||
stashCapacity = stashCapacity,
|
stashCapacity = stashCapacity,
|
||||||
stashOverflowStrategy,
|
stashOverflowStrategy,
|
||||||
logOnStashing = logOnStashing,
|
logOnStashing = logOnStashing,
|
||||||
recoveryEventTimeout,
|
|
||||||
journalPluginId,
|
journalPluginId,
|
||||||
snapshotPluginId,
|
snapshotPluginId,
|
||||||
|
journalPluginConfig,
|
||||||
|
snapshotPluginConfig,
|
||||||
useContextLoggerForInternalLogging)
|
useContextLoggerForInternalLogging)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def journalConfigFor(config: Config, journalPluginId: String): Config = {
|
|
||||||
def defaultJournalPluginId = {
|
|
||||||
val configPath = config.getString("pekko.persistence.journal.plugin")
|
|
||||||
Persistence.verifyPluginConfigIsDefined(configPath, "Default journal")
|
|
||||||
configPath
|
|
||||||
}
|
|
||||||
|
|
||||||
val configPath = if (journalPluginId == "") defaultJournalPluginId else journalPluginId
|
|
||||||
Persistence.verifyPluginConfigExists(config, configPath, "Journal")
|
|
||||||
config.getConfig(configPath).withFallback(config.getConfig(Persistence.JournalFallbackConfigPath))
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -87,9 +83,10 @@ private[pekko] final case class EventSourcedSettings(
|
||||||
stashCapacity: Int,
|
stashCapacity: Int,
|
||||||
stashOverflowStrategy: StashOverflowStrategy,
|
stashOverflowStrategy: StashOverflowStrategy,
|
||||||
logOnStashing: Boolean,
|
logOnStashing: Boolean,
|
||||||
recoveryEventTimeout: FiniteDuration,
|
|
||||||
journalPluginId: String,
|
journalPluginId: String,
|
||||||
snapshotPluginId: String,
|
snapshotPluginId: String,
|
||||||
|
journalPluginConfig: Option[Config],
|
||||||
|
snapshotPluginConfig: Option[Config],
|
||||||
useContextLoggerForInternalLogging: Boolean) {
|
useContextLoggerForInternalLogging: Boolean) {
|
||||||
|
|
||||||
require(journalPluginId != null, "journal plugin id must not be null; use empty string for 'default' journal")
|
require(journalPluginId != null, "journal plugin id must not be null; use empty string for 'default' journal")
|
||||||
|
|
|
||||||
|
|
@ -230,7 +230,7 @@ private[pekko] final class ReplayingEvents[C, E, S](
|
||||||
this
|
this
|
||||||
} else {
|
} else {
|
||||||
val msg =
|
val msg =
|
||||||
s"Replay timed out, didn't get event within [${setup.settings.recoveryEventTimeout}], highest sequence number seen [${state.seqNr}]"
|
s"Replay timed out, didn't get event within [${setup.recoveryEventTimeout}], highest sequence number seen [${state.seqNr}]"
|
||||||
onRecoveryFailure(new RecoveryTimedOut(msg), None)
|
onRecoveryFailure(new RecoveryTimedOut(msg), None)
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
|
|
||||||
|
|
@ -127,7 +127,7 @@ private[pekko] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetu
|
||||||
if (snapshot) {
|
if (snapshot) {
|
||||||
// we know we're in snapshotting mode; snapshot recovery timeout arrived
|
// we know we're in snapshotting mode; snapshot recovery timeout arrived
|
||||||
val ex = new RecoveryTimedOut(
|
val ex = new RecoveryTimedOut(
|
||||||
s"Recovery timed out, didn't get snapshot within ${setup.settings.recoveryEventTimeout}")
|
s"Recovery timed out, didn't get snapshot within ${setup.recoveryEventTimeout}")
|
||||||
onRecoveryFailure(ex)
|
onRecoveryFailure(ex)
|
||||||
} else Behaviors.same // ignore, since we received the snapshot already
|
} else Behaviors.same // ignore, since we received the snapshot already
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,7 @@ package org.apache.pekko.persistence.typed.javadsl
|
||||||
import java.util.Collections
|
import java.util.Collections
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
|
|
||||||
|
import com.typesafe.config.Config
|
||||||
import org.apache.pekko
|
import org.apache.pekko
|
||||||
import pekko.actor.typed
|
import pekko.actor.typed
|
||||||
import pekko.actor.typed.BackoffSupervisorStrategy
|
import pekko.actor.typed.BackoffSupervisorStrategy
|
||||||
|
|
@ -26,6 +27,7 @@ import pekko.annotation.InternalApi
|
||||||
import pekko.persistence.typed._
|
import pekko.persistence.typed._
|
||||||
import pekko.persistence.typed.EventAdapter
|
import pekko.persistence.typed.EventAdapter
|
||||||
import pekko.persistence.typed.internal._
|
import pekko.persistence.typed.internal._
|
||||||
|
import pekko.util.OptionConverters._
|
||||||
import pekko.util.unused
|
import pekko.util.unused
|
||||||
|
|
||||||
abstract class EventSourcedBehavior[Command, Event, State] private[pekko] (
|
abstract class EventSourcedBehavior[Command, Event, State] private[pekko] (
|
||||||
|
|
@ -130,6 +132,16 @@ abstract class EventSourcedBehavior[Command, Event, State] private[pekko] (
|
||||||
*/
|
*/
|
||||||
def snapshotPluginId: String = ""
|
def snapshotPluginId: String = ""
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Override and define the journal plugin config that this actor should use instead of the default.
|
||||||
|
*/
|
||||||
|
def journalPluginConfig: Optional[Config] = Optional.empty()
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Override and define the snapshot store plugin config that this actor should use instead of the default.
|
||||||
|
*/
|
||||||
|
def snapshotPluginConfig: Optional[Config] = Optional.empty()
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Override and define the snapshot selection criteria used by this actor instead of the default.
|
* Override and define the snapshot selection criteria used by this actor instead of the default.
|
||||||
* By default the most recent snapshot is used, and the remaining state updates are recovered by replaying events
|
* By default the most recent snapshot is used, and the remaining state updates are recovered by replaying events
|
||||||
|
|
@ -225,6 +237,8 @@ abstract class EventSourcedBehavior[Command, Event, State] private[pekko] (
|
||||||
.withJournalPluginId(journalPluginId)
|
.withJournalPluginId(journalPluginId)
|
||||||
.withSnapshotPluginId(snapshotPluginId)
|
.withSnapshotPluginId(snapshotPluginId)
|
||||||
.withRecovery(recovery.asScala)
|
.withRecovery(recovery.asScala)
|
||||||
|
.withJournalPluginConfig(journalPluginConfig.toScala)
|
||||||
|
.withSnapshotPluginConfig(snapshotPluginConfig.toScala)
|
||||||
|
|
||||||
val handler = signalHandler()
|
val handler = signalHandler()
|
||||||
val behaviorWithSignalHandler =
|
val behaviorWithSignalHandler =
|
||||||
|
|
|
||||||
|
|
@ -13,6 +13,7 @@
|
||||||
|
|
||||||
package org.apache.pekko.persistence.typed.scaladsl
|
package org.apache.pekko.persistence.typed.scaladsl
|
||||||
|
|
||||||
|
import com.typesafe.config.Config
|
||||||
import org.apache.pekko
|
import org.apache.pekko
|
||||||
import pekko.actor.typed.BackoffSupervisorStrategy
|
import pekko.actor.typed.BackoffSupervisorStrategy
|
||||||
import pekko.actor.typed.Behavior
|
import pekko.actor.typed.Behavior
|
||||||
|
|
@ -162,6 +163,20 @@ object EventSourcedBehavior {
|
||||||
*/
|
*/
|
||||||
def withSnapshotPluginId(id: String): EventSourcedBehavior[Command, Event, State]
|
def withSnapshotPluginId(id: String): EventSourcedBehavior[Command, Event, State]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Change the journal plugin config that this actor should use.
|
||||||
|
*
|
||||||
|
* @since 1.1.3
|
||||||
|
*/
|
||||||
|
def withJournalPluginConfig(id: Option[Config]): EventSourcedBehavior[Command, Event, State]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Change the snapshot store plugin config that this actor should use.
|
||||||
|
*
|
||||||
|
* @since 1.1.3
|
||||||
|
*/
|
||||||
|
def withSnapshotPluginConfig(id: Option[Config]): EventSourcedBehavior[Command, Event, State]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Changes the snapshot selection criteria used by this behavior.
|
* Changes the snapshot selection criteria used by this behavior.
|
||||||
* By default the most recent snapshot is used, and the remaining state updates are recovered by replaying events
|
* By default the most recent snapshot is used, and the remaining state updates are recovered by replaying events
|
||||||
|
|
|
||||||
|
|
@ -74,9 +74,10 @@ class StashStateSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with
|
||||||
stashCapacity = capacity,
|
stashCapacity = capacity,
|
||||||
stashOverflowStrategy = StashOverflowStrategy.Fail,
|
stashOverflowStrategy = StashOverflowStrategy.Fail,
|
||||||
logOnStashing = false,
|
logOnStashing = false,
|
||||||
recoveryEventTimeout = 3.seconds,
|
|
||||||
journalPluginId = "",
|
journalPluginId = "",
|
||||||
snapshotPluginId = "",
|
snapshotPluginId = "",
|
||||||
|
journalPluginConfig = None,
|
||||||
|
snapshotPluginConfig = None,
|
||||||
useContextLoggerForInternalLogging = false)
|
useContextLoggerForInternalLogging = false)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue