PersistencePluginProxy

* Rename to PersistencePluginProxy.
* Watch target journal.
* Create PersistencePluginProxyExtension to enable eager initialization of
  the persistence plugin.
* Add initialization via configuration.
* Add tests and documentation.
* Clearer log messages.
This commit is contained in:
Tal Pressman 2015-11-17 16:50:54 +02:00
parent d39b368cac
commit f610952ae7
9 changed files with 338 additions and 40 deletions

View file

@ -892,6 +892,10 @@ backup node.
A shared LevelDB instance is a single point of failure and should therefore only be used for testing
purposes. Highly-available, replicated journal are available as `Community plugins`_.
.. note::
This plugin has been supplanted by :ref:`Persistence Plugin Proxy<persistence-plugin-proxy-lambda>`.
A shared LevelDB instance is started by instantiating the ``SharedLeveldbStore`` actor.
.. includecode:: code/docs/persistence/PersistencePluginDocTest.java#shared-store-creation
@ -932,6 +936,43 @@ directory. This can be changed by configuration where the specified path can be
Note that it is not mandatory to specify a snapshot store plugin. If you don't use snapshots
you don't have to configure it.
.. _persistence-plugin-proxy-lambda:
Persistence Plugin Proxy
------------------------
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 therefor 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
purposes. Highly-available, replicated persistence plugins are available as `Community plugins`_.
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
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
``PersistencePluginProxy.setTargetLocation`` method.
.. note::
Akka starts extensions lazily when they are required, and this includes the proxy. This means that in order for the
proxy to work, the persistence plugin on the target node must be instantiated. This can be done by instantiating the
``PersistencePluginProxyExtension`` :ref:`extension<extending-akka-java>`, or by calling the ``PersistencePluginProxy.start`` method.
.. note::
The proxied persistence plugin can (and should) be configured using its original configuration keys.
.. _custom-serialization-lambda:
Custom serialization
====================

View file

@ -882,6 +882,10 @@ backup node.
A shared LevelDB instance is a single point of failure and should therefore only be used for testing
purposes. Highly-available, replicated journal are available as `Community plugins`_.
.. note::
This plugin has been supplanted by :ref:`Persistence Plugin Proxy<persistence-plugin-proxy-java>`.
A shared LevelDB instance is started by instantiating the ``SharedLeveldbStore`` actor.
.. includecode:: code/docs/persistence/PersistencePluginDocTest.java#shared-store-creation
@ -922,6 +926,43 @@ directory. This can be changed by configuration where the specified path can be
Note that it is not mandatory to specify a snapshot store plugin. If you don't use snapshots
you don't have to configure it.
.. _persistence-plugin-proxy-java:
Persistence Plugin Proxy
------------------------
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 therefor 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
purposes. Highly-available, replicated persistence plugins are available as `Community plugins`_.
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
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
``PersistencePluginProxy.setTargetLocation`` method.
.. note::
Akka starts extensions lazily when they are required, and this includes the proxy. This means that in order for the
proxy to work, the persistence plugin on the target node must be instantiated. This can be done by instantiating the
``PersistencePluginProxyExtension`` :ref:`extension<extending-akka-java>`, or by calling the ``PersistencePluginProxy.start`` method.
.. note::
The proxied persistence plugin can (and should) be configured using its original configuration keys.
.. _custom-serialization-java:
Custom serialization
====================

View file

@ -0,0 +1,16 @@
.. _migration-guide-2.4.x-2.5.x:
##############################
Migration Guide 2.4.x to 2.5.x
##############################
Akka Persistence
================
Persistence Plugin Proxy
------------------------
A new :ref:`persistence plugin proxy<persistence-plugin-proxy>` was added, that allows sharing of an otherwise
non-sharable journal or snapshot store. The proxy is available by setting ``akka.persistence.journal.plugin`` or
``akka.persistence.snapshot-store.plugin`` to ``akka.persistence.journal.proxy`` or ``akka.persistence.snapshot-store.proxy``,
respectively. The proxy supplants the :ref:`Shared LevelDB journal<shared-leveldb-journal>`.

View file

@ -13,4 +13,4 @@ Migration Guides
migration-guide-persistence-experimental-2.3.x-2.4.x
migration-guide-eventsourced-2.3.x
migration-guide-2.3.x-2.4.x
migration-guide-2.4.x-2.5.x

View file

