!per persistAsync

Breaks binary compatibility because adding new methods to Eventsourced
trait. Since akka-persistence is experimental this is ok, yet
source-level compatibility has been perserved thankfuly :-)

Deprecates:
* Rename of EventsourcedProcessor -> PersistentActor
* Processor -> suggest using PersistentActor
* Migration guide for akka-persistence is separate, as wel'll deprecate in minor versions (its experimental)
* Persistent as well as ConfirmablePersistent - since Processor, their
  main user will be removed soon.

Other changes:
* persistAsync works as expected when mixed with persist
* A counter must be kept for pending stashing invocations
* Uses only 1 shared list buffer for persit / persistAsync
* Includes small benchmark
* Docs also include info about not using Persistent() wrapper
* uses java LinkedList, for best performance of append / head on
  persistInvocations; the get(0) is safe, because these msgs only
  come in response to persistInvocations
* Renamed internal *MessagesSuccess/Failure messages because we kept
  small mistakes seeing the class "with s" and "without s" as the same
* Updated everything that refered to EventsourcedProcessor to
  PersistentActor, including samples

Refs #15227

Conflicts:
	akka-docs/rst/project/migration-guides.rst
	akka-persistence/src/main/scala/akka/persistence/JournalProtocol.scala
	akka-persistence/src/main/scala/akka/persistence/Persistent.scala
	akka-persistence/src/test/scala/akka/persistence/PersistentActorSpec.scala
	project/AkkaBuild.scala
This commit is contained in:
Konrad 'ktoso' Malawski 2014-05-21 01:35:21 +02:00
parent 5f3d6029b1
commit d51b79c95a
32 changed files with 907 additions and 134 deletions

View file

@ -0,0 +1,144 @@
/**
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.persistence
import org.openjdk.jmh.annotations._
import org.openjdk.jmh._
import com.typesafe.config.ConfigFactory
import akka.actor._
import akka.testkit.TestProbe
import java.io.File
import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations.Scope
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
class PersistentActorThroughputBenchmark {
val config = PersistenceSpec.config("leveldb", "benchmark")
lazy 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)))
var system: ActorSystem = _
var probe: TestProbe = _
var actor: ActorRef = _
var persist1EventProcessor: ActorRef = _
var persist1CommandProcessor: ActorRef = _
var persistAsync1EventProcessor: ActorRef = _
var persistAsync1QuickReplyEventProcessor: ActorRef = _
val data10k = (1 to 10000).toArray
@Setup
def setup() {
system = ActorSystem("test", config)
probe = TestProbe()(system)
storageLocations.foreach(FileUtils.deleteDirectory)
actor = system.actorOf(Props(classOf[BaselineActor], data10k.last), "a-1")
persist1CommandProcessor = system.actorOf(Props(classOf[Persist1EventPersistentActor], data10k.last), "p-1")
persist1EventProcessor = system.actorOf(Props(classOf[Persist1EventPersistentActor], data10k.last), "ep-1")
persistAsync1EventProcessor = system.actorOf(Props(classOf[PersistAsync1EventPersistentActor], data10k.last), "epa-1")
persistAsync1QuickReplyEventProcessor = system.actorOf(Props(classOf[PersistAsync1EventQuickReplyPersistentActor], data10k.last), "epa-2")
}
@TearDown
def shutdown() {
system.shutdown()
system.awaitTermination()
storageLocations.foreach(FileUtils.deleteDirectory)
}
@GenerateMicroBenchmark
@OperationsPerInvocation(10000)
def tell_normalActor_reply_baseline() {
for (i <- data10k) actor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
@GenerateMicroBenchmark
@OperationsPerInvocation(10000)
def tell_persist_reply() {
for (i <- data10k) persist1EventProcessor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
@GenerateMicroBenchmark
@OperationsPerInvocation(10000)
def tell_commandPersist_reply() {
for (i <- data10k) persist1CommandProcessor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
@GenerateMicroBenchmark
@OperationsPerInvocation(10000)
def tell_persistAsync_reply() {
for (i <- data10k) persistAsync1EventProcessor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
@GenerateMicroBenchmark
@OperationsPerInvocation(10000)
def tell_persistAsync_replyRightOnCommandReceive() {
for (i <- data10k) persistAsync1QuickReplyEventProcessor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
}
final case class Evt(i: Int)
class Persist1EventPersistentActor(respondAfter: Int) extends PersistentActor {
override def receiveCommand = {
case n: Int => persist(Evt(n)) { e => if (e.i == respondAfter) sender() ! e }
}
override def receiveRecover = {
case _ => // do nothing
}
}
class Persist1CommandProcessor(respondAfter: Int) extends Processor {
override def receive = {
case n: Int => if (n == respondAfter) sender() ! Evt(n)
}
}
class PersistAsync1EventPersistentActor(respondAfter: Int) extends PersistentActor {
override def receiveCommand = {
case n: Int =>
persistAsync(Evt(n)) { e => if (e.i == respondAfter) sender() ! e }
}
override def receiveRecover = {
case _ => // do nothing
}
}
class PersistAsync1EventQuickReplyPersistentActor(respondAfter: Int) extends PersistentActor {
override def receiveCommand = {
case n: Int =>
val e = Evt(n)
if (n == respondAfter) sender() ! e
persistAsync(e)(identity)
}
override def receiveRecover = {
case _ => // do nothing
}
}
/** only as a "the best we could possibly get" baseline, does not persist anything */
class BaselineActor(respondAfter: Int) extends Actor {
override def receive = {
case n: Int => if (n == respondAfter) sender() ! Evt(n)
}
}

View file

@ -29,7 +29,7 @@ This is how an entry actor may look like:
.. includecode:: @contribSrc@/src/test/java/akka/contrib/pattern/ClusterShardingTest.java#counter-actor
The above actor uses event sourcing and the support provided in ``UntypedEventsourcedProcessor`` to store its state.
The above actor uses event sourcing and the support provided in ``UntypedPersistentActor`` to store its state.
It does not have to be a processor, but in case of failure or migration of entries between nodes it must be able to recover
its state if it is valuable.
@ -73,7 +73,7 @@ This is how an entry actor may look like:
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterShardingSpec.scala#counter-actor
The above actor uses event sourcing and the support provided in ``EventsourcedProcessor`` to store its state.
The above actor uses event sourcing and the support provided in ``PersistentActor`` to store its state.
It does not have to be a processor, but in case of failure or migration of entries between nodes it must be able to recover
its state if it is valuable.

View file

