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:
Domantas Petrauskas 2024-10-17 00:50:58 +03:00 committed by GitHub
parent 0e10d291db
commit 4afe7cf331
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
13 changed files with 287 additions and 49 deletions

View file

@ -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)))

View file

@ -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
}
})
}

View file

@ -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)

View file

@ -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")
}
}
}
}

View file

@ -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")

View file

@ -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)
} }

View file

@ -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))

View file

@ -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")

View file

@ -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 {

View file

@ -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

View file

@ -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 =

View file

@ -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

View file

@ -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)
} }