@ -931,6 +931,10 @@ backup node.
A shared LevelDB instance is a single point of failure and should therefore only be used for testing
purposes. Highly-available, replicated journal are available as `Community plugins`_.
.. note::
This plugin has been supplanted by :ref:`Persistence Plugin Proxy<persistence-plugin-proxy>`.
A shared LevelDB instance is started by instantiating the ``SharedLeveldbStore`` actor.
.. includecode:: code/docs/persistence/PersistencePluginDocSpec.scala#shared-store-creation
@ -953,6 +957,7 @@ done by calling the ``SharedLeveldbJournal.setStore`` method with the actor refe
Internal journal commands (sent by persistent actors) are buffered until injection completes. Injection is idempotent
i.e. only the first injection is used.
.. _local-snapshot-store:
Local snapshot store
@ -971,6 +976,42 @@ directory. This can be changed by configuration where the specified path can be
Note that it is not mandatory to specify a snapshot store plugin. If you don't use snapshots
you don't have to configure it.
.. _persistence-plugin-proxy:
Persistence Plugin Proxy
------------------------
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 therefor 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
purposes. Highly-available, replicated persistence plugins are available as `Community plugins`_.
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
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
``PersistencePluginProxy.setTargetLocation`` method.
.. note::
Akka starts extensions lazily when they are required, and this includes the proxy. This means that in order for the
proxy to work, the persistence plugin on the target node must be instantiated. This can be done by instantiating the
``PersistencePluginProxyExtension`` :ref:`extension<extending-akka-scala>`, or by calling the ``PersistencePluginProxy.start`` method.
.. note::
The proxied persistence plugin can (and should) be configured using its original configuration keys.
.. _custom-serialization:
Custom serialization

View file