@ -32,7 +32,7 @@ import akka.cluster.ClusterEvent.MemberUp
import akka.cluster.Member
import akka.cluster.MemberStatus
import akka.pattern.ask
import akka.persistence.EventsourcedProcessor
import akka.persistence.PersistentActor
import akka.cluster.ClusterEvent.ClusterDomainEvent
import akka.persistence.SnapshotOffer
import akka.persistence.SaveSnapshotSuccess
@ -1183,7 +1183,7 @@ object ShardCoordinator {
*/
class ShardCoordinator(handOffTimeout: FiniteDuration, rebalanceInterval: FiniteDuration,
snapshotInterval: FiniteDuration, allocationStrategy: ShardCoordinator.ShardAllocationStrategy)
extends EventsourcedProcessor with ActorLogging {
extends PersistentActor with ActorLogging {
import ShardCoordinator._
import ShardCoordinator.Internal._
import ShardRegion.ShardId

View file

@ -11,7 +11,7 @@ import akka.actor.Identify
import akka.actor.PoisonPill
import akka.actor.Props
import akka.cluster.Cluster
import akka.persistence.EventsourcedProcessor
import akka.persistence.PersistentActor
import akka.persistence.Persistence
import akka.persistence.journal.leveldb.SharedLeveldbJournal
import akka.persistence.journal.leveldb.SharedLeveldbStore
@ -72,7 +72,7 @@ object ClusterShardingSpec extends MultiNodeConfig {
case object Stop
final case class CounterChanged(delta: Int)
class Counter extends EventsourcedProcessor {
class Counter extends PersistentActor {
import ShardRegion.Passivate
context.setReceiveTimeout(120.seconds)

View file

@ -13,7 +13,7 @@ import akka.actor.PoisonPill;
import akka.actor.Props;
import akka.actor.ReceiveTimeout;
import akka.japi.Procedure;
import akka.persistence.UntypedEventsourcedProcessor;
import akka.persistence.UntypedPersistentActor;
// Doc code, compile only
public class ClusterShardingTest {
@ -78,7 +78,7 @@ public class ClusterShardingTest {
}
static//#counter-actor
public class Counter extends UntypedEventsourcedProcessor {
public class Counter extends UntypedPersistentActor {
public static enum CounterOp {
INCREMENT, DECREMENT

View file

@ -4,13 +4,17 @@
package docs.persistence;
import java.util.concurrent.TimeUnit;
import scala.Option;
import scala.concurrent.duration.Duration;
import akka.actor.*;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.japi.Procedure;
import akka.persistence.*;
import scala.Option;
import scala.concurrent.duration.Duration;
import java.util.concurrent.TimeUnit;
import static java.util.Arrays.asList;
public class PersistenceDocTest {
@ -353,11 +357,11 @@ public class PersistenceDocTest {
static Object o8 = new Object() {
//#reliable-event-delivery
class MyEventsourcedProcessor extends UntypedEventsourcedProcessor {
class MyPersistentActor extends UntypedPersistentActor {
private ActorRef destination;
private ActorRef channel;
public MyEventsourcedProcessor(ActorRef destination) {
public MyPersistentActor(ActorRef destination) {
this.destination = destination;
this.channel = getContext().actorOf(Channel.props(), "channel");
}
@ -390,6 +394,53 @@ public class PersistenceDocTest {
};
static Object o9 = new Object() {
//#persist-async
class MyPersistentActor extends UntypedPersistentActor {
@Override
public void onReceiveRecover(Object msg) {
// handle recovery here
}
@Override
public void onReceiveCommand(Object msg) {
sender().tell(msg, getSelf());
persistAsync(String.format("evt-%s-1", msg), new Procedure<String>(){
@Override
public void apply(String event) throws Exception {
sender().tell(event, self());
}
});
persistAsync(String.format("evt-%s-2", msg), new Procedure<String>(){
@Override
public void apply(String event) throws Exception {
sender().tell(event, self());
}
});
}
}
//#persist-async
public void usage() {
final ActorSystem system = ActorSystem.create("example");
//#view-update
final ActorRef processor = system.actorOf(Props.create(MyPersistentActor.class));
processor.tell("a", null);
processor.tell("b", null);
// possible order of received messages:
// a
// b
// evt-a-1
// evt-a-2
// evt-b-1
// evt-b-2
//#view-update
}
};
static Object o10 = new Object() {
//#view
class MyView extends UntypedView {
@Override

View file

@ -442,13 +442,13 @@ also process commands that do not change application state, such as query comman
.. _Event Sourcing: http://martinfowler.com/eaaDev/EventSourcing.html
Akka persistence supports event sourcing with the ``AbstractEventsourcedProcessor`` abstract class (which implements
Akka persistence supports event sourcing with the ``AbstractPersistentActor`` abstract class (which implements
event sourcing as a pattern on top of command sourcing). A processor that extends this abstract class does not handle
``Persistent`` messages directly but uses the ``persist`` method to persist and handle events. The behavior of an
``AbstractEventsourcedProcessor`` is defined by implementing ``receiveRecover`` and ``receiveCommand``. This is
``AbstractEventsPersistentActordefined by implementing ``receiveRecover`` and ``receiveCommand``. This is
demonstrated in the following example.
.. includecode:: ../../../akka-samples/akka-sample-persistence-java-lambda/src/main/java/sample/persistence/EventsourcedExample.java#eventsourced-example
.. includecode:: ../../../akka-samples/akka-sample-persistence-java-lambda/src/main/java/sample/persistence/PersistentActorExample.java#persistent-actor-example
The example defines two data types, ``Cmd`` and ``Evt`` to represent commands and events, respectively. The
``state`` of the ``ExampleProcessor`` is a list of persisted event data contained in ``ExampleState``.
@ -473,7 +473,7 @@ calls in context of a single command.
The easiest way to run this example yourself is to download `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_
and open the tutorial named `Akka Persistence Samples in Java with Lambdas <http://www.typesafe.com/activator/template/akka-sample-persistence-java-lambda>`_.
It contains instructions on how to run the ``EventsourcedExample``.
It contains instructions on how to run the ``PersistentActorExample``.
.. note::
@ -518,9 +518,9 @@ Applications that want to have more explicit control over batch writes and batch
size is greater than ``max-message-batch-size``. Also, a ``PersistentBatch`` is written isolated from other batches.
``Persistent`` messages contained in a ``PersistentBatch`` are received individually by a processor.
``PersistentBatch`` messages, for example, are used internally by an ``AbstractEventsourcedProcessor`` to ensure atomic
``PersistentBatch`` messages, for example, are used internally by an ``AbstractEventsourcedPersistentActor atomic
writes of events. All events that are persisted in context of a single command are written as a single batch to the
journal (even if ``persist`` is called multiple times per command). The recovery of an ``AbstractEventsourcedProcessor``
journal (even if ``persist`` is called multiple times per command). The recovery of an ``AbstractPersistentActor``
will therefore never be done partially (with only a subset of events persisted by a single command).
Confirmation and deletion operations performed by :ref:`channels-java-lambda` are also batched. The maximum

View file

@ -48,9 +48,14 @@ Akka persistence is a separate jar file. Make sure that you have the following d
Architecture
============
* *Processor*: A processor is a persistent, stateful actor. Messages sent to a processor are written to a journal
before its ``onReceive`` method is called. When a processor is started or restarted, journaled messages are replayed
to that processor, so that it can recover internal state from these messages.
* *Processor* (deprecated, use *PersistentActor* instead): A processor is a persistent, stateful actor. Messages sent
to a processor are written to a journal before its ``onReceive`` method is called. When a processor is started or
restarted, journaled messages are replayed to that processor, so that it can recover internal state from these messages.
* *PersistentActor*: Is a persistent, stateful actor. It is able to persist events to a journal and can react to
them in a thread-safe manner. It can be used to implement both *command* as well as *event sourced* actors.
When a persistent actor is started or restarted, journaled messages are replayed to that actor, so that it can
recover internal state from these messages.
* *View*: A view is a persistent, stateful actor that receives journaled messages that have been written by another
processor. A view itself does not journal new messages, instead, it updates internal state only from a processor's
@ -79,6 +84,12 @@ Architecture
Processors
==========
.. warning::
``Processor`` is deprecated. Instead the current ``PersistentActor`` will be extended to provide equivalent
functionality if required (by introducing the ``persistAsync`` method).
For details see `Relaxed local consistency requirements and high throughput use-cases`_ as well as the discussion
and pull requests related to this `issue on Github <https://github.com/akka/akka/issues/15230>`_.
A processor can be implemented by extending the abstract ``UntypedProcessor`` class and implementing the
``onReceive`` method.
@ -451,6 +462,11 @@ use the ``deleteSnapshots`` method.
Event sourcing
==============
.. note::
The ``PersistentActor`` introduced in this section was previously known as ``EventsourcedProcessor``,
which was a subset of the ``PersistentActor``. Migrating your code to use persistent actors instead is
very simple and is explained in the :ref:`migration-guide-persistence-experimental-2.3.x-2.4.x`.
In all the examples so far, messages that change a processor's state have been sent as ``Persistent`` messages
by an application, so that they can be replayed during recovery. From this point of view, the journal acts as
a write-ahead-log for whatever ``Persistent`` messages a processor receives. This is also known as *command
@ -468,13 +484,13 @@ also process commands that do not change application state, such as query comman
.. _Event Sourcing: http://martinfowler.com/eaaDev/EventSourcing.html
Akka persistence supports event sourcing with the abstract ``UntypedEventsourcedProcessor`` class (which implements
Akka persistence supports event sourcing with the abstract ``UntypedPersistentActor`` class (which implements
event sourcing as a pattern on top of command sourcing). A processor that extends this abstract class does not handle
``Persistent`` messages directly but uses the ``persist`` method to persist and handle events. The behavior of an
``UntypedEventsourcedProcessor`` is defined by implementing ``onReceiveRecover`` and ``onReceiveCommand``. This is
``UntypedPersistentActor`` is defined by implementing ``onReceiveRecover`` and ``onReceiveCommand``. This is
demonstrated in the following example.
.. includecode:: ../../../akka-samples/akka-sample-persistence-java/src/main/java/sample/persistence/EventsourcedExample.java#eventsourced-example
.. includecode:: ../../../akka-samples/akka-sample-persistence-java/src/main/java/sample/persistence/PersistentActorExample.java#persistent-actor-example
The example defines two data types, ``Cmd`` and ``Evt`` to represent commands and events, respectively. The
``state`` of the ``ExampleProcessor`` is a list of persisted event data contained in ``ExampleState``.
@ -499,7 +515,7 @@ calls in context of a single command.
The easiest way to run this example yourself is to download `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_
and open the tutorial named `Akka Persistence Samples with Java <http://www.typesafe.com/activator/template/akka-sample-persistence-java>`_.
It contains instructions on how to run the ``EventsourcedExample``.
It contains instructions on how to run the ``PersistentActorExample``.
.. note::
@ -508,6 +524,29 @@ It contains instructions on how to run the ``EventsourcedExample``.
recovery you need to take special care to perform the same state transitions with ``become`` and
``unbecome`` in the ``receiveRecover`` method as you would have done in the command handler.
.. _persist-async-java:
Relaxed local consistency requirements and high throughput use-cases
--------------------------------------------------------------------
If faced with Relaxed local consistency requirements and high throughput demands sometimes ``PersistentActor`` and it's
``persist`` may not be enough in terms of consuming incoming Commands at a high rate, because it has to wait until all
Events related to a given Command are processed in order to start processing the next Command. While this abstraction is
very useful for most cases, sometimes you may be faced with relaxed requirements about consistency for example you may
want to process commands as fast as you can, assuming that Event will eventually be persisted and handled properly in
the background and retroactively reacting to persistence failures if needed.
The ``persistAsync`` method provides a tool for implementing high-throughput processors. It will *not*
stash incoming Commands while the Journal is still working on persisting and/or user code is executing event callbacks.
In the below example, the event callbacks may be called "at any time", even after the next Command has been processed.
The ordering between events is still guaranteed ("evt-b-1" will be sent after "evt-a-2", which will be sent after "evt-a-1" etc.).
.. includecode:: code/docs/persistence/PersistenceDocTest.java#persist-async
.. note::
In order to implement the pattern known as "*command sourcing*" simply ``persistAsync`` all incoming events right away,
and handle them in the callback.
Reliable event delivery
-----------------------
@ -543,9 +582,9 @@ Applications that want to have more explicit control over batch writes and batch
size is greater than ``max-message-batch-size``. Also, a ``PersistentBatch`` is written isolated from other batches.
``Persistent`` messages contained in a ``PersistentBatch`` are received individually by a processor.
``PersistentBatch`` messages, for example, are used internally by an ``UntypedEventsourcedProcessor`` to ensure atomic
``PersistentBatch`` messages, for example, are used internally by an ``UntypedPersistentActor`` to ensure atomic
writes of events. All events that are persisted in context of a single command are written as a single batch to the
journal (even if ``persist`` is called multiple times per command). The recovery of an ``UntypedEventsourcedProcessor``
journal (even if ``persist`` is called multiple times per command). The recovery of an ``UntypedPersistentActor``
will therefore never be done partially (with only a subset of events persisted by a single command).
Confirmation and deletion operations performed by :ref:`channels-java` are also batched. The maximum confirmation

View file

@ -1,8 +1,8 @@
.. _migration-eventsourced-2.3:
#######################################################
Migration Guide Eventsourced to Akka Persistence 2.3.x
#######################################################
######################################################
Migration Guide Eventsourced to Akka Persistence 2.3.x
######################################################
General notes
=============

View file

@ -0,0 +1,67 @@
.. _migration-guide-persistence-experimental-2.3.x-2.4.x:
#####################################################
Migration Guide Akka Persistence (experimental) 2.3.3
#####################################################
**Akka Persistence** is an **experimental module**, which means that neither Binary Compatibility nor API stability
is provided for Persistence while under the *experimental* flag. The goal of this phase is to gather user feedback
before we freeze the APIs in a major release.
Renamed EventsourcedProcessor to PersistentActor
================================================
``EventsourcedProcessor`` is now deprecated and replaced by ``PersistentActor`` which provides the same (and more) API.
Migrating to ``2.4.x`` is as simple as changing all your classes to extending ``PersistentActor``.
Replace all classes like::
class DeprecatedProcessor extends EventsourcedProcessor { /*...*/ }
To extend ``PersistentActor``::
class NewPersistentProcessor extends PersistentActor { /*...*/ }
No other API changes are required for this migration.
Removed Processor in favour of extending PersistentActor with persistAsync
==========================================================================
The ``Processor`` is now deprecated since ``2.3.4`` and will be removed in ``2.4.x``.
It's semantics replicated in ``PersistentActor`` in the form of an additional ``persist`` method: ``persistAsync``.
In essence, the difference betwen ``persist`` and ``persistAsync`` is that the former will stash all incomming commands
until all persist callbacks have been processed, whereas the latter does not stash any commands. The new ``persistAsync``
should be used in cases of low consistency yet high responsiveness requirements, the Actor can keep processing incomming
commands, even though not all previous events have been handled.
When these ``persist`` and ``persistAsync`` are used together in the same ``PersistentActor``, the ``persist``
logic will win over the async version so that all guarantees concerning persist still hold. This will however lower
the throughput
Now deprecated code using Processor::
class OldProcessor extends Processor {
def receive = {
case Persistent(cmd) => sender() ! cmd
}
}
Replacement code, with the same semantics, using PersistentActor::
class NewProcessor extends PersistentActor {
def receiveCommand = {
case cmd =>
persistAsync(cmd) { e => sender() ! e }
}
def receiveEvent = {
case _ => // logic for handling replay
}
}
It is worth pointing out that using ``sender()`` inside the persistAsync callback block is **valid**, and does *not* suffer
any of the problems Futures have when closing over the sender reference.
Using the``PersistentActor`` instead of ``Processor`` also shifts the responsibility of deciding if a message should be persisted
to the receiver instead of the sender of the message. Previously, using ``Processor``, clients would have to wrap messages as ``Persistent(cmd)``
manually, as well as have to be aware of the receiver being a ``Processor``, which didn't play well with transparency of the ActorRefs in general.

View file

@ -11,4 +11,5 @@ Migration Guides
migration-guide-2.1.x-2.2.x
migration-guide-2.2.x-2.3.x
migration-guide-2.3.x-2.4.x
migration-guide-persistence-experimental-2.3.x-2.4.x
migration-guide-eventsourced-2.3.x

View file

@ -7,7 +7,7 @@ package docs.persistence
import scala.concurrent.duration._
import scala.language.postfixOps
import akka.actor.{ Actor, ActorSystem }
import akka.actor.{ Props, Actor, ActorSystem }
import akka.persistence._
trait PersistenceDocSpec {
@ -314,7 +314,7 @@ trait PersistenceDocSpec {
import akka.actor.ActorRef
//#reliable-event-delivery
class MyEventsourcedProcessor(destination: ActorRef) extends EventsourcedProcessor {
class MyPersistentActor(destination: ActorRef) extends PersistentActor {
val channel = context.actorOf(Channel.props("channel"))
def handleEvent(event: String) = {
@ -337,6 +337,42 @@ trait PersistenceDocSpec {
}
//#reliable-event-delivery
}
new AnyRef {
import akka.actor.ActorRef
val processor = system.actorOf(Props[MyPersistentActor]())
//#persist-async
class MyPersistentActor extends PersistentActor {
def receiveRecover: Receive = {
case _ => // handle recovery here
}
def receiveCommand: Receive = {
case c: String => {
sender() ! c
persistAsync(s"evt-$c-1") { e => sender() ! e }
persistAsync(s"evt-$c-2") { e => sender() ! e }
}
}
}
// usage
processor ! "a"
processor ! "b"
// possible order of received messages:
// a
// b
// evt-a-1
// evt-a-2
// evt-b-1
// evt-b-2
//#persist-async
}
new AnyRef {
import akka.actor.Props

View file

@ -36,9 +36,14 @@ Akka persistence is a separate jar file. Make sure that you have the following d
Architecture
============
* *Processor*: A processor is a persistent, stateful actor. Messages sent to a processor are written to a journal
before its ``receive`` method is called. When a processor is started or restarted, journaled messages are replayed
to that processor, so that it can recover internal state from these messages.
* *Processor* (deprecated, use *PersistentActor* instead): A processor is a persistent, stateful actor. Messages sent
to a processor are written to a journal before its ``onReceive`` method is called. When a processor is started or
restarted, journaled messages are replayed to that processor, so that it can recover internal state from these messages.
* *PersistentActor*: Is a persistent, stateful actor. It is able to persist events to a journal and can react to
them in a thread-safe manner. It can be used to implement both *command* as well as *event sourced* actors.
When a persistent actor is started or restarted, journaled messages are replayed to that actor, so that it can
recover internal state from these messages.
* *View*: A view is a persistent, stateful actor that receives journaled messages that have been written by another
processor. A view itself does not journal new messages, instead, it updates internal state only from a processor's
@ -67,6 +72,12 @@ Architecture
Processors
==========
.. warning::
``Processor`` is deprecated. Instead the current ``PersistentActor`` will be extended to provide equivalent
functionality if required (by introducing the ``persistAsync`` method).
For details see `Relaxed local consistency requirements and high throughput use-cases`_ as well as the discussion
and pull requests related to this `issue on Github <https://github.com/akka/akka/issues/15230>`_.
A processor can be implemented by extending the ``Processor`` trait and implementing the ``receive`` method.
.. includecode:: code/docs/persistence/PersistenceDocSpec.scala#definition
@ -463,6 +474,11 @@ use the ``deleteSnapshots`` method.
Event sourcing
==============
.. note::
The ``PersistentActor`` introduced in this section was previously known as ``EventsourcedProcessor``
which was a subset of the ``PersistentActor``. Migrating your code to use persistent actors instead is
very simple and is explained in the :ref:`migration-guide-persistence-experimental-2.3.x-2.4.x`.
In all the examples so far, messages that change a processor's state have been sent as ``Persistent`` messages
by an application, so that they can be replayed during recovery. From this point of view, the journal acts as
a write-ahead-log for whatever ``Persistent`` messages a processor receives. This is also known as *command
@ -480,12 +496,12 @@ also process commands that do not change application state, such as query comman
.. _Event Sourcing: http://martinfowler.com/eaaDev/EventSourcing.html
Akka persistence supports event sourcing with the ``EventsourcedProcessor`` trait (which implements event sourcing
Akka persistence supports event sourcing with the ``PersistentActor`` trait (which implements event sourcing
as a pattern on top of command sourcing). A processor that extends this trait does not handle ``Persistent`` messages
directly but uses the ``persist`` method to persist and handle events. The behavior of an ``EventsourcedProcessor``
directly but uses the ``persist`` method to persist and handle events. The behavior of an ``PersistentActor``
is defined by implementing ``receiveRecover`` and ``receiveCommand``. This is demonstrated in the following example.
.. includecode:: ../../../akka-samples/akka-sample-persistence-scala/src/main/scala/sample/persistence/EventsourcedExample.scala#eventsourced-example
.. includecode:: ../../../akka-samples/akka-sample-persistence-scala/src/main/scala/sample/persistence/PersistentActorExample.scala#persistent-actor-example
The example defines two data types, ``Cmd`` and ``Evt`` to represent commands and events, respectively. The
``state`` of the ``ExampleProcessor`` is a list of persisted event data contained in ``ExampleState``.
@ -510,7 +526,7 @@ calls in context of a single command.
The easiest way to run this example yourself is to download `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_
and open the tutorial named `Akka Persistence Samples with Scala <http://www.typesafe.com/activator/template/akka-sample-persistence-scala>`_.
It contains instructions on how to run the ``EventsourcedExample``.
It contains instructions on how to run the ``PersistentActorExample``.
.. note::
@ -519,6 +535,34 @@ It contains instructions on how to run the ``EventsourcedExample``.
recovery you need to take special care to perform the same state transitions with ``become`` and
``unbecome`` in the ``receiveRecover`` method as you would have done in the command handler.
.. _persist-async-scala:
Relaxed local consistency requirements and high throughput use-cases
--------------------------------------------------------------------
If faced with Relaxed local consistency requirements and high throughput demands sometimes ``PersistentActor`` and it's
``persist`` may not be enough in terms of consuming incoming Commands at a high rate, because it has to wait until all
Events related to a given Command are processed in order to start processing the next Command. While this abstraction is
very useful for most cases, sometimes you may be faced with relaxed requirements about consistency for example you may
want to process commands as fast as you can, assuming that Event will eventually be persisted and handled properly in
the background and retroactively reacting to persistence failures if needed.
The ``persistAsync`` method provides a tool for implementing high-throughput processors. It will *not*
stash incoming Commands while the Journal is still working on persisting and/or user code is executing event callbacks.
In the below example, the event callbacks may be called "at any time", even after the next Command has been processed.
The ordering between events is still guaranteed ("evt-b-1" will be sent after "evt-a-2", which will be sent after "evt-a-1" etc.).
.. includecode:: code/docs/persistence/PersistenceDocSpec.scala#persist-async
Notice that the client does not have to wrap any messages in the `Persistent` class in order to obtain "command sourcing like"
semantics. It's up to the processor to decide about persisting (or not) of messages, unlike ``Processor`` where this decision
was made by the sender.
.. note::
In order to implement the "*command sourcing*" simply call ``persistAsync(cmd)(...)`` right away on all incomming
messages right away, and handle them in the callback.
Reliable event delivery
-----------------------
@ -556,9 +600,9 @@ Applications that want to have more explicit control over batch writes and batch
size is greater than ``max-message-batch-size``. Also, a ``PersistentBatch`` is written isolated from other batches.
``Persistent`` messages contained in a ``PersistentBatch`` are received individually by a processor.
``PersistentBatch`` messages, for example, are used internally by an ``EventsourcedProcessor`` to ensure atomic
``PersistentBatch`` messages, for example, are used internally by an ``PersistentActor`` to ensure atomic
writes of events. All events that are persisted in context of a single command are written as a single batch to the
journal (even if ``persist`` is called multiple times per command). The recovery of an ``EventsourcedProcessor``
journal (even if ``persist`` is called multiple times per command). The recovery of an ``PersistentActor``
will therefore never be done partially (with only a subset of events persisted by a single command).
Confirmation and deletion operations performed by :ref:`channels` are also batched. The maximum confirmation

View file

@ -18,6 +18,8 @@ import akka.actor.AbstractActor
* Event sourcing mixin for a [[Processor]].
*/
private[persistence] trait Eventsourced extends Processor {
// TODO consolidate these traits as PersistentActor #15230
/**
* Processor recovery state. Waits for recovery completion and then changes to
* `processingCommands`
@ -56,22 +58,44 @@ private[persistence] trait Eventsourced extends Processor {
throw new UnsupportedOperationException("Persistent command batches not supported")
case _: PersistentRepr
throw new UnsupportedOperationException("Persistent commands not supported")
case WriteMessageSuccess(p)
withCurrentPersistent(p)(p persistInvocations.get(0).handler(p.payload))
onWriteComplete()
case s @ WriteMessagesSuccessful Eventsourced.super.aroundReceive(receive, s)
case f: WriteMessagesFailed Eventsourced.super.aroundReceive(receive, f)
case _
doAroundReceive(receive, message)
}
private def doAroundReceive(receive: Receive, message: Any): Unit = {
Eventsourced.super.aroundReceive(receive, LoopMessageSuccess(message))
if (!persistInvocations.isEmpty) {
if (pendingStashingPersistInvocations > 0) {
currentState = persistingEvents
}
if (persistentEventBatch.nonEmpty) {
Eventsourced.super.aroundReceive(receive, PersistentBatch(persistentEventBatch.reverse))
persistInvocations = persistInvocations.reverse
persistentEventBatch = Nil
} else {
processorStash.unstash()
}
}
private def onWriteComplete(): Unit = {
persistInvocations.remove(0)
val nextIsStashing = !persistInvocations.isEmpty && persistInvocations.get(0).isInstanceOf[StashingPersistInvocation]
if (nextIsStashing) {
currentState = persistingEvents
}
if (persistInvocations.isEmpty) {
processorStash.unstash()
}
}
}
/**
@ -86,29 +110,38 @@ private[persistence] trait Eventsourced extends Processor {
case _: ConfirmablePersistent
processorStash.stash()
case PersistentBatch(b)
b.foreach(p deleteMessage(p.sequenceNr, true))
b.foreach(p deleteMessage(p.sequenceNr, permanent = true))
throw new UnsupportedOperationException("Persistent command batches not supported")
case p: PersistentRepr
deleteMessage(p.sequenceNr, true)
deleteMessage(p.sequenceNr, permanent = true)
throw new UnsupportedOperationException("Persistent commands not supported")
case WriteMessageSuccess(p)
withCurrentPersistent(p)(p persistInvocations.head._2(p.payload))
onWriteComplete()
val invocation = persistInvocations.get(0)
withCurrentPersistent(p)(p invocation.handler(p.payload))
onWriteComplete(invocation)
case e @ WriteMessageFailure(p, _)
Eventsourced.super.aroundReceive(receive, message) // stops actor by default
onWriteComplete()
case s @ WriteMessagesSuccess Eventsourced.super.aroundReceive(receive, s)
case f: WriteMessagesFailure Eventsourced.super.aroundReceive(receive, f)
case other processorStash.stash()
onWriteComplete(persistInvocations.get(0))
case s @ WriteMessagesSuccessful Eventsourced.super.aroundReceive(receive, s)
case f: WriteMessagesFailed Eventsourced.super.aroundReceive(receive, f)
case other processorStash.stash()
}
def onWriteComplete(): Unit = {
persistInvocations = persistInvocations.tail
if (persistInvocations.isEmpty) {
private def onWriteComplete(invocation: PersistInvocation): Unit = {
if (invocation.isInstanceOf[StashingPersistInvocation]) {
// enables an early return to `processingCommands`, because if this counter hits `0`,
// we know the remaining persistInvocations are all `persistAsync` created, which
// means we can go back to processing commands also - and these callbacks will be called as soon as possible
pendingStashingPersistInvocations -= 1
}
persistInvocations.remove(0)
if (persistInvocations.isEmpty || pendingStashingPersistInvocations == 0) {
currentState = processingCommands
processorStash.unstash()
}
}
}
/**
@ -126,7 +159,18 @@ private[persistence] trait Eventsourced extends Processor {
receiveRecover(f)
}
private var persistInvocations: List[(Any, Any Unit)] = Nil
sealed trait PersistInvocation {
def handler: Any Unit
}
/** forces processor to stash incoming commands untill all these invocations are handled */
final case class StashingPersistInvocation(evt: Any, handler: Any Unit) extends PersistInvocation
/** does not force the processor to stash commands */
final case class AsyncPersistInvocation(evt: Any, handler: Any Unit) extends PersistInvocation
/** Used instead of iterating `persistInvocations` in order to check if safe to revert to processing commands */
private var pendingStashingPersistInvocations: Long = 0
/** Holds user-supplied callbacks for persist/persistAsync calls */
private val persistInvocations = new java.util.LinkedList[PersistInvocation]() // we only append / isEmpty / get(0) on it
private var persistentEventBatch: List[PersistentRepr] = Nil
private var currentState: State = recovering
@ -151,11 +195,12 @@ private[persistence] trait Eventsourced extends Processor {
* If persistence of an event fails, the processor will be stopped. This can be customized by
* handling [[PersistenceFailure]] in [[receiveCommand]].
*
* @param event event to be persisted.
* @param event event to be persisted
* @param handler handler for each persisted `event`
*/
final def persist[A](event: A)(handler: A Unit): Unit = {
persistInvocations = (event, handler.asInstanceOf[Any Unit]) :: persistInvocations
pendingStashingPersistInvocations += 1
persistInvocations addLast StashingPersistInvocation(event, handler.asInstanceOf[Any Unit])
persistentEventBatch = PersistentRepr(event) :: persistentEventBatch
}
@ -164,12 +209,47 @@ private[persistence] trait Eventsourced extends Processor {
* `persist[A](event: A)(handler: A => Unit)` multiple times with the same `handler`,
* except that `events` are persisted atomically with this method.
*
* @param events events to be persisted.
* @param events events to be persisted
* @param handler handler for each persisted `events`
*/
final def persist[A](events: immutable.Seq[A])(handler: A Unit): Unit =
events.foreach(persist(_)(handler))
/**
* Asynchronously persists `event`. On successful persistence, `handler` is called with the
* persisted event.
*
* Unlike `persist` the processor will continue to receive incomming commands between the
* call to `persist` and executing it's `handler`. This asynchronous, non-stashing, version of
* of persist should be used when you favor throughput over the "command-2 only processed after
* command-1 effects' have been applied" guarantee, which is provided by the plain [[persist]] method.
*
* An event `handler` may close over processor state and modify it. The `sender` of a persisted
* event is the sender of the corresponding command. This means that one can reply to a command
* sender within an event `handler`.
*
* If persistence of an event fails, the processor will be stopped. This can be customized by
* handling [[PersistenceFailure]] in [[receiveCommand]].
*
* @param event event to be persisted
* @param handler handler for each persisted `event`
*/
final def persistAsync[A](event: A)(handler: A Unit): Unit = {
persistInvocations addLast AsyncPersistInvocation(event, handler.asInstanceOf[Any Unit])
persistentEventBatch = PersistentRepr(event) :: persistentEventBatch
}
/**
* Asynchronously persists `events` in specified order. This is equivalent to calling
* `persistAsync[A](event: A)(handler: A => Unit)` multiple times with the same `handler`,
* except that `events` are persisted atomically with this method.
*
* @param events events to be persisted
* @param handler handler for each persisted `events`
*/
final def persistAsync[A](events: immutable.Seq[A])(handler: A Unit): Unit =
events.foreach(persistAsync(_)(handler))
/**
* Recovery handler that receives persisted events during recovery. If a state snapshot
* has been captured and saved, this handler will receive a [[SnapshotOffer]] message
@ -235,13 +315,31 @@ private[persistence] trait Eventsourced extends Processor {
/**
* An event sourced processor.
*/
@deprecated("EventsourcedProcessor will be removed in 2.4.x, instead extend the API equivalent `akka.persistence.PersistentProcessor`", since = "2.3.4")
trait EventsourcedProcessor extends Processor with Eventsourced {
// TODO remove Processor #15230
def receive = receiveCommand
}
/**
* An persistent Actor - can be used to implement command or event sourcing.
*/
// TODO remove EventsourcedProcessor / Processor #15230
trait PersistentActor extends EventsourcedProcessor
/**
* Java API: an persistent actor - can be used to implement command or event sourcing.
*/
abstract class UntypedPersistentActor extends UntypedEventsourcedProcessor
/**
* Java API: an persistent actor - can be used to implement command or event sourcing.
*/
abstract class AbstractPersistentActor extends AbstractEventsourcedProcessor
/**
* Java API: an event sourced processor.
*/
@deprecated("UntypedEventsourcedProcessor will be removed in 2.4.x, instead extend the API equivalent `akka.persistence.PersistentProcessor`", since = "2.3.4")
abstract class UntypedEventsourcedProcessor extends UntypedProcessor with Eventsourced {
final def onReceive(message: Any) = onReceiveCommand(message)
@ -289,6 +387,39 @@ abstract class UntypedEventsourcedProcessor extends UntypedProcessor with Events
final def persist[A](events: JIterable[A], handler: Procedure[A]): Unit =
persist(Util.immutableSeq(events))(event handler(event))
/**
* JAVA API: asynchronously persists `event`. On successful persistence, `handler` is called with the
* persisted event.
*
* Unlike `persist` the processor will continue to receive incomming commands between the
* call to `persist` and executing it's `handler`. This asynchronous, non-stashing, version of
* of persist should be used when you favor throughput over the "command-2 only processed after
* command-1 effects' have been applied" guarantee, which is provided by the plain [[persist]] method.
*
* An event `handler` may close over processor state and modify it. The `sender` of a persisted
* event is the sender of the corresponding command. This means that one can reply to a command
* sender within an event `handler`.
*
* If persistence of an event fails, the processor will be stopped. This can be customized by
* handling [[PersistenceFailure]] in [[receiveCommand]].
*
* @param event event to be persisted
* @param handler handler for each persisted `event`
*/
final def persistAsync[A](event: A)(handler: Procedure[A]): Unit =
super[Eventsourced].persistAsync(event)(event handler(event))
/**
* JAVA API: asynchronously persists `events` in specified order. This is equivalent to calling
* `persistAsync[A](event: A)(handler: A => Unit)` multiple times with the same `handler`,
* except that `events` are persisted atomically with this method.
*
* @param events events to be persisted
* @param handler handler for each persisted `events`
*/
final def persistAsync[A](events: JIterable[A])(handler: A Unit): Unit =
super[Eventsourced].persistAsync(Util.immutableSeq(events))(event handler(event))
/**
* Java API: recovery handler that receives persisted events during recovery. If a state snapshot
* has been captured and saved, this handler will receive a [[SnapshotOffer]] message
@ -322,6 +453,7 @@ abstract class UntypedEventsourcedProcessor extends UntypedProcessor with Events
* [[PersistentBatch]] messages. In this case an `UnsupportedOperationException` is
* thrown by the processor.
*/
@deprecated("AbstractEventsourcedProcessor will be removed in 2.4.x, instead extend the API equivalent `akka.persistence.PersistentProcessor`", since = "2.3.4")
abstract class AbstractEventsourcedProcessor extends AbstractActor with EventsourcedProcessor {
/**
* Java API: asynchronously persists `event`. On successful persistence, `handler` is called with the
@ -359,9 +491,37 @@ abstract class AbstractEventsourcedProcessor extends AbstractActor with Eventsou
final def persist[A](events: JIterable[A], handler: Procedure[A]): Unit =
persist(Util.immutableSeq(events))(event handler(event))
/**
* Java API: asynchronously persists `event`. On successful persistence, `handler` is called with the
* persisted event.
*
* Unlike `persist` the processor will continue to receive incomming commands between the
* call to `persistAsync` and executing it's `handler`. This asynchronous, non-stashing, version of
* of persist should be used when you favor throughput over the strict ordering guarantees that `persist` guarantees.
*
* If persistence of an event fails, the processor will be stopped. This can be customized by
* handling [[PersistenceFailure]] in [[receiveCommand]].
*
* @param event event to be persisted
* @param handler handler for each persisted `event`
*/
final def persistAsync[A](event: A, handler: Procedure[A]): Unit =
persistAsync(event)(event handler(event))
/**
* Java API: asynchronously persists `events` in specified order. This is equivalent to calling
* `persistAsync[A](event: A)(handler: A => Unit)` multiple times with the same `handler`,
* except that `events` are persisted atomically with this method.
*
* @param events events to be persisted
* @param handler handler for each persisted `events`
*/
final def persistAsync[A](events: JIterable[A], handler: Procedure[A]): Unit =
persistAsync(Util.immutableSeq(events))(event handler(event))
override def receive = super[EventsourcedProcessor].receive
override def receive(receive: Receive): Unit = {
throw new IllegalArgumentException("Define the behavior by overriding receiveRecover and receiveCommand")
}
}
}

View file

@ -66,7 +66,7 @@ private[persistence] object JournalProtocol {
* Reply message to a successful [[WriteMessages]] request. This reply is sent to the requestor
* before all subsequent [[WriteMessageSuccess]] replies.
*/
case object WriteMessagesSuccess
case object WriteMessagesSuccessful
/**
* Reply message to a failed [[WriteMessages]] request. This reply is sent to the requestor
@ -74,7 +74,7 @@ private[persistence] object JournalProtocol {
*
* @param cause failure cause.
*/
final case class WriteMessagesFailure(cause: Throwable)
final case class WriteMessagesFailed(cause: Throwable)
/**
* Reply message to a successful [[WriteMessages]] request. For each contained [[PersistentRepr]] message

View file

@ -17,6 +17,9 @@ import akka.persistence.serialization.Message
/**
* Persistent message.
*/
@deprecated("Messages wrapped in Persistent were only required by Processor and Command Sourcing, " +
"which is now deprecated. Use `akka.persistence.PersistentActor` and Event Sourcing instead.",
since = "2.3.4")
sealed abstract class Persistent {
/**
* This persistent message's payload.
@ -38,6 +41,9 @@ sealed abstract class Persistent {
def withPayload(payload: Any): Persistent
}
@deprecated("Messages wrapped in Persistent were only required by Processor and Command Sourcing, " +
"which is now deprecated. Use `akka.persistence.PersistentActor` and Event Sourcing instead.",
since = "2.3.4")
object Persistent {
/**
* Java API: creates a new persistent message. Must only be used outside processors.
@ -65,6 +71,9 @@ object Persistent {
* @param payload payload of the new persistent message.
* @param currentPersistentMessage optional current persistent message, defaults to `None`.
*/
@deprecated("Messages wrapped in Persistent were only required by Processor and Command Sourcing, " +
"which is now deprecated. Use `akka.persistence.PersistentActor` and Event Sourcing instead.",
since = "2.3.4")
def apply(payload: Any)(implicit currentPersistentMessage: Option[Persistent] = None): Persistent =
currentPersistentMessage.map(_.withPayload(payload)).getOrElse(PersistentRepr(payload))
@ -79,6 +88,9 @@ object Persistent {
* Persistent message that has been delivered by a [[Channel]] or [[PersistentChannel]]. Channel
* destinations that receive messages of this type can confirm their receipt by calling [[confirm]].
*/
@deprecated("Messages wrapped in Persistent were only required by Processor and Command Sourcing, " +
"which is now deprecated. Use `akka.persistence.PersistentActor` and Event Sourcing instead.",
since = "2.3.4")
sealed abstract class ConfirmablePersistent extends Persistent {
/**
* Called by [[Channel]] and [[PersistentChannel]] destinations to confirm the receipt of a
@ -93,6 +105,9 @@ sealed abstract class ConfirmablePersistent extends Persistent {
def redeliveries: Int
}
@deprecated("Messages wrapped in Persistent were only required by Processor and Command Sourcing, " +
"which is now deprecated. Use `akka.persistence.PersistentActor` and Event Sourcing instead.",
since = "2.3.4")
object ConfirmablePersistent {
/**
* [[ConfirmablePersistent]] extractor.
@ -106,7 +121,12 @@ object ConfirmablePersistent {
* journal. The processor receives the written messages individually as [[Persistent]] messages.
* During recovery, they are also replayed individually.
*/
final case class PersistentBatch(persistentBatch: immutable.Seq[Persistent]) extends Message {
@deprecated("Messages wrapped in Persistent were only required by Processor and Command Sourcing, " +
"which is now deprecated. Use `akka.persistence.PersistentActor` and Event Sourcing instead.",
since = "2.3.4")
case class PersistentBatch(persistentBatch: immutable.Seq[Persistent]) extends Message {
// todo while we want to remove Persistent() from user-land, the batch may (probably?) become private[akka] to remain for journal internals #15230
/**
* INTERNAL API.
*/
@ -151,6 +171,8 @@ private[persistence] final case class PersistentIdImpl(processorId: String, sequ
* @see [[journal.AsyncRecovery]]
*/
trait PersistentRepr extends Persistent with PersistentId with Message {
// todo we want to get rid of the Persistent() wrapper from user land; PersistentRepr is here to stay. #15230
import scala.collection.JavaConverters._
/**

View file

@ -265,7 +265,7 @@ private class RequestWriter(channelId: String, channelSettings: PersistentChanne
override protected[akka] def aroundReceive(receive: Receive, message: Any): Unit = {
super.aroundReceive(receive, message)
message match {
case WriteMessagesSuccess | WriteMessagesFailure(_)
case WriteMessagesSuccessful | WriteMessagesFailed(_)
// activate reader after to reduce delivery latency
reader ! RequestsWritten
case _

View file

@ -51,7 +51,10 @@ import akka.dispatch._
* @see [[Recover]]
* @see [[PersistentBatch]]
*/
@deprecated("Processor will be removed. Instead extend `akka.persistence.PersistentActor` and use it's `persistAsync(command)(callback)` method to get equivalent semantics.", since = "2.3.4")
trait Processor extends Actor with Recovery {
// todo remove Processor in favor of PersistentActor #15230
import JournalProtocol._
/**
@ -96,7 +99,7 @@ trait Processor extends Actor with Recovery {
throw new ActorKilledException(errorMsg)
}
case LoopMessageSuccess(m) process(receive, m)
case WriteMessagesSuccess | WriteMessagesFailure(_)
case WriteMessagesSuccessful | WriteMessagesFailed(_)
if (processorBatch.isEmpty) batching = false else journalBatch()
case p: PersistentRepr
addToBatch(p)
@ -113,7 +116,7 @@ trait Processor extends Actor with Recovery {
}
def addToBatch(p: PersistentRepr): Unit =
processorBatch = processorBatch :+ p.update(processorId = processorId, sequenceNr = nextSequenceNr(), sender = sender)
processorBatch = processorBatch :+ p.update(processorId = processorId, sequenceNr = nextSequenceNr(), sender = sender())
def addToBatch(pb: PersistentBatch): Unit =
pb.persistentReprList.foreach(addToBatch)
@ -386,6 +389,7 @@ final case class RecoveryException(message: String, cause: Throwable) extends Ak
* @see [[Recover]]
* @see [[PersistentBatch]]
*/
@deprecated("UntypedProcessor will be removed. Instead extend `akka.persistence.UntypedPersistentActor` and use it's `persistAsync(command)(callback)` method to get equivalent semantics.", since = "2.3.4")
abstract class UntypedProcessor extends UntypedActor with Processor
/**
@ -440,4 +444,5 @@ abstract class UntypedProcessor extends UntypedActor with Processor
* @see [[Recover]]
* @see [[PersistentBatch]]
*/
@deprecated("AbstractProcessor will be removed. Instead extend `akka.persistence.AbstractPersistentActor` and use it's `persistAsync(command)(callback)` method to get equivalent semantics.", since = "2.3.4")
abstract class AbstractProcessor extends AbstractActor with Processor

View file

@ -36,7 +36,8 @@ trait Recovery extends Actor with Snapshotter with Stash with StashFactory {
def updateLastSequenceNr(value: Long): Unit =
_lastSequenceNr = value
protected def withCurrentPersistent(persistent: Persistent)(body: Persistent Unit): Unit = try {
/** INTERNAL API */
private[akka] def withCurrentPersistent(persistent: Persistent)(body: Persistent Unit): Unit = try {
_currentPersistent = persistent
updateLastSequenceNr(persistent)
body(persistent)

View file

@ -35,10 +35,10 @@ trait AsyncWriteJournal extends Actor with AsyncRecovery {
}
asyncWriteMessages(persistentBatch.map(_.prepareWrite())) onComplete {
case Success(_)
resequencer ! Desequenced(WriteMessagesSuccess, cctr, processor, self)
resequencer ! Desequenced(WriteMessagesSuccessful, cctr, processor, self)
resequence(WriteMessageSuccess(_))
case Failure(e)
resequencer ! Desequenced(WriteMessagesFailure(e), cctr, processor, self)
resequencer ! Desequenced(WriteMessagesFailed(e), cctr, processor, self)
resequence(WriteMessageFailure(_, e))
}
resequencerCounter += persistentBatch.length + 1

View file

@ -26,10 +26,10 @@ trait SyncWriteJournal extends Actor with AsyncRecovery {
case WriteMessages(persistentBatch, processor)
Try(writeMessages(persistentBatch.map(_.prepareWrite()))) match {
case Success(_)
processor ! WriteMessagesSuccess
processor ! WriteMessagesSuccessful
persistentBatch.foreach(p processor.tell(WriteMessageSuccess(p), p.sender))
case Failure(e)
processor ! WriteMessagesFailure(e)
processor ! WriteMessagesFailed(e)
persistentBatch.foreach(p processor tell (WriteMessageFailure(p, e), p.sender))
throw e
}

View file

@ -87,7 +87,7 @@ object PerformanceSpec {
}
}
class EventsourcedTestProcessor(name: String) extends PerformanceTestProcessor(name) with EventsourcedProcessor {
class EventsourcedTestProcessor(name: String) extends PerformanceTestProcessor(name) with PersistentActor {
val receiveRecover: Receive = {
case _ if (lastSequenceNr % 1000 == 0) print("r")
}
@ -100,7 +100,7 @@ object PerformanceSpec {
}
}
class StashingEventsourcedTestProcessor(name: String) extends PerformanceTestProcessor(name) with EventsourcedProcessor {
class StashingEventsourcedTestProcessor(name: String) extends PerformanceTestProcessor(name) with PersistentActor {
val receiveRecover: Receive = {
case _ if (lastSequenceNr % 1000 == 0) print("r")
}
@ -141,7 +141,7 @@ class PerformanceSpec extends AkkaSpec(PersistenceSpec.config("leveldb", "Perfor
}
}
def stressEventsourcedProcessor(failAt: Option[Long]): Unit = {
def stressPersistentActor(failAt: Option[Long]): Unit = {
val processor = namedProcessor[EventsourcedTestProcessor]
failAt foreach { processor ! FailAt(_) }
1 to warmupCycles foreach { i processor ! s"msg${i}" }
@ -153,7 +153,7 @@ class PerformanceSpec extends AkkaSpec(PersistenceSpec.config("leveldb", "Perfor
}
}
def stressStashingEventsourcedProcessor(): Unit = {
def stressStashingPersistentActor(): Unit = {
val processor = namedProcessor[StashingEventsourcedTestProcessor]
1 to warmupCycles foreach { i processor ! "b" }
processor ! StartMeasure
@ -195,13 +195,13 @@ class PerformanceSpec extends AkkaSpec(PersistenceSpec.config("leveldb", "Perfor
"An event sourced processor" should {
"have some reasonable throughput" in {
stressEventsourcedProcessor(None)
stressPersistentActor(None)
}
"have some reasonable throughput under failure conditions" in {
stressEventsourcedProcessor(Some(warmupCycles + loadCycles / 10))
stressPersistentActor(Some(warmupCycles + loadCycles / 10))
}
"have some reasonable throughput with stashing and unstashing every 3rd command" in {
stressStashingEventsourcedProcessor()
stressStashingPersistentActor()
}
}

View file

@ -11,12 +11,14 @@ import akka.actor._
import akka.testkit.{ ImplicitSender, AkkaSpec }
import akka.testkit.EventFilter
import akka.testkit.TestProbe
import java.util.concurrent.atomic.AtomicInteger
import scala.util.Random
object EventsourcedSpec {
object PersistentActorSpec {
final case class Cmd(data: Any)
final case class Evt(data: Any)
abstract class ExampleProcessor(name: String) extends NamedProcessor(name) with EventsourcedProcessor {
abstract class ExampleProcessor(name: String) extends NamedProcessor(name) with PersistentActor {
var events: List[Any] = Nil
val updateState: Receive = {
@ -25,7 +27,7 @@ object EventsourcedSpec {
val commonBehavior: Receive = {
case "boom" throw new TestException("boom")
case GetState sender ! events.reverse
case GetState sender() ! events.reverse
}
def receiveRecover = updateState
@ -122,7 +124,7 @@ object EventsourcedSpec {
}
}
class SnapshottingEventsourcedProcessor(name: String, probe: ActorRef) extends ExampleProcessor(name) {
class SnapshottingPersistentActor(name: String, probe: ActorRef) extends ExampleProcessor(name) {
override def receiveRecover = super.receiveRecover orElse {
case SnapshotOffer(_, events: List[_])
probe ! "offered"
@ -141,7 +143,7 @@ object EventsourcedSpec {
}
}
class SnapshottingBecomingEventsourcedProcessor(name: String, probe: ActorRef) extends SnapshottingEventsourcedProcessor(name, probe) {
class SnapshottingBecomingPersistentActor(name: String, probe: ActorRef) extends SnapshottingPersistentActor(name, probe) {
val becomingRecover: Receive = {
case msg: SnapshotOffer
context.become(becomingCommand)
@ -160,17 +162,17 @@ object EventsourcedSpec {
class ReplyInEventHandlerProcessor(name: String) extends ExampleProcessor(name) {
val receiveCommand: Receive = {
case Cmd("a") persist(Evt("a"))(evt sender ! evt.data)
case p: Persistent sender ! p // not expected
case Cmd("a") persist(Evt("a"))(evt sender() ! evt.data)
case p: Persistent sender() ! p // not expected
}
}
class UserStashProcessor(name: String) extends ExampleProcessor(name) {
var stashed = false
val receiveCommand: Receive = {
case Cmd("a") if (!stashed) { stash(); stashed = true } else sender ! "a"
case Cmd("b") persist(Evt("b"))(evt sender ! evt.data)
case Cmd("c") unstashAll(); sender ! "c"
case Cmd("a") if (!stashed) { stash(); stashed = true } else sender() ! "a"
case Cmd("b") persist(Evt("b"))(evt sender() ! evt.data)
case Cmd("c") unstashAll(); sender() ! "c"
}
}
@ -194,6 +196,110 @@ object EventsourcedSpec {
case other stash()
}
}
class AsyncPersistProcessor(name: String) extends ExampleProcessor(name) {
var counter = 0
val receiveCommand: Receive = commonBehavior orElse {
case Cmd(data)
sender() ! data
persistAsync(Evt(s"$data-${incCounter()}")) { evt
sender() ! evt.data
}
}
private def incCounter(): Int = {
counter += 1
counter
}
}
class AsyncPersistThreeTimesProcessor(name: String) extends ExampleProcessor(name) {
var counter = 0
val receiveCommand: Receive = commonBehavior orElse {
case Cmd(data)
sender() ! data
1 to 3 foreach { i
persistAsync(Evt(s"$data-${incCounter()}")) { evt
sender() ! ("a" + evt.data.toString.drop(1)) // c-1 => a-1, as in "ack"
}
}
}
private def incCounter(): Int = {
counter += 1
counter
}
}
class AsyncPersistSameEventTwiceProcessor(name: String) extends ExampleProcessor(name) {
// atomic because used from inside the *async* callbacks
val sendMsgCounter = new AtomicInteger()
val receiveCommand: Receive = commonBehavior orElse {
case Cmd(data)
sender() ! data
val event = Evt(data)
persistAsync(event) { evt
// be way slower, in order to be overtaken by the other callback
Thread.sleep(300)
sender() ! s"${evt.data}-a-${sendMsgCounter.incrementAndGet()}"
}
persistAsync(event) { evt sender() ! s"${evt.data}-b-${sendMsgCounter.incrementAndGet()}" }
}
}
class AsyncPersistAndPersistMixedSyncAsyncSyncProcessor(name: String) extends ExampleProcessor(name) {
var counter = 0
val receiveCommand: Receive = commonBehavior orElse {
case Cmd(data)
sender() ! data
persist(Evt(data + "-e1")) { evt
sender() ! s"${evt.data}-${incCounter()}"
}
// this should be happily executed
persistAsync(Evt(data + "-ea2")) { evt
sender() ! s"${evt.data}-${incCounter()}"
}
persist(Evt(data + "-e3")) { evt
sender() ! s"${evt.data}-${incCounter()}"
}
}
private def incCounter(): Int = {
counter += 1
counter
}
}
class AsyncPersistAndPersistMixedSyncAsyncProcessor(name: String) extends ExampleProcessor(name) {
var sendMsgCounter = 0
val start = System.currentTimeMillis()
def time = s" ${System.currentTimeMillis() - start}ms"
val receiveCommand: Receive = commonBehavior orElse {
case Cmd(data)
sender() ! data
persist(Evt(data + "-e1")) { evt
sender() ! s"${evt.data}-${incCounter()}"
}
persistAsync(Evt(data + "-ea2")) { evt
sender() ! s"${evt.data}-${incCounter()}"
}
}
def incCounter() = {
sendMsgCounter += 1
sendMsgCounter
}
}
class UserStashFailureProcessor(name: String) extends ExampleProcessor(name) {
val receiveCommand: Receive = commonBehavior orElse {
@ -218,13 +324,13 @@ object EventsourcedSpec {
class AnyValEventProcessor(name: String) extends ExampleProcessor(name) {
val receiveCommand: Receive = {
case Cmd("a") persist(5)(evt sender ! evt)
case Cmd("a") persist(5)(evt sender() ! evt)
}
}
}
abstract class EventsourcedSpec(config: Config) extends AkkaSpec(config) with PersistenceSpec with ImplicitSender {
import EventsourcedSpec._
abstract class PersistentActorSpec(config: Config) extends AkkaSpec(config) with PersistenceSpec with ImplicitSender {
import PersistentActorSpec._
override protected def beforeEach() {
super.beforeEach()
@ -235,7 +341,7 @@ abstract class EventsourcedSpec(config: Config) extends AkkaSpec(config) with Pe
expectMsg(List("a-1", "a-2"))
}
"An eventsourced processor" must {
"A persistent actor" must {
"recover from persisted events" in {
val processor = namedProcessor[Behavior1Processor]
processor ! GetState
@ -306,7 +412,7 @@ abstract class EventsourcedSpec(config: Config) extends AkkaSpec(config) with Pe
expectMsg(List("a-1", "a-2", "b-0", "c-30", "c-31", "c-32", "d-0", "e-30", "e-31", "e-32"))
}
"support snapshotting" in {
val processor1 = system.actorOf(Props(classOf[SnapshottingEventsourcedProcessor], name, testActor))
val processor1 = system.actorOf(Props(classOf[SnapshottingPersistentActor], name, testActor))
processor1 ! Cmd("b")
processor1 ! "snap"
processor1 ! Cmd("c")
@ -314,13 +420,13 @@ abstract class EventsourcedSpec(config: Config) extends AkkaSpec(config) with Pe
processor1 ! GetState
expectMsg(List("a-1", "a-2", "b-41", "b-42", "c-41", "c-42"))
val processor2 = system.actorOf(Props(classOf[SnapshottingEventsourcedProcessor], name, testActor))
val processor2 = system.actorOf(Props(classOf[SnapshottingPersistentActor], name, testActor))
expectMsg("offered")
processor2 ! GetState
expectMsg(List("a-1", "a-2", "b-41", "b-42", "c-41", "c-42"))
}
"support context.become during recovery" in {
val processor1 = system.actorOf(Props(classOf[SnapshottingEventsourcedProcessor], name, testActor))
val processor1 = system.actorOf(Props(classOf[SnapshottingPersistentActor], name, testActor))
processor1 ! Cmd("b")
processor1 ! "snap"
processor1 ! Cmd("c")
@ -328,14 +434,14 @@ abstract class EventsourcedSpec(config: Config) extends AkkaSpec(config) with Pe
processor1 ! GetState
expectMsg(List("a-1", "a-2", "b-41", "b-42", "c-41", "c-42"))
val processor2 = system.actorOf(Props(classOf[SnapshottingBecomingEventsourcedProcessor], name, testActor))
val processor2 = system.actorOf(Props(classOf[SnapshottingBecomingPersistentActor], name, testActor))
expectMsg("offered")
expectMsg("I am becoming")
processor2 ! GetState
expectMsg(List("a-1", "a-2", "b-41", "b-42", "c-41", "c-42"))
}
"support confirmable persistent" in {
val processor1 = system.actorOf(Props(classOf[SnapshottingEventsourcedProcessor], name, testActor))
val processor1 = system.actorOf(Props(classOf[SnapshottingPersistentActor], name, testActor))
processor1 ! Cmd("b")
processor1 ! "snap"
processor1 ! ConfirmablePersistentImpl(Cmd("c"), 4711, "some-id", false, 0, Seq.empty, null, null, null)
@ -343,7 +449,7 @@ abstract class EventsourcedSpec(config: Config) extends AkkaSpec(config) with Pe
processor1 ! GetState
expectMsg(List("a-1", "a-2", "b-41", "b-42", "c-41", "c-42"))
val processor2 = system.actorOf(Props(classOf[SnapshottingEventsourcedProcessor], name, testActor))
val processor2 = system.actorOf(Props(classOf[SnapshottingPersistentActor], name, testActor))
expectMsg("offered")
processor2 ! GetState
expectMsg(List("a-1", "a-2", "b-41", "b-42", "c-41", "c-42"))
@ -356,9 +462,9 @@ abstract class EventsourcedSpec(config: Config) extends AkkaSpec(config) with Pe
processor.tell(Persistent("not allowed"), probe.ref)
}
processor.tell(Cmd("a"), probe.ref)
processor.tell(Cmd("a"), probe.ref)
processor.tell(Cmd("a"), probe.ref)
processor.tell(Cmd("w"), probe.ref)
processor.tell(Cmd("w"), probe.ref)
processor.tell(Cmd("w"), probe.ref)
EventFilter[UnsupportedOperationException](occurrences = 1) intercept {
processor.tell(Persistent("not allowed when persisting"), probe.ref)
}
@ -401,8 +507,105 @@ abstract class EventsourcedSpec(config: Config) extends AkkaSpec(config) with Pe
processor ! Cmd("a")
expectMsg(5)
}
"be able to opt-out from stashing messages until all events have been processed" in {
val processor = namedProcessor[AsyncPersistProcessor]
processor ! Cmd("x")
processor ! Cmd("y")
expectMsg("x")
expectMsg("y") // "y" command was processed before event persisted
expectMsg("x-1")
expectMsg("y-2")
}
"support multiple persistAsync calls for one command, and execute them 'when possible', not hindering command processing" in {
val processor = namedProcessor[AsyncPersistThreeTimesProcessor]
val commands = 1 to 10 map { i Cmd(s"c-$i") }
commands foreach { i
Thread.sleep(Random.nextInt(10))
processor ! i
}
val all: Seq[String] = this.receiveN(40).asInstanceOf[Seq[String]] // each command = 1 reply + 3 event-replies
val replies = all.filter(r r.count(_ == '-') == 1)
replies should equal(commands.map(_.data))
val expectedAcks = (3 to 32) map { i s"a-${i / 3}-${i - 2}" }
val acks = all.filter(r r.count(_ == '-') == 2)
acks should equal(expectedAcks)
}
"reply to the original sender() of a command, even when using persistAsync" in {
// sanity check, the setting of sender() for PersistentRepl is handled by Processor currently
// but as we want to remove it soon, keeping the explicit test here.
val processor = namedProcessor[AsyncPersistThreeTimesProcessor]
val commands = 1 to 10 map { i Cmd(s"c-$i") }
val probes = Vector.fill(10)(TestProbe())
(probes zip commands) foreach {
case (p, c)
processor.tell(c, p.ref)
}
val ackClass = classOf[String]
within(3.seconds) {
probes foreach { _.expectMsgAllClassOf(ackClass, ackClass, ackClass) }
}
}
"support the same event being asyncPersist'ed multiple times" in {
val processor = namedProcessor[AsyncPersistSameEventTwiceProcessor]
processor ! Cmd("x")
expectMsg("x")
expectMsg("x-a-1")
expectMsg("x-b-2")
expectNoMsg(100.millis)
}
"support a mix of persist calls (sync, async, sync) and persist calls in expected order" in {
val processor = namedProcessor[AsyncPersistAndPersistMixedSyncAsyncSyncProcessor]
processor ! Cmd("a")
processor ! Cmd("b")
processor ! Cmd("c")
expectMsg("a")
expectMsg("a-e1-1") // persist
expectMsg("a-ea2-2") // persistAsync, but ordering enforced by sync persist below
expectMsg("a-e3-3") // persist
expectMsg("b")
expectMsg("b-e1-4")
expectMsg("b-ea2-5")
expectMsg("b-e3-6")
expectMsg("c")
expectMsg("c-e1-7")
expectMsg("c-ea2-8")
expectMsg("c-e3-9")
expectNoMsg(100.millis)
}
"support a mix of persist calls (sync, async) and persist calls" in {
val processor = namedProcessor[AsyncPersistAndPersistMixedSyncAsyncProcessor]
processor ! Cmd("a")
processor ! Cmd("b")
processor ! Cmd("c")
expectMsg("a")
expectMsg("a-e1-1") // persist, must be before next command
var expectInAnyOrder1 = Set("b", "a-ea2-2")
expectInAnyOrder1 -= expectMsgAnyOf(expectInAnyOrder1.toList: _*) // ea2 is persistAsync, b (command) can processed before it
expectMsgAnyOf(expectInAnyOrder1.toList: _*)
expectMsg("b-e1-3") // persist, must be before next command
var expectInAnyOrder2 = Set("c", "b-ea2-4")
expectInAnyOrder2 -= expectMsgAnyOf(expectInAnyOrder2.toList: _*) // ea2 is persistAsync, b (command) can processed before it
expectMsgAnyOf(expectInAnyOrder2.toList: _*)
expectMsg("c-e1-5")
expectMsg("c-ea2-6")
expectNoMsg(100.millis)
}
}
}
class LeveldbEventsourcedSpec extends EventsourcedSpec(PersistenceSpec.config("leveldb", "LeveldbEventsourcedSpec"))
class InmemEventsourcedSpec extends EventsourcedSpec(PersistenceSpec.config("inmem", "InmemEventsourcedSpec"))
class LeveldbPersistentActorSpec extends PersistentActorSpec(PersistenceSpec.config("leveldb", "LeveldbEventsourcedSpec"))
class InmemPersistentActorSpec extends PersistentActorSpec(PersistenceSpec.config("inmem", "InmemEventsourcedSpec"))

View file

@ -47,7 +47,7 @@ object ProcessorChannelSpec {
}
}
class ResendingEventsourcedProcessor(name: String, channelProps: Props, destination: ActorRef) extends NamedProcessor(name) with EventsourcedProcessor {
class ResendingPersistentActor(name: String, channelProps: Props, destination: ActorRef) extends NamedProcessor(name) with PersistentActor {
val channel = context.actorOf(channelProps)
var events: List[String] = Nil
@ -167,10 +167,10 @@ abstract class ProcessorChannelSpec(config: Config) extends AkkaSpec(config) wit
}
}
"An eventsourced processor that uses a channel" can {
"A persistent actor that uses a channel" can {
"reliably deliver events" in {
val probe = TestProbe()
val ep = system.actorOf(Props(classOf[ResendingEventsourcedProcessor], "rep", testResendingChannelProps, probe.ref))
val ep = system.actorOf(Props(classOf[ResendingPersistentActor], "rep", testResendingChannelProps, probe.ref))
ep ! "cmd"

View file

@ -4,13 +4,13 @@
package sample.persistence;
//#eventsourced-example
//#persistent-actor-example
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.japi.pf.ReceiveBuilder;
import akka.persistence.AbstractEventsourcedProcessor;
import akka.persistence.AbstractPersistentActor;
import akka.persistence.SnapshotOffer;
import scala.PartialFunction;
import scala.runtime.BoxedUnit;
@ -73,7 +73,7 @@ class ExampleState implements Serializable {
}
}
class ExampleProcessor extends AbstractEventsourcedProcessor {
class ExampleProcessor extends AbstractPersistentActor {
private ExampleState state = new ExampleState();
public int getNumEvents() {
@ -104,9 +104,9 @@ class ExampleProcessor extends AbstractEventsourcedProcessor {
match(String.class, s -> s.equals("print"), s -> System.out.println(state)).build();
}
}
//#eventsourced-example
//#persistent-actor-example
public class EventsourcedExample {
public class PersistentActorExample {
public static void main(String... args) throws Exception {
final ActorSystem system = ActorSystem.create("example");
final ActorRef processor = system.actorOf(Props.create(ExampleProcessor.class), "processor-4-java8");

View file

@ -11,8 +11,6 @@ import akka.japi.pf.ReceiveBuilder;
import akka.persistence.AbstractProcessor;
import akka.persistence.Persistent;
import scala.Option;
import scala.PartialFunction;
import scala.runtime.BoxedUnit;
import java.util.ArrayList;

View file

@ -72,7 +72,7 @@ current processor state to <code>stdout</code>.
<p>
To run this example, go to the <a href="#run" class="shortcut">Run</a> tab, and run the application main class
<b><code>sample.persistence.EventsourcedExample</code></b> several times.
<b><code>sample.persistence.PersistentActorExample</code></b> several times.
</p>
</div>

View file

@ -1,13 +1,16 @@
package sample.persistence;
//#eventsourced-example
//#persistent-actor-example
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.japi.Procedure;
import akka.persistence.SnapshotOffer;
import akka.persistence.UntypedPersistentActor;
import java.io.Serializable;
import java.util.ArrayList;
import akka.actor.*;
import akka.japi.Procedure;
import akka.persistence.*;
import static java.util.Arrays.asList;
class Cmd implements Serializable {
@ -63,7 +66,7 @@ class ExampleState implements Serializable {
}
}
class ExampleProcessor extends UntypedEventsourcedProcessor {
class ExampleProcessor extends UntypedPersistentActor {
private ExampleState state = new ExampleState();
public int getNumEvents() {
@ -100,9 +103,9 @@ class ExampleProcessor extends UntypedEventsourcedProcessor {
}
}
}
//#eventsourced-example
//#persistent-actor-example
public class EventsourcedExample {
public class PersistentActorExample {
public static void main(String... args) throws Exception {
final ActorSystem system = ActorSystem.create("example");
final ActorRef processor = system.actorOf(Props.create(ExampleProcessor.class), "processor-4-java");

View file

@ -72,7 +72,7 @@ current processor state to <code>stdout</code>.
<p>
To run this example, go to the <a href="#run" class="shortcut">Run</a> tab, and run the application main class
<b><code>sample.persistence.EventsourcedExample</code></b> several times.
<b><code>sample.persistence.PersistentActorExample</code></b> several times.
</p>
</div>

View file

@ -1,6 +1,6 @@
package sample.persistence
//#eventsourced-example
//#persistent-actor-example
import akka.actor._
import akka.persistence._
@ -13,7 +13,7 @@ final case class ExampleState(events: List[String] = Nil) {
override def toString: String = events.reverse.toString
}
class ExampleProcessor extends EventsourcedProcessor {
class ExampleProcessor extends PersistentActor {
var state = ExampleState()
def updateState(event: Evt): Unit =
@ -39,9 +39,9 @@ class ExampleProcessor extends EventsourcedProcessor {
}
}
//#eventsourced-example
//#persistent-actor-example
object EventsourcedExample extends App {
object PersistentActorExample extends App {
val system = ActorSystem("example")
val processor = system.actorOf(Props[ExampleProcessor], "processor-4-scala")

View file

@ -90,7 +90,7 @@ current processor state to <code>stdout</code>.
<p>
To run this example, go to the <a href="#run" class="shortcut">Run</a> tab, and run the application main class
<b><code>sample.persistence.EventsourcedExample</code></b> several times.
<b><code>sample.persistence.PersistentActorExample</code></b> several times.
</p>
</div>

View file

@ -478,7 +478,6 @@ object AkkaBuild extends Build {
validatePullRequest <<= validatePullRequest.dependsOn(test in Test),
// add reportBinaryIssues to validatePullRequest on minor version maintenance branch
validatePullRequest <<= validatePullRequest.dependsOn(reportBinaryIssues)
) ++
mavenLocalResolverSettings ++
JUnitFileReporting.settings ++ StatsDMetrics.settings
@ -488,9 +487,9 @@ object AkkaBuild extends Build {
val validatePullRequestTask = validatePullRequest := ()
lazy val mimaIgnoredProblems = {
import com.typesafe.tools.mima.core._
Seq(
// add filters here, see release-2.2 branch
import com.typesafe.tools.mima.core._
Seq(
// add filters here, see release-2.2 branch
)
}