Deprecate LevelDB (#30213)

* Deprecate LevelDB

In preparation of moving into the testing infra (or deleting it completely) at some distant future point in time

* Remove leveldb tests where there also is an inmem one

* More details in deprecation text, recommend inmem + journal proxy for testing etc.
This commit is contained in:
Johan Andrén 2021-06-02 13:37:52 +02:00 committed by GitHub
parent 1c6f184127
commit d4fdf5639e
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
33 changed files with 79 additions and 490 deletions

View file

@ -6,18 +6,17 @@ package akka.persistence
import java.io.File
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations._
import akka.actor._
import akka.persistence.journal.AsyncWriteTarget._
import akka.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbStore }
import akka.testkit.TestProbe
import scala.annotation.nowarn
/*
# OS: OSX 10.9.3
# CPU: Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz
@ -49,6 +48,7 @@ class LevelDbBatchingBenchmark {
val batch_200 = List.fill(200) { AtomicWrite(PersistentRepr("data", 12, "pa")) }
@Setup(Level.Trial)
@nowarn("msg=deprecated")
def setup(): Unit = {
sys = ActorSystem("sys")
deleteStorage(sys)

View file

@ -9,7 +9,6 @@ import scala.concurrent.Promise
import scala.util.Failure
import scala.util.Success
import scala.util.Try
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
@ -21,6 +20,8 @@ import akka.persistence._
import akka.persistence.journal.leveldb.SharedLeveldbJournal
import akka.persistence.journal.leveldb.SharedLeveldbStore
import scala.annotation.nowarn
/**
* Utility program that removes the internal data stored with Akka Persistence
* by the Cluster `ShardCoordinator`. The data contains the locations of the
@ -91,6 +92,7 @@ object RemoveInternalClusterShardingData {
if (journalPluginId == "") system.settings.config.getString("akka.persistence.journal.plugin")
else journalPluginId
if (resolvedJournalPluginId == "akka.persistence.journal.leveldb-shared") {
@nowarn("msg=deprecated")
val store = system.actorOf(Props[SharedLeveldbStore](), "store")
SharedLeveldbJournal.setStore(store, system)
}

View file

@ -22,6 +22,8 @@ import akka.remote.testconductor.RoleName
import akka.testkit._
import akka.testkit.TestEvent.Mute
import scala.annotation.nowarn
object ClusterShardingSpec {
//#counter-actor
case object Increment
@ -273,6 +275,7 @@ class DDataClusterShardingWithEntityRecoveryMultiJvmNode5 extends DDataClusterSh
class DDataClusterShardingWithEntityRecoveryMultiJvmNode6 extends DDataClusterShardingWithEntityRecoverySpec
class DDataClusterShardingWithEntityRecoveryMultiJvmNode7 extends DDataClusterShardingWithEntityRecoverySpec
@nowarn("msg=deprecated")
abstract class ClusterShardingSpec(multiNodeConfig: ClusterShardingSpecConfig)
extends MultiNodeClusterShardingSpec(multiNodeConfig)
with ImplicitSender

View file

@ -5,11 +5,8 @@
package akka.cluster.sharding
import java.io.File
import scala.concurrent.duration._
import org.apache.commons.io.FileUtils
import akka.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, ActorSystem, Identify, PoisonPill, Props }
import akka.cluster.MultiNodeClusterSpec
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
@ -21,6 +18,8 @@ import akka.serialization.jackson.CborSerializable
import akka.testkit.{ TestActors, TestProbe }
import akka.util.ccompat._
import scala.annotation.nowarn
@ccompatUsedUntil213
object MultiNodeClusterShardingSpec {
@ -207,6 +206,7 @@ abstract class MultiNodeClusterShardingSpec(val config: MultiNodeClusterSharding
* @param startOn the node to start the `SharedLeveldbStore` store on
* @param setStoreOn the nodes to `SharedLeveldbJournal.setStore` on
*/
@nowarn("msg=deprecated")
protected def startPersistence(startOn: RoleName, setStoreOn: Seq[RoleName]): Unit = {
info("Setting up setup shared journal.")

View file

@ -70,6 +70,9 @@ This plugin writes events to a local LevelDB instance.
The LevelDB plugin cannot be used in an Akka Cluster since the storage is in a local file system.
@@@
The LevelDB journal is deprecated and it is not advised to build new applications with it.
As a replacement we recommend using [Akka Persistence JDBC](https://doc.akka.io/docs/akka-persistence-jdbc/current/index.html).
The LevelDB journal plugin config entry is `akka.persistence.journal.leveldb`. Enable this plugin by
defining config property:
@ -99,14 +102,8 @@ this end, LevelDB offers a special journal compaction function that is exposed v
### Shared LevelDB journal
For testing purposes a LevelDB instance can also be shared by multiple actor systems (on the same or on different nodes). This, for
example, allows persistent actors to failover to a backup node and continue using the shared journal instance from the
backup node.
@@@ warning
A shared LevelDB instance is a single point of failure and should therefore only be used for testing
purposes.
@@@
The LevelDB journal is deprecated and will be removed from a future Akka version, it is not advised to build new
applications with it. For testing in a multi node environment the "inmem" journal together with the @ref[proxy plugin](#persistence-plugin-proxy) can be used, but the actual journal used in production of applications is also a good choice.
@@@ note
This plugin has been supplanted by @ref:[Persistence Plugin Proxy](#persistence-plugin-proxy).
@ -165,20 +162,21 @@ you don't have to configure it.
### Persistence Plugin Proxy
For testing purposes a persistence plugin proxy allows sharing of journals and snapshot stores across multiple actor systems (on the same or
on different nodes). This, for example, allows persistent actors to failover to a backup node and continue using the
shared journal instance from the backup node. The proxy works by forwarding all the journal/snapshot store messages to a
single, shared, persistence plugin instance, and therefore supports any use case supported by the proxied plugin.
For testing purposes a persistence plugin proxy allows sharing of a journal and snapshot store on a single node across multiple
actor systems (on the same or on different nodes). This, for example, allows persistent actors to failover to a backup
node and continue using the shared journal instance from the backup node. The proxy works by forwarding all the
journal/snapshot store messages to a single, shared, persistence plugin instance, and therefore supports any use case
supported by the proxied plugin.
@@@ warning
A shared journal/snapshot store is a single point of failure and should therefore only be used for testing
A shared journal/snapshot store is a single point of failure and should only be used for testing
purposes.
@@@
The journal and snapshot store proxies are controlled via the `akka.persistence.journal.proxy` and
`akka.persistence.snapshot-store.proxy` configuration entries, respectively. Set the `target-journal-plugin` or
`target-snapshot-store-plugin` keys to the underlying plugin you wish to use (for example:
`akka.persistence.journal.leveldb`). The `start-target-journal` and `start-target-snapshot-store` keys should be
`akka.persistence.journal.inmem`). The `start-target-journal` and `start-target-snapshot-store` keys should be
set to `on` in exactly one actor system - this is the system that will instantiate the shared persistence plugin.
Next, the proxy needs to be told how to find the shared plugin. This can be done by setting the `target-journal-address`
and `target-snapshot-store-address` configuration keys, or programmatically by calling the

View file

@ -1,5 +1,8 @@
# Persistence Query for LevelDB
The LevelDB journal and query plugin is deprecated and it is not advised to build new applications with it.
As a replacement we recommend using [Akka Persistence JDBC](https://doc.akka.io/docs/akka-persistence-jdbc/current/index.html).
## Dependency
To use Persistence Query, you must add the following dependency in your project:

View file

@ -266,8 +266,6 @@ used the next time this projection is started. This pattern is implemented in th
Query plugins are various (mostly community driven) `ReadJournal` implementations for all kinds
of available datastores. The complete list of available plugins is maintained on the Akka Persistence Query [Community Plugins](https://akka.io/community/#plugins-to-akka-persistence-query) page.
The plugin for LevelDB is described in @ref:[Persistence Query for LevelDB](persistence-query-leveldb.md).
This section aims to provide tips and guide plugin developers through implementing a custom query plugin.
Most users will not need to implement journals themselves, except if targeting a not yet supported datastore.

View file

@ -44,7 +44,7 @@ recover its state from these messages.
case of sender and receiver JVM crashes.
* `AsyncWriteJournal`: A journal stores the sequence of messages sent to a persistent actor. An application can control which messages
are journaled and which are received by the persistent actor without being journaled. Journal maintains `highestSequenceNr` that is increased on each message.
The storage backend of a journal is pluggable. The persistence extension comes with a "leveldb" journal plugin, which writes to the local filesystem.
The storage backend of a journal is pluggable.
Replicated journals are available as [Community plugins](https://akka.io/community/).
* *Snapshot store*: A snapshot store persists snapshots of a persistent actor's state. Snapshots are
used for optimizing recovery times. The storage backend of a snapshot store is pluggable.
@ -828,6 +828,10 @@ For more advanced schema evolution techniques refer to the @ref:[Persistence - S
## Testing with LevelDB journal
The LevelDB journal is deprecated and will be removed from a future Akka version, it is not advised to build new applications
with it. For testing the built in "inmem" journal or the actual journal that will be used in production of the application
is recommended. See @ref[Persistence Plugins](persistence-plugins.md) for some journal implementation choices.
When running tests with LevelDB default settings in `sbt`, make sure to set `fork := true` in your sbt project. Otherwise, you'll see an `UnsatisfiedLinkError`. Alternatively, you can switch to a LevelDB Java port by setting
@@snip [PersistencePluginDocSpec.scala](/akka-docs/src/test/scala/docs/persistence/PersistencePluginDocSpec.scala) { #native-config }
@ -836,9 +840,7 @@ or
@@snip [PersistencePluginDocSpec.scala](/akka-docs/src/test/scala/docs/persistence/PersistencePluginDocSpec.scala) { #shared-store-native-config }
in your Akka configuration. The LevelDB Java port is for testing purposes only.
Also note that for the LevelDB Java port, you will need the following dependencies:
in your Akka configuration. Also note that for the LevelDB Java port, you will need the following dependencies:
@@dependency[sbt,Maven,Gradle] {
group="org.iq80.leveldb"

View file

@ -9,6 +9,7 @@ import com.typesafe.config.Config
import akka.actor.ExtendedActorSystem
import akka.persistence.query.ReadJournalProvider
@deprecated("Use another journal/query implementation", "2.6.15")
class LeveldbReadJournalProvider(system: ExtendedActorSystem, config: Config) extends ReadJournalProvider {
override val scaladslReadJournal: scaladsl.LeveldbReadJournal =

View file

@ -25,6 +25,7 @@ import akka.stream.javadsl.Source
* for the default [[LeveldbReadJournal#Identifier]]. See `reference.conf`.
*
*/
@deprecated("Use another journal implementation", "2.6.15")
class LeveldbReadJournal(scaladslReadJournal: akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal)
extends ReadJournal
with PersistenceIdsQuery

View file

@ -39,6 +39,7 @@ import akka.util.ByteString
* absolute path corresponding to the identifier, which is `"akka.persistence.query.journal.leveldb"`
* for the default [[LeveldbReadJournal#Identifier]]. See `reference.conf`.
*/
@deprecated("Use another journal implementation", "2.6.15")
class LeveldbReadJournal(system: ExtendedActorSystem, config: Config)
extends ReadJournal
with PersistenceIdsQuery

View file

@ -5,7 +5,6 @@
package akka.persistence.query.journal.leveldb
import scala.concurrent.duration._
import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.PersistenceIdsQuery
@ -13,6 +12,8 @@ import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import scala.annotation.nowarn
object AllPersistenceIdsSpec {
val config = """
akka.loglevel = INFO
@ -27,6 +28,7 @@ object AllPersistenceIdsSpec {
class AllPersistenceIdsSpec extends AkkaSpec(AllPersistenceIdsSpec.config) with Cleanup with ImplicitSender {
@nowarn("msg=deprecated")
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
"Leveldb query AllPersistenceIds" must {

View file

@ -5,7 +5,6 @@
package akka.persistence.query.journal.leveldb
import scala.concurrent.duration._
import akka.actor.ActorRef
import akka.persistence.query.EventEnvelope
import akka.persistence.query.PersistenceQuery
@ -15,6 +14,8 @@ import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import scala.annotation.nowarn
object EventsByPersistenceIdSpec {
val config = """
akka.loglevel = INFO
@ -30,6 +31,7 @@ object EventsByPersistenceIdSpec {
class EventsByPersistenceIdSpec extends AkkaSpec(EventsByPersistenceIdSpec.config) with Cleanup with ImplicitSender {
@nowarn("msg=deprecated")
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
def setup(persistenceId: String): ActorRef = {

View file

@ -5,7 +5,6 @@
package akka.persistence.query.journal.leveldb
import scala.concurrent.duration._
import akka.persistence.journal.Tagged
import akka.persistence.journal.WriteEventAdapter
import akka.persistence.query.EventEnvelope
@ -18,6 +17,8 @@ import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import scala.annotation.nowarn
object EventsByTagSpec {
val config = s"""
akka.loglevel = INFO
@ -61,6 +62,7 @@ class ColorTagger extends WriteEventAdapter {
class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config) with Cleanup with ImplicitSender {
@nowarn("msg=deprecated")
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
"Leveldb query EventsByTag" must {
@ -212,6 +214,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config) with Cleanup with
}
"finds events without refresh" in {
@nowarn("msg=deprecated")
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal]("leveldb-no-refresh")
val d = system.actorOf(TestActor.props("y"))

View file

@ -6,18 +6,15 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.util.Failure
import scala.util.Success
import scala.util.Try
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import org.scalatest.wordspec.AnyWordSpecLike
import akka.Done
import akka.actor.ActorInitializationException
import akka.actor.testkit.typed.TestException
@ -47,6 +44,8 @@ import akka.persistence.typed.SnapshotSelectionCriteria
import akka.serialization.jackson.CborSerializable
import akka.stream.scaladsl.Sink
import scala.annotation.nowarn
object EventSourcedBehaviorSpec {
class SlowInMemorySnapshotStore extends SnapshotStore {
@ -286,6 +285,7 @@ class EventSourcedBehaviorSpec
import EventSourcedBehaviorSpec._
@nowarn("msg=deprecated")
val queries: LeveldbReadJournal =
PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)

View file

@ -6,10 +6,8 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
import com.typesafe.config.ConfigFactory
import org.scalatest.wordspec.AnyWordSpecLike
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
@ -26,6 +24,8 @@ import akka.serialization.jackson.CborSerializable
import akka.stream.scaladsl.Sink
import akka.testkit.JavaSerializable
import scala.annotation.nowarn
object EventSourcedEventAdapterSpec {
private val conf = ConfigFactory.parseString(s"""
@ -100,6 +100,7 @@ class EventSourcedEventAdapterSpec
val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId.ofUniqueId(s"c${pidCounter.incrementAndGet()})")
@nowarn("msg=deprecated")
val queries: LeveldbReadJournal =
PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)

View file

@ -6,11 +6,9 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import org.scalatest.wordspec.AnyWordSpecLike
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe
@ -21,6 +19,8 @@ import akka.persistence.typed.PersistenceId
import akka.persistence.typed.SnapshotAdapter
import akka.serialization.jackson.CborSerializable
import scala.annotation.nowarn
object EventSourcedSnapshotAdapterSpec {
private val conf: Config = ConfigFactory.parseString(s"""
akka.persistence.journal.leveldb.dir = "target/typed-persistence-${UUID.randomUUID().toString}"
@ -45,6 +45,7 @@ class EventSourcedSnapshotAdapterSpec
val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId.ofUniqueId(s"c${pidCounter.incrementAndGet()})")
@nowarn("msg=deprecated")
val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)

View file

@ -24,6 +24,7 @@ import akka.util.Timeout
*
* Journal backed by a local LevelDB store. For production use.
*/
@deprecated("Use another journal implementation", "2.6.15")
private[persistence] class LeveldbJournal(cfg: Config) extends AsyncWriteJournal with LeveldbStore {
import LeveldbJournal._

View file

@ -21,6 +21,7 @@ import akka.persistence.journal.AsyncWriteTarget
* set for each actor system that uses the store via `SharedLeveldbJournal.setStore`. The
* shared LevelDB store is for testing only.
*/
@deprecated("Use another journal implementation or the in-mem journal in combination with the journal-proxy", "2.6.15")
class SharedLeveldbStore(cfg: Config) extends LeveldbStore {
import AsyncWriteTarget._
import context.dispatcher

View file

@ -8,8 +8,6 @@ import scala.concurrent.duration._
import scala.util.Failure
import scala.util.control.NoStackTrace
import com.typesafe.config._
import akka.actor._
import akka.persistence.AtLeastOnceDelivery.{ AtLeastOnceDeliverySnapshot, UnconfirmedWarning }
import akka.testkit._
@ -189,7 +187,10 @@ object AtLeastOnceDeliverySpec {
}
abstract class AtLeastOnceDeliverySpec(config: Config) extends PersistenceSpec(config) with ImplicitSender {
class AtLeastOnceDeliverySpec
extends PersistenceSpec(PersistenceSpec.config("inmem", "AtLeastOnceDeliverySpec"))
with ImplicitSender {
import akka.persistence.AtLeastOnceDeliverySpec._
"AtLeastOnceDelivery" must {
@ -440,9 +441,3 @@ abstract class AtLeastOnceDeliverySpec(config: Config) extends PersistenceSpec(c
}
}
}
class LeveldbAtLeastOnceDeliverySpec
extends AtLeastOnceDeliverySpec(PersistenceSpec.config("leveldb", "AtLeastOnceDeliverySpec"))
class InmemAtLeastOnceDeliverySpec
extends AtLeastOnceDeliverySpec(PersistenceSpec.config("inmem", "AtLeastOnceDeliverySpec"))

View file

@ -109,12 +109,14 @@ object EndToEndEventAdapterSpec {
}
abstract class EndToEndEventAdapterSpec(journalName: String, journalConfig: Config)
extends AnyWordSpecLike
with Matchers
with BeforeAndAfterAll {
// needs persistence between actor systems, thus not running with the inmem journal
// FIXME move to inmem + proxy
class EndToEndEventAdapterSpec extends AnyWordSpecLike with Matchers with BeforeAndAfterAll {
import EndToEndEventAdapterSpec._
val journalName = "leveldb"
val journalConfig = PersistenceSpec.config("leveldb", "LeveldbEndToEndEventAdapterSpec")
val storageLocations = List("akka.persistence.journal.leveldb.dir").map(s => new File(journalConfig.getString(s)))
override protected def beforeAll(): Unit = {
@ -253,7 +255,3 @@ abstract class EndToEndEventAdapterSpec(journalName: String, journalConfig: Conf
}
}
}
// needs persistence between actor systems, thus not running with the inmem journal
class LeveldbEndToEndEventAdapterSpec
extends EndToEndEventAdapterSpec("leveldb", PersistenceSpec.config("leveldb", "LeveldbEndToEndEventAdapterSpec"))

View file

@ -10,7 +10,6 @@ import com.typesafe.config.{ Config, ConfigFactory }
import akka.actor._
import akka.event.Logging
import akka.persistence.EventAdapterSpec.{ Tagged, UserDataChanged }
import akka.persistence.journal.{ EventAdapter, EventSeq, SingleEventSeq }
import akka.testkit.ImplicitSender
@ -98,13 +97,13 @@ object EventAdapterSpec {
}
abstract class EventAdapterSpec(journalName: String, journalConfig: Config, adapterConfig: Config)
class EventAdapterSpec(journalName: String, journalConfig: Config, adapterConfig: Config)
extends PersistenceSpec(journalConfig.withFallback(adapterConfig))
with ImplicitSender {
import EventAdapterSpec._
def this(journalName: String) =
def this() =
this(
"inmem",
PersistenceSpec.config("inmem", "InmemPersistentTaggingSpec"),
@ -183,13 +182,6 @@ abstract class EventAdapterSpec(journalName: String, journalConfig: Config, adap
toJournal(event, "with-actor-system") should equal(event)
fromJournal(event, "with-actor-system") should equal(SingleEventSeq(event))
}
}
}
trait ReplayPassThrough { this: EventAdapterSpec =>
"EventAdapter" must {
"store events after applying adapter" in {
val replayPassThroughJournalId = "replay-pass-through-adapter-journal"
@ -210,12 +202,6 @@ trait ReplayPassThrough { this: EventAdapterSpec =>
expectMsg(Tagged(m1, Set("adult")))
expectMsg(m2)
}
}
}
trait NoAdapters { this: EventAdapterSpec =>
"EventAdapter" must {
"work when plugin defines no adapter" in {
val p2 = persister("p2", journalId = "no-adapter")
val m1 = UserDataChanged("name", 64)
@ -234,15 +220,5 @@ trait NoAdapters { this: EventAdapterSpec =>
expectMsg(m1)
expectMsg(m2)
}
}
}
// this style of testing allows us to try different leveldb journal plugin configurations
// because it always would use the same leveldb directory anyway (based on class name),
// yet we need different instances of the plugin. For inmem it does not matter, it can survive many instances.
class InmemEventAdapterSpec extends EventAdapterSpec("inmem") with ReplayPassThrough with NoAdapters
class LeveldbBaseEventAdapterSpec extends EventAdapterSpec("leveldb")
class LeveldbReplayPassThroughEventAdapterSpec extends EventAdapterSpec("leveldb") with ReplayPassThrough
class LeveldbNoAdaptersEventAdapterSpec extends EventAdapterSpec("leveldb") with NoAdapters

View file

@ -35,7 +35,6 @@ object OptionalSnapshotStoreSpec {
class OptionalSnapshotStoreSpec extends PersistenceSpec(ConfigFactory.parseString(s"""
akka.persistence.publish-plugin-commands = on
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
akka.persistence.journal.leveldb.dir = "target/journal-${classOf[OptionalSnapshotStoreSpec].getName}"
akka.actor.warn-about-java-serializer-usage = off

View file

@ -116,7 +116,7 @@ object PerformanceSpec {
class PerformanceSpec
extends PersistenceSpec(
PersistenceSpec
.config("leveldb", "PerformanceSpec", serialization = "off")
.config("inmem", "PerformanceSpec", serialization = "off")
.withFallback(ConfigFactory.parseString(PerformanceSpec.config)))
with ImplicitSender {
import PerformanceSpec._

View file

@ -79,10 +79,7 @@ object PersistenceSpec {
trait Cleanup { this: AkkaSpec =>
val storageLocations =
List(
"akka.persistence.journal.leveldb.dir",
"akka.persistence.journal.leveldb-shared.store.dir",
"akka.persistence.snapshot-store.local.dir").map(s => new File(system.settings.config.getString(s)))
List("akka.persistence.snapshot-store.local.dir").map(s => new File(system.settings.config.getString(s)))
override protected def atStartup(): Unit = {
storageLocations.foreach(FileUtils.deleteDirectory)

View file

@ -51,18 +51,6 @@ object PersistentActorSpec {
def receiveRecover = updateState
}
trait LevelDbRuntimePluginConfig extends PersistenceIdentity with RuntimePluginConfig {
val providedConfig: Config
override def journalPluginId: String = s"custom.persistence.journal.leveldb"
override def snapshotPluginId: String = "custom.persistence.snapshot-store.local"
override def journalPluginConfig: Config = providedConfig
override def snapshotPluginConfig: Config = providedConfig
}
trait InmemRuntimePluginConfig extends PersistenceIdentity with RuntimePluginConfig {
val providedConfig: Config
@ -101,9 +89,6 @@ object PersistentActorSpec {
case _ => super.onPersistFailure(cause, event, seqNr)
}
}
class Behavior1PersistentActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends Behavior1PersistentActor(name)
with LevelDbRuntimePluginConfig
class Behavior1PersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends Behavior1PersistentActor(name)
with InmemRuntimePluginConfig
@ -115,9 +100,6 @@ object PersistentActorSpec {
persistAll(Seq(Evt(s"${data}-3"), Evt(s"${data}-4")))(updateState)
}
}
class Behavior2PersistentActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends Behavior2PersistentActor(name)
with LevelDbRuntimePluginConfig
class Behavior2PersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends Behavior2PersistentActor(name)
with InmemRuntimePluginConfig
@ -129,9 +111,6 @@ object PersistentActorSpec {
updateState(Evt(s"${data}-10"))
}
}
class Behavior3PersistentActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends Behavior3PersistentActor(name)
with LevelDbRuntimePluginConfig
class Behavior3PersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends Behavior3PersistentActor(name)
with InmemRuntimePluginConfig
@ -154,11 +133,6 @@ object PersistentActorSpec {
}
}
}
class ChangeBehaviorInLastEventHandlerPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends ChangeBehaviorInLastEventHandlerPersistentActor(name)
with LevelDbRuntimePluginConfig
class ChangeBehaviorInLastEventHandlerPersistentActorWithInmemRuntimePluginConfig(
name: String,
val providedConfig: Config)
@ -183,11 +157,6 @@ object PersistentActorSpec {
}
}
}
class ChangeBehaviorInFirstEventHandlerPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends ChangeBehaviorInFirstEventHandlerPersistentActor(name)
with LevelDbRuntimePluginConfig
class ChangeBehaviorInFirstEventHandlerPersistentActorWithInmemRuntimePluginConfig(
name: String,
val providedConfig: Config)
@ -208,11 +177,6 @@ object PersistentActorSpec {
persist(Evt(s"${data}-0"))(updateState)
}
}
class ChangeBehaviorInCommandHandlerFirstPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends ChangeBehaviorInCommandHandlerFirstPersistentActor(name)
with LevelDbRuntimePluginConfig
class ChangeBehaviorInCommandHandlerFirstPersistentActorWithInmemRuntimePluginConfig(
name: String,
val providedConfig: Config)
@ -233,11 +197,6 @@ object PersistentActorSpec {
context.become(newBehavior)
}
}
class ChangeBehaviorInCommandHandlerLastPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends ChangeBehaviorInCommandHandlerLastPersistentActor(name)
with LevelDbRuntimePluginConfig
class ChangeBehaviorInCommandHandlerLastPersistentActorWithInmemRuntimePluginConfig(
name: String,
val providedConfig: Config)
@ -261,12 +220,6 @@ object PersistentActorSpec {
case "snap" => saveSnapshot(events)
}
}
class SnapshottingPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
probe: ActorRef,
val providedConfig: Config)
extends SnapshottingPersistentActor(name, probe)
with LevelDbRuntimePluginConfig
class SnapshottingPersistentActorWithInmemRuntimePluginConfig(
name: String,
probe: ActorRef,
@ -291,12 +244,6 @@ object PersistentActorSpec {
case "It's changing me" => probe ! "I am becoming"
}
}
class SnapshottingBecomingPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
probe: ActorRef,
val providedConfig: Config)
extends SnapshottingBecomingPersistentActor(name, probe)
with LevelDbRuntimePluginConfig
class SnapshottingBecomingPersistentActorWithInmemRuntimePluginConfig(
name: String,
probe: ActorRef,
@ -309,9 +256,6 @@ object PersistentActorSpec {
case Cmd("a") => persist(Evt("a"))(evt => sender() ! evt.data)
}
}
class ReplyInEventHandlerPersistentActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends ReplyInEventHandlerPersistentActor(name)
with LevelDbRuntimePluginConfig
class ReplyInEventHandlerPersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends ReplyInEventHandlerPersistentActor(name)
with InmemRuntimePluginConfig
@ -339,9 +283,6 @@ object PersistentActorSpec {
}
}
class AsyncPersistPersistentActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends AsyncPersistPersistentActor(name)
with LevelDbRuntimePluginConfig
class AsyncPersistPersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends AsyncPersistPersistentActor(name)
with InmemRuntimePluginConfig
@ -365,9 +306,6 @@ object PersistentActorSpec {
counter
}
}
class AsyncPersistThreeTimesPersistentActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends AsyncPersistThreeTimesPersistentActor(name)
with LevelDbRuntimePluginConfig
class AsyncPersistThreeTimesPersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends AsyncPersistThreeTimesPersistentActor(name)
with InmemRuntimePluginConfig
@ -392,11 +330,6 @@ object PersistentActorSpec {
}
}
}
class AsyncPersistSameEventTwicePersistentActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends AsyncPersistSameEventTwicePersistentActor(name)
with LevelDbRuntimePluginConfig
class AsyncPersistSameEventTwicePersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends AsyncPersistSameEventTwicePersistentActor(name)
with InmemRuntimePluginConfig
@ -418,9 +351,6 @@ object PersistentActorSpec {
sender() ! data
}
}
class PersistAllNilPersistentActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends PersistAllNilPersistentActor(name)
with LevelDbRuntimePluginConfig
class PersistAllNilPersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends PersistAllNilPersistentActor(name)
with InmemRuntimePluginConfig
@ -452,11 +382,6 @@ object PersistentActorSpec {
counter
}
}
class AsyncPersistAndPersistMixedSyncAsyncSyncPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends AsyncPersistAndPersistMixedSyncAsyncSyncPersistentActor(name)
with LevelDbRuntimePluginConfig
class AsyncPersistAndPersistMixedSyncAsyncSyncPersistentActorWithInmemRuntimePluginConfig(
name: String,
val providedConfig: Config)
@ -485,11 +410,6 @@ object PersistentActorSpec {
sendMsgCounter
}
}
class AsyncPersistAndPersistMixedSyncAsyncPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends AsyncPersistAndPersistMixedSyncAsyncPersistentActor(name)
with LevelDbRuntimePluginConfig
class AsyncPersistAndPersistMixedSyncAsyncPersistentActorWithInmemRuntimePluginConfig(
name: String,
val providedConfig: Config)
@ -507,9 +427,6 @@ object PersistentActorSpec {
}
}
}
class AsyncPersistHandlerCorrelationCheckWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends AsyncPersistHandlerCorrelationCheck(name)
with LevelDbRuntimePluginConfig
class AsyncPersistHandlerCorrelationCheckWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends AsyncPersistHandlerCorrelationCheck(name)
with InmemRuntimePluginConfig
@ -519,9 +436,6 @@ object PersistentActorSpec {
case Cmd("a") => persist(5)(evt => sender() ! evt)
}
}
class PrimitiveEventPersistentActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends PrimitiveEventPersistentActor(name)
with LevelDbRuntimePluginConfig
class PrimitiveEventPersistentActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends PrimitiveEventPersistentActor(name)
with InmemRuntimePluginConfig
@ -547,12 +461,6 @@ object PersistentActorSpec {
}
}
class HandleRecoveryFinishedEventPersistentActorWithLevelDbRuntimePluginConfig(
name: String,
probe: ActorRef,
val providedConfig: Config)
extends HandleRecoveryFinishedEventPersistentActor(name, probe)
with LevelDbRuntimePluginConfig
class HandleRecoveryFinishedEventPersistentActorWithInmemRuntimePluginConfig(
name: String,
probe: ActorRef,
@ -582,12 +490,6 @@ object PersistentActorSpec {
}
class DeferringAsyncWithPersistActor(name: String) extends DeferringWithPersistActor(name) with DeferAsync
class DeferringSyncWithPersistActor(name: String) extends DeferringWithPersistActor(name) with DeferSync
class DeferringAsyncWithPersistActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringAsyncWithPersistActor(name)
with LevelDbRuntimePluginConfig
class DeferringSyncWithPersistActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringSyncWithPersistActor(name)
with LevelDbRuntimePluginConfig
class DeferringAsyncWithPersistActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringAsyncWithPersistActor(name)
with InmemRuntimePluginConfig
@ -606,12 +508,6 @@ object PersistentActorSpec {
}
class DeferringAsyncWithAsyncPersistActor(name: String) extends DeferringWithAsyncPersistActor(name) with DeferAsync
class DeferringSyncWithAsyncPersistActor(name: String) extends DeferringWithAsyncPersistActor(name) with DeferSync
class DeferringAsyncWithAsyncPersistActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringAsyncWithAsyncPersistActor(name)
with LevelDbRuntimePluginConfig
class DeferringSyncWithAsyncPersistActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringSyncWithAsyncPersistActor(name)
with LevelDbRuntimePluginConfig
class DeferringAsyncWithAsyncPersistActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringAsyncWithAsyncPersistActor(name)
with InmemRuntimePluginConfig
@ -638,16 +534,6 @@ object PersistentActorSpec {
class DeferringSyncMixedCallsPPADDPADPersistActor(name: String)
extends DeferringMixedCallsPPADDPADPersistActor(name)
with DeferSync
class DeferringAsyncMixedCallsPPADDPADPersistActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends DeferringAsyncMixedCallsPPADDPADPersistActor(name)
with LevelDbRuntimePluginConfig
class DeferringSyncMixedCallsPPADDPADPersistActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends DeferringSyncMixedCallsPPADDPADPersistActor(name)
with LevelDbRuntimePluginConfig
class DeferringAsyncMixedCallsPPADDPADPersistActorWithInmemRuntimePluginConfig(
name: String,
val providedConfig: Config)
@ -675,16 +561,6 @@ object PersistentActorSpec {
class DeferringSyncWithNoPersistCallsPersistActor(name: String)
extends DeferringWithNoPersistCallsPersistActor(name)
with DeferSync
class DeferringAsyncWithNoPersistCallsPersistActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends DeferringAsyncWithNoPersistCallsPersistActor(name)
with LevelDbRuntimePluginConfig
class DeferringSyncWithNoPersistCallsPersistActorWithLevelDbRuntimePluginConfig(
name: String,
val providedConfig: Config)
extends DeferringSyncWithNoPersistCallsPersistActor(name)
with LevelDbRuntimePluginConfig
class DeferringAsyncWithNoPersistCallsPersistActorWithInmemRuntimePluginConfig(
name: String,
val providedConfig: Config)
@ -709,12 +585,6 @@ object PersistentActorSpec {
}
class DeferringAsyncActor(name: String) extends DeferringActor(name) with DeferAsync
class DeferringSyncActor(name: String) extends DeferringActor(name) with DeferSync
class DeferringAsyncActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringAsyncActor(name)
with LevelDbRuntimePluginConfig
class DeferringSyncActorWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringSyncActor(name)
with LevelDbRuntimePluginConfig
class DeferringAsyncActorWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends DeferringAsyncActor(name)
with InmemRuntimePluginConfig
@ -735,9 +605,6 @@ object PersistentActorSpec {
sender() ! s
}
}
class StressOrderingWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends StressOrdering(name)
with LevelDbRuntimePluginConfig
class StressOrderingWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends StressOrdering(name)
with InmemRuntimePluginConfig
@ -763,9 +630,6 @@ object PersistentActorSpec {
}
}
class RecoverMessageCausedRestartWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends RecoverMessageCausedRestart(name)
with LevelDbRuntimePluginConfig
class RecoverMessageCausedRestartWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends RecoverMessageCausedRestart(name)
with InmemRuntimePluginConfig
@ -788,12 +652,6 @@ object PersistentActorSpec {
}
}
}
class MultipleAndNestedPersistsWithLevelDbRuntimePluginConfig(
name: String,
probe: ActorRef,
val providedConfig: Config)
extends MultipleAndNestedPersists(name, probe)
with LevelDbRuntimePluginConfig
class MultipleAndNestedPersistsWithInmemRuntimePluginConfig(name: String, probe: ActorRef, val providedConfig: Config)
extends MultipleAndNestedPersists(name, probe)
with InmemRuntimePluginConfig
@ -816,12 +674,6 @@ object PersistentActorSpec {
}
}
}
class MultipleAndNestedPersistAsyncsWithLevelDbRuntimePluginConfig(
name: String,
probe: ActorRef,
val providedConfig: Config)
extends MultipleAndNestedPersistAsyncs(name, probe)
with LevelDbRuntimePluginConfig
class MultipleAndNestedPersistAsyncsWithInmemRuntimePluginConfig(
name: String,
probe: ActorRef,
@ -850,13 +702,6 @@ object PersistentActorSpec {
persistAsync(s + "-" + 1)(weMustGoDeeper)
}
}
class DeeplyNestedPersistAsyncsWithLevelDbRuntimePluginConfig(
name: String,
maxDepth: Int,
probe: ActorRef,
val providedConfig: Config)
extends DeeplyNestedPersistAsyncs(name, maxDepth, probe)
with LevelDbRuntimePluginConfig
class DeeplyNestedPersistAsyncsWithInmemRuntimePluginConfig(
name: String,
maxDepth: Int,
@ -883,12 +728,6 @@ object PersistentActorSpec {
}
}
}
class NestedPersistNormalAndAsyncsWithLevelDbRuntimePluginConfig(
name: String,
probe: ActorRef,
val providedConfig: Config)
extends NestedPersistNormalAndAsyncs(name, probe)
with LevelDbRuntimePluginConfig
class NestedPersistNormalAndAsyncsWithInmemRuntimePluginConfig(
name: String,
probe: ActorRef,
@ -914,12 +753,6 @@ object PersistentActorSpec {
}
}
}
class NestedPersistAsyncsAndNormalWithLevelDbRuntimePluginConfig(
name: String,
probe: ActorRef,
val providedConfig: Config)
extends NestedPersistAsyncsAndNormal(name, probe)
with LevelDbRuntimePluginConfig
class NestedPersistAsyncsAndNormalWithInmemRuntimePluginConfig(
name: String,
probe: ActorRef,
@ -942,12 +775,6 @@ object PersistentActorSpec {
}
}
}
class NestedPersistInAsyncEnforcesStashingWithLevelDbRuntimePluginConfig(
name: String,
probe: ActorRef,
val providedConfig: Config)
extends NestedPersistInAsyncEnforcesStashing(name, probe)
with LevelDbRuntimePluginConfig
class NestedPersistInAsyncEnforcesStashingWithInmemRuntimePluginConfig(
name: String,
probe: ActorRef,
@ -976,13 +803,6 @@ object PersistentActorSpec {
persist(s + "-" + 1)(weMustGoDeeper)
}
}
class DeeplyNestedPersistsWithLevelDbRuntimePluginConfig(
name: String,
maxDepth: Int,
probe: ActorRef,
val providedConfig: Config)
extends DeeplyNestedPersists(name, maxDepth, probe)
with LevelDbRuntimePluginConfig
class DeeplyNestedPersistsWithInmemRuntimePluginConfig(
name: String,
maxDepth: Int,
@ -1029,9 +849,6 @@ object PersistentActorSpec {
}
}
class StackableTestPersistentActorWithLevelDbRuntimePluginConfig(probe: ActorRef, val providedConfig: Config)
extends StackableTestPersistentActor(probe)
with LevelDbRuntimePluginConfig
class StackableTestPersistentActorWithInmemRuntimePluginConfig(probe: ActorRef, val providedConfig: Config)
extends StackableTestPersistentActor(probe)
with InmemRuntimePluginConfig
@ -1117,9 +934,6 @@ object PersistentActorSpec {
case Cmd(d) => persist(Evt(d))(updateState)
}
}
class PersistInRecoveryWithLevelDbRuntimePluginConfig(name: String, val providedConfig: Config)
extends PersistInRecovery(name)
with LevelDbRuntimePluginConfig
class PersistInRecoveryWithInmemRuntimePluginConfig(name: String, val providedConfig: Config)
extends PersistInRecovery(name)
with InmemRuntimePluginConfig
@ -1825,212 +1639,8 @@ abstract class PersistentActorSpec(config: Config) extends PersistenceSpec(confi
}
class LeveldbPersistentActorSpec
extends PersistentActorSpec(PersistenceSpec.config("leveldb", "LeveldbPersistentActorSpec"))
class InmemPersistentActorSpec extends PersistentActorSpec(PersistenceSpec.config("inmem", "InmemPersistentActorSpec"))
/**
* Same test suite as [[LeveldbPersistentActorSpec]], the only difference is that all persistent actors are using the
* provided [[Config]] instead of the [[Config]] coming from the [[ActorSystem]].
*/
class LeveldbPersistentActorWithRuntimePluginConfigSpec
extends PersistentActorSpec(PersistenceSpec.config("leveldb", "LeveldbPersistentActorWithRuntimePluginConfigSpec")) {
val providedActorConfig: Config = {
ConfigFactory
.parseString(s"""
| custom.persistence.journal.leveldb.dir = target/journal-LeveldbPersistentActorWithRuntimePluginConfigSpec
| custom.persistence.snapshot-store.local.dir = target/snapshots-LeveldbPersistentActorWithRuntimePluginConfigSpec/
""".stripMargin)
.withValue(
s"custom.persistence.journal.leveldb",
system.settings.config.getValue(s"akka.persistence.journal.leveldb"))
.withValue(
"custom.persistence.snapshot-store.local",
system.settings.config.getValue("akka.persistence.snapshot-store.local"))
}
override protected def behavior1PersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[Behavior1PersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def behavior2PersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[Behavior2PersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def behavior3PersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[Behavior3PersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def changeBehaviorInFirstEventHandlerPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[
ChangeBehaviorInFirstEventHandlerPersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def changeBehaviorInLastEventHandlerPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[
ChangeBehaviorInLastEventHandlerPersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def changeBehaviorInCommandHandlerFirstPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[
ChangeBehaviorInCommandHandlerFirstPersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def changeBehaviorInCommandHandlerLastPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[
ChangeBehaviorInCommandHandlerLastPersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def snapshottingPersistentActor: ActorRef =
system.actorOf(
Props(classOf[SnapshottingPersistentActorWithLevelDbRuntimePluginConfig], name, testActor, providedActorConfig))
override protected def snapshottingBecomingPersistentActor: ActorRef =
system.actorOf(
Props(
classOf[SnapshottingBecomingPersistentActorWithLevelDbRuntimePluginConfig],
name,
testActor,
providedActorConfig))
override protected def replyInEventHandlerPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[ReplyInEventHandlerPersistentActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def primitiveEventPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[PrimitiveEventPersistentActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def asyncPersistPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[AsyncPersistPersistentActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def asyncPersistThreeTimesPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[AsyncPersistThreeTimesPersistentActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def asyncPersistSameEventTwicePersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[AsyncPersistSameEventTwicePersistentActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def persistAllNilPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[PersistAllNilPersistentActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def asyncPersistAndPersistMixedSyncAsyncSyncPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[
AsyncPersistAndPersistMixedSyncAsyncSyncPersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def asyncPersistAndPersistMixedSyncAsyncPersistentActor: ActorRef =
namedPersistentActorWithProvidedConfig[
AsyncPersistAndPersistMixedSyncAsyncPersistentActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def asyncPersistHandlerCorrelationCheck: ActorRef =
namedPersistentActorWithProvidedConfig[AsyncPersistHandlerCorrelationCheckWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def handleRecoveryFinishedEventPersistentActor: ActorRef =
system.actorOf(
Props(
classOf[HandleRecoveryFinishedEventPersistentActorWithLevelDbRuntimePluginConfig],
name,
testActor,
providedActorConfig))
override protected def stressOrdering: ActorRef =
namedPersistentActorWithProvidedConfig[StressOrderingWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def stackableTestPersistentActor: ActorRef =
system.actorOf(
Props(classOf[StackableTestPersistentActorWithLevelDbRuntimePluginConfig], testActor, providedActorConfig))
override protected def multipleAndNestedPersists: ActorRef =
system.actorOf(
Props(classOf[MultipleAndNestedPersistsWithLevelDbRuntimePluginConfig], name, testActor, providedActorConfig))
override protected def multipleAndNestedPersistAsyncs: ActorRef =
system.actorOf(
Props(
classOf[MultipleAndNestedPersistAsyncsWithLevelDbRuntimePluginConfig],
name,
testActor,
providedActorConfig))
override protected def deeplyNestedPersists(nestedPersists: Int): ActorRef =
system.actorOf(
Props(
classOf[DeeplyNestedPersistsWithLevelDbRuntimePluginConfig],
name,
nestedPersists,
testActor,
providedActorConfig))
override protected def deeplyNestedPersistAsyncs(nestedPersistAsyncs: Int): ActorRef =
system.actorOf(
Props(
classOf[DeeplyNestedPersistAsyncsWithLevelDbRuntimePluginConfig],
name,
nestedPersistAsyncs,
testActor,
providedActorConfig))
override protected def nestedPersistNormalAndAsyncs: ActorRef =
system.actorOf(
Props(classOf[NestedPersistNormalAndAsyncsWithLevelDbRuntimePluginConfig], name, testActor, providedActorConfig))
override protected def nestedPersistAsyncsAndNormal: ActorRef =
system.actorOf(
Props(classOf[NestedPersistAsyncsAndNormalWithLevelDbRuntimePluginConfig], name, testActor, providedActorConfig))
override protected def nestedPersistInAsyncEnforcesStashing: ActorRef =
system.actorOf(
Props(
classOf[NestedPersistInAsyncEnforcesStashingWithLevelDbRuntimePluginConfig],
name,
testActor,
providedActorConfig))
override protected def persistInRecovery: ActorRef =
namedPersistentActorWithProvidedConfig[PersistInRecoveryWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def recoverMessageCausedRestart: ActorRef =
namedPersistentActorWithProvidedConfig[RecoverMessageCausedRestartWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringAsyncWithPersistActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringAsyncWithPersistActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringSyncWithPersistActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringSyncWithPersistActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringAsyncWithAsyncPersistActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringAsyncWithAsyncPersistActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringSyncWithAsyncPersistActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringSyncWithAsyncPersistActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringAsyncMixedCallsPPADDPADPersistActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringAsyncMixedCallsPPADDPADPersistActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringSyncMixedCallsPPADDPADPersistActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringSyncMixedCallsPPADDPADPersistActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringAsyncWithNoPersistCallsPersistActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringAsyncWithNoPersistCallsPersistActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringSyncWithNoPersistCallsPersistActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringSyncWithNoPersistCallsPersistActorWithLevelDbRuntimePluginConfig](
providedActorConfig)
override protected def deferringAsyncActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringAsyncActorWithLevelDbRuntimePluginConfig](providedActorConfig)
override protected def deferringSyncActor: ActorRef =
namedPersistentActorWithProvidedConfig[DeferringSyncActorWithLevelDbRuntimePluginConfig](providedActorConfig)
}
/**
* Same test suite as [[InmemPersistentActorSpec]], the only difference is that all persistent actors are using the
* provided [[Config]] instead of the [[Config]] coming from the [[ActorSystem]].

View file

@ -309,7 +309,5 @@ class SteppingInMemPersistentActorStashingSpec
}
class LeveldbPersistentActorStashingSpec
extends PersistentActorStashingSpec(PersistenceSpec.config("leveldb", "LeveldbPersistentActorStashingSpec"))
class InmemPersistentActorStashingSpec
extends PersistentActorStashingSpec(PersistenceSpec.config("inmem", "InmemPersistentActorStashingSpec"))

View file

@ -31,7 +31,7 @@ object SnapshotDirectoryFailureSpec {
class SnapshotDirectoryFailureSpec
extends AkkaSpec(
PersistenceSpec.config(
"leveldb",
"inmem",
"SnapshotDirectoryFailureSpec",
extraConfig = Some(s"""
akka.persistence.snapshot-store.local.dir = "${SnapshotDirectoryFailureSpec.inUseSnapshotPath}"

View file

@ -100,7 +100,7 @@ object SnapshotFailureRobustnessSpec {
class SnapshotFailureRobustnessSpec
extends PersistenceSpec(
PersistenceSpec.config(
"leveldb",
"inmem",
"SnapshotFailureRobustnessSpec",
serialization = "off",
extraConfig = Some(s"""

View file

@ -63,7 +63,7 @@ object SnapshotRecoveryWithEmptyJournalSpec {
class SnapshotRecoveryWithEmptyJournalSpec
extends PersistenceSpec(
PersistenceSpec.config(
"leveldb",
"inmem",
"SnapshotRecoveryWithEmptyJournalSpec",
extraConfig = Some(s"""
akka.persistence.snapshot-store.local.dir = "${SnapshotRecoveryWithEmptyJournalSpec.survivingSnapshotPath}"

View file

@ -68,7 +68,7 @@ object SnapshotSerializationSpec {
class SnapshotSerializationSpec
extends PersistenceSpec(
PersistenceSpec.config(
"leveldb",
"inmem",
"SnapshotSerializationSpec",
serialization = "off",
extraConfig = Some("""

View file

@ -83,7 +83,7 @@ object SnapshotSpec {
}
}
class SnapshotSpec extends PersistenceSpec(PersistenceSpec.config("leveldb", "SnapshotSpec")) with ImplicitSender {
class SnapshotSpec extends PersistenceSpec(PersistenceSpec.config("inmem", "SnapshotSpec")) with ImplicitSender {
import SnapshotProtocol._
import SnapshotSpec._

View file

@ -369,7 +369,7 @@ abstract class PersistentFSMSpec(config: Config) extends PersistenceSpec(config)
ConfigFactory.parseString("""
akka.persistence.fsm.enable-snapshot-after = on
akka.persistence.fsm.snapshot-after = 3
""").withFallback(PersistenceSpec.config("leveldb", "PersistentFSMSpec2")))
""").withFallback(PersistenceSpec.config("inmem", "PersistentFSMSpec2")))
try {
val probe = TestProbe()
@ -390,10 +390,7 @@ abstract class PersistentFSMSpec(config: Config) extends PersistenceSpec(config)
} finally {
val storageLocations =
List(
"akka.persistence.journal.leveldb.dir",
"akka.persistence.journal.leveldb-shared.store.dir",
"akka.persistence.snapshot-store.local.dir").map(s => new File(sys2.settings.config.getString(s)))
List("akka.persistence.snapshot-store.local.dir").map(s => new File(sys2.settings.config.getString(s)))
shutdown(sys2)
storageLocations.foreach(FileUtils.deleteDirectory)
}
@ -653,5 +650,4 @@ object PersistentFSMSpec {
}
class LeveldbPersistentFSMSpec extends PersistentFSMSpec(PersistenceSpec.config("leveldb", "PersistentFSMSpec"))
class InmemPersistentFSMSpec extends PersistentFSMSpec(PersistenceSpec.config("inmem", "PersistentFSMSpec"))