@ -245,7 +245,7 @@ akka.persistence.journal.leveldb-shared {
akka.persistence.journal.proxy {
# Class name of the plugin.
class = "akka.persistence.journal.JournalProxy"
class = "akka.persistence.journal.PersistencePluginProxy"
# Dispatcher for the plugin actor.
plugin-dispatcher = "akka.actor.default-dispatcher"
# Set this to on in the configuration of the ActorSystem
@ -253,13 +253,15 @@ akka.persistence.journal.proxy {
start-target-journal = off
# The journal plugin config path to use for the target journal
target-journal-plugin = ""
# The address of the proxy to connect to from other nodes. Optional setting.
target-journal-address = ""
# Initialization timeout of target lookup
init-timeout = 10s
}
akka.persistence.snapshot-store.proxy {
# Class name of the plugin.
class = "akka.persistence.journal.JournalProxy"
class = "akka.persistence.journal.PersistencePluginProxy"
# Dispatcher for the plugin actor.
plugin-dispatcher = "akka.actor.default-dispatcher"
# Set this to on in the configuration of the ActorSystem
@ -267,6 +269,8 @@ akka.persistence.snapshot-store.proxy {
start-target-snapshot-store = off
# The journal plugin config path to use for the target snapshot-store
target-snapshot-store-plugin = ""
# The address of the proxy to connect to from other nodes. Optional setting.
target-snapshot-store-address = ""
# Initialization timeout of target lookup
init-timeout = 10s
}

View file

@ -3,34 +3,17 @@
*/
package akka.persistence.journal
import akka.util.Helpers.Requiring
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.Stash
import scala.concurrent.duration.FiniteDuration
import akka.actor.ActorRef
import akka.persistence.JournalProtocol
import akka.actor.ActorSystem
import akka.persistence.Persistence
import scala.util.control.NoStackTrace
import java.net.URISyntaxException
import java.util.concurrent.TimeoutException
import akka.persistence.AtomicWrite
import akka.persistence.NonPersistentRepr
import akka.persistence.DeleteMessagesFailure
import akka.actor.ActorLogging
import com.typesafe.config.Config
import akka.actor.Address
import akka.actor.ActorIdentity
import akka.actor.RootActorPath
import akka.actor.Identify
import akka.actor.ReceiveTimeout
import akka.actor.ExtendedActorSystem
import akka.persistence.SaveSnapshotFailure
import akka.persistence.DeleteSnapshotFailure
import akka.persistence.DeleteSnapshotsFailure
import akka.persistence.SnapshotProtocol
object JournalProxy {
import akka.actor._
import akka.persistence.{ AtomicWrite, DeleteMessagesFailure, DeleteSnapshotFailure, DeleteSnapshotsFailure, JournalProtocol, NonPersistentRepr, Persistence, SaveSnapshotFailure, SnapshotProtocol }
import akka.util.Helpers.Requiring
import com.typesafe.config.Config
import scala.concurrent.duration._
object PersistencePluginProxy {
final case class TargetLocation(address: Address)
private case object InitTimeout
@ -40,6 +23,13 @@ object JournalProxy {
Persistence(system).snapshotStoreFor(null) ! TargetLocation(address)
}
def start(system: ActorSystem): Unit = {
Persistence(system).journalFor(null)
if (system.settings.config.getString("akka.persistence.snapshot-store.plugin") != "")
Persistence(system).snapshotStoreFor(null)
}
private sealed trait PluginType {
def qualifier: String
}
@ -51,9 +41,23 @@ object JournalProxy {
}
}
// FIXME document me
final class JournalProxy(config: Config) extends Actor with Stash with ActorLogging {
import JournalProxy._
/**
* PersistencePluginProxyExtensionImpl is an `Extension` that enables initialization of the `PersistencePluginProxy`
* via configuration, without requiring any code changes or the creation of any actors.
* @param system The actor system to initialize the extension for
*/
class PersistencePluginProxyExtensionImpl(system: ActorSystem) extends Extension {
PersistencePluginProxy.start(system)
}
object PersistencePluginProxyExtension extends ExtensionId[PersistencePluginProxyExtensionImpl] with ExtensionIdProvider {
override def createExtension(system: ExtendedActorSystem): PersistencePluginProxyExtensionImpl = new PersistencePluginProxyExtensionImpl(system)
override def lookup(): ExtensionId[_ <: Extension] = PersistencePluginProxyExtension
override def get(system: ActorSystem): PersistencePluginProxyExtensionImpl = super.get(system)
}
final class PersistencePluginProxy(config: Config) extends Actor with Stash with ActorLogging {
import PersistencePluginProxy._
import JournalProtocol._
import SnapshotProtocol._
@ -65,7 +69,7 @@ final class JournalProxy(config: Config) extends Actor with Stash with ActorLogg
throw new IllegalArgumentException("Unknown plugin type: " + other)
}
private val timeout: FiniteDuration = config.getDuration("init-timeout", MILLISECONDS).millis
private val initTimeout: FiniteDuration = config.getDuration("init-timeout", MILLISECONDS).millis
private val targetPluginId: String = {
val key = s"target-${pluginType.qualifier}-plugin"
config.getString(key).requiring(_ != "", s"$pluginId.$key must be defined")
@ -84,7 +88,18 @@ final class JournalProxy(config: Config) extends Actor with Stash with ActorLogg
}
context.become(active(target, targetAtThisNode = true))
} else {
context.system.scheduler.scheduleOnce(timeout, self, InitTimeout)(context.dispatcher)
val targetAddressKey = s"target-${pluginType.qualifier}-address"
val targetAddress = config.getString(targetAddressKey)
if (targetAddress != "") {
try {
log.info("Setting target {} address to {}", pluginType.qualifier, targetAddress)
PersistencePluginProxy.setTargetLocation(context.system, AddressFromURIString(targetAddress))
} catch {
case _: URISyntaxException log.warning("Invalid URL provided for target {} address: {}", pluginType.qualifier, targetAddress)
}
}
context.system.scheduler.scheduleOnce(initTimeout, self, InitTimeout)(context.dispatcher)
}
}
@ -92,7 +107,7 @@ final class JournalProxy(config: Config) extends Actor with Stash with ActorLogg
context.system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
private def timeoutException() = new TimeoutException(s"Target ${pluginType.qualifier} not initialized. " +
"Use `JournalProxy.setTargetLocation`")
s"Use `PersistencePluginProxy.setTargetLocation` or set `target-${pluginType.qualifier}-address`")
def receive = init
@ -101,9 +116,10 @@ final class JournalProxy(config: Config) extends Actor with Stash with ActorLogg
context.setReceiveTimeout(1.second) // for retries
context.become(identifying(address))
case InitTimeout
log.info("Initialization timeout, Use `JournalProxy.setTargetLocation`")
log.info("Initialization timed-out (after {}), Use `PersistencePluginProxy.setTargetLocation` or set `target-{}-address`", initTimeout, pluginType.qualifier)
context.become(initTimedOut)
unstashAll() // will trigger appropriate failures
case Terminated(_)
case msg
stash()
}
@ -124,9 +140,11 @@ final class JournalProxy(config: Config) extends Actor with Stash with ActorLogg
case ActorIdentity(`targetPluginId`, Some(target))
log.info("Found target {} at [{}]", pluginType.qualifier, address)
context.setReceiveTimeout(Duration.Undefined)
context.watch(target)
unstashAll()
context.become(active(target, address == selfAddress))
case _: ActorIdentity // will retry after ReceiveTimeout
case Terminated(_)
case ReceiveTimeout
sendIdentify(address)
}: Receive).orElse(init)
@ -135,9 +153,13 @@ final class JournalProxy(config: Config) extends Actor with Stash with ActorLogg
case TargetLocation(address)
if (targetAtThisNode && address != selfAddress)
becomeIdentifying(address)
case InitTimeout
case Terminated(`targetJournal`)
context.unwatch(targetJournal)
context.become(initTimedOut)
case Terminated(_)
case InitTimeout
case msg
targetJournal.forward(msg)
targetJournal forward msg
}
def initTimedOut: Receive = {
@ -173,9 +195,11 @@ final class JournalProxy(config: Config) extends Actor with Stash with ActorLogg
case TargetLocation(address)
becomeIdentifying(address)
case Terminated(_)
case other
val e = timeoutException()
log.error(e, "Failed JournalProxy request: {}", e.getMessage)
log.error(e, "Failed PersistencePluginProxy request: {}", e.getMessage)
}
}

View file

@ -0,0 +1,129 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.persistence.journal.leveldb
import akka.actor._
import akka.persistence._
import akka.persistence.journal.PersistencePluginProxy
import akka.testkit.{ TestProbe, AkkaSpec }
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
object PersistencePluginProxySpec {
lazy val config = ConfigFactory.parseString(
"""
akka {
actor {
provider = "akka.remote.RemoteActorRefProvider"
}
persistence {
journal {
plugin = "akka.persistence.journal.proxy"
proxy.target-journal-plugin = "akka.persistence.journal.inmem"
}
snapshot-store {
plugin = "akka.persistence.snapshot-store.proxy"
proxy.target-snapshot-store-plugin = "akka.persistence.snapshot-store.local"
local.dir = target/snapshots-PersistencePluginProxySpec
}
}
remote {
enabled-transports = ["akka.remote.netty.tcp"]
netty.tcp {
hostname = "127.0.0.1"
port = 0
}
}
loglevel = ERROR
log-dead-letters = 0
log-dead-letters-during-shutdown = off
test.single-expect-default = 10s
}
""")
lazy val startTargetConfig = ConfigFactory.parseString(
"""
|akka.extensions = ["akka.persistence.journal.PersistencePluginProxyExtension"]
|akka.persistence {
| journal.proxy.start-target-journal = on
| snapshot-store.proxy.start-target-snapshot-store = on
|}
""".stripMargin)
def targetAddressConfig(system: ActorSystem) = ConfigFactory.parseString(
s"""
|akka.persistence.journal.proxy.target-journal-address = "${system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress}"
|akka.persistence.snapshot-store.proxy.target-snapshot-store-address = "${system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress}"
""".stripMargin)
class ExamplePersistentActor(probe: ActorRef, name: String) extends NamedPersistentActor(name) {
override def receiveRecover = {
case RecoveryCompleted // ignore
case payload
probe ! payload
}
override def receiveCommand = {
case payload
persist(payload) { _
probe ! payload
}
}
}
class ExampleApp(probe: ActorRef) extends Actor {
val p = context.actorOf(Props(classOf[ExamplePersistentActor], probe, context.system.name))
def receive = {
case m p forward m
}
}
}
class PersistencePluginProxySpec extends AkkaSpec(PersistencePluginProxySpec.startTargetConfig withFallback PersistencePluginProxySpec.config) with Cleanup {
import PersistencePluginProxySpec._
val systemA = ActorSystem("SysA", config)
val systemB = ActorSystem("SysB", targetAddressConfig(system) withFallback PersistencePluginProxySpec.config)
override protected def afterTermination() {
shutdown(systemA)
shutdown(systemB)
super.afterTermination()
}
"A persistence proxy" can {
"be shared by multiple actor systems" in {
val address = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
val probeA = new TestProbe(systemA)
val probeB = new TestProbe(systemB)
PersistencePluginProxy.setTargetLocation(systemA, address)
val appA = systemA.actorOf(Props(classOf[ExampleApp], probeA.ref))
val appB = systemB.actorOf(Props(classOf[ExampleApp], probeB.ref))
appA ! "a1"
appB ! "b1"
probeA.expectMsg("a1")
probeB.expectMsg("b1")
val recoveredAppA = systemA.actorOf(Props(classOf[ExampleApp], probeA.ref))
val recoveredAppB = systemB.actorOf(Props(classOf[ExampleApp], probeB.ref))
recoveredAppA ! "a2"
recoveredAppB ! "b2"
probeA.expectMsg("a1")
probeA.expectMsg("a2")
probeB.expectMsg("b1")
probeB.expectMsg("b2")
}
}
}

View file

@ -565,12 +565,14 @@ object MiMa extends AutoPlugin {
Map(
"2.3.11" -> Seq(
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.ActorCell.clearActorFields"), // #17805, incomatibility with 2.4.x fixed in 2.3.12
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.ActorCell.clearActorFields"), // #17805, incompatibility with 2.4.x fixed in 2.3.12
ProblemFilters.exclude[MissingMethodProblem]("akka.japi.Pair.toString") // reported on PR validation machine which uses Java 1.8.0_45
),
"2.3.14" -> bcIssuesBetween23and24,
"2.4.0" -> Seq(
FilterAnyProblem("akka.remote.transport.ProtocolStateActor"),
FilterAnyProblem("akka.persistence.journal.inmem.InmemJournal"),
FilterAnyProblem("akka.persistence.journal.inmem.InmemStore"),
//#18353 Changes to methods and fields private to remoting actors
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.EndpointManager.retryGateEnabled"),