per #15423 Remove deprecated features from akka-persistence
* remove channels * remove View * remove Processor * collapse the complicated internal state management that was spread out between Processor, Eventsourced and Recovery * remove Recovery trait, this caused some duplication between Eventsourced and PersistentView, but but the enhanced PersistentView will not be based on recovery infrastructure, and therefore PersistentView code will be replaced anyway * remove PersistentBatch * remove LoopMessage * remove deleteMessages of individual messages * remove Persistent, PersistentRepr and PersistentImpl are kept * remove processorId * update doc sample code * note in migration guide about persistenceId * rename Resequencable to PersistentEnvelope
This commit is contained in:
parent
86a5b3d9d7
commit
c566d5a106
84 changed files with 2162 additions and 9560 deletions
|
|
@ -24,7 +24,7 @@ public class PersistenceDocTest {
|
|||
|
||||
public interface SomeOtherMessage {}
|
||||
|
||||
public interface ProcessorMethods {
|
||||
public interface PersistentActorMethods {
|
||||
//#persistence-id
|
||||
public String persistenceId();
|
||||
//#persistence-id
|
||||
|
|
@ -32,64 +32,25 @@ public class PersistenceDocTest {
|
|||
public boolean recoveryRunning();
|
||||
public boolean recoveryFinished();
|
||||
//#recovery-status
|
||||
//#current-message
|
||||
public Persistent getCurrentPersistentMessage();
|
||||
//#current-message
|
||||
}
|
||||
|
||||
static Object o1 = new Object() {
|
||||
//#definition
|
||||
class MyProcessor extends UntypedProcessor {
|
||||
public void onReceive(Object message) throws Exception {
|
||||
if (message instanceof Persistent) {
|
||||
// message successfully written to journal
|
||||
Persistent persistent = (Persistent)message;
|
||||
Object payload = persistent.payload();
|
||||
Long sequenceNr = persistent.sequenceNr();
|
||||
// ...
|
||||
} else if (message instanceof PersistenceFailure) {
|
||||
// message failed to be written to journal
|
||||
PersistenceFailure failure = (PersistenceFailure)message;
|
||||
Object payload = failure.payload();
|
||||
Long sequenceNr = failure.sequenceNr();
|
||||
Throwable cause = failure.cause();
|
||||
// ...
|
||||
} else if (message instanceof SomeOtherMessage) {
|
||||
// message not written to journal
|
||||
}
|
||||
else {
|
||||
unhandled(message);
|
||||
}
|
||||
}
|
||||
}
|
||||
//#definition
|
||||
|
||||
class MyActor extends UntypedActor {
|
||||
ActorRef processor;
|
||||
|
||||
public MyActor() {
|
||||
//#usage
|
||||
processor = getContext().actorOf(Props.create(MyProcessor.class), "myProcessor");
|
||||
|
||||
processor.tell(Persistent.create("foo"), getSelf());
|
||||
processor.tell("bar", getSelf());
|
||||
//#usage
|
||||
}
|
||||
ActorRef persistentActor;
|
||||
|
||||
public void onReceive(Object message) throws Exception {
|
||||
// ...
|
||||
}
|
||||
|
||||
private void recover() {
|
||||
//#recover-explicit
|
||||
processor.tell(Recover.create(), getSelf());
|
||||
persistentActor.tell(Recover.create(), getSelf());
|
||||
//#recover-explicit
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
static Object o2 = new Object() {
|
||||
abstract class MyProcessor1 extends UntypedPersistentActor {
|
||||
abstract class MyPersistentActor1 extends UntypedPersistentActor {
|
||||
//#recover-on-start-disabled
|
||||
@Override
|
||||
public void preStart() {}
|
||||
|
|
@ -101,7 +62,7 @@ public class PersistenceDocTest {
|
|||
//#recover-on-restart-disabled
|
||||
}
|
||||
|
||||
abstract class MyProcessor2 extends UntypedPersistentActor {
|
||||
abstract class MyPersistentActor2 extends UntypedPersistentActor {
|
||||
//#recover-on-start-custom
|
||||
@Override
|
||||
public void preStart() {
|
||||
|
|
@ -110,19 +71,7 @@ public class PersistenceDocTest {
|
|||
//#recover-on-start-custom
|
||||
}
|
||||
|
||||
abstract class MyProcessor3 extends UntypedPersistentActor {
|
||||
//#deletion
|
||||
@Override
|
||||
public void preRestart(Throwable reason, Option<Object> message) {
|
||||
if (message.isDefined() && message.get() instanceof Persistent) {
|
||||
deleteMessage(((Persistent) message.get()).sequenceNr());
|
||||
}
|
||||
super.preRestart(reason, message);
|
||||
}
|
||||
//#deletion
|
||||
}
|
||||
|
||||
class MyProcessor4 extends UntypedPersistentActor implements ProcessorMethods {
|
||||
class MyPersistentActor4 extends UntypedPersistentActor implements PersistentActorMethods {
|
||||
//#persistence-id-override
|
||||
@Override
|
||||
public String persistenceId() {
|
||||
|
|
@ -135,40 +84,34 @@ public class PersistenceDocTest {
|
|||
public void onReceiveCommand(Object message) throws Exception {}
|
||||
}
|
||||
|
||||
class MyProcessor5 extends UntypedPersistentActor {
|
||||
@Override
|
||||
public String persistenceId() { return "persistence-id"; }
|
||||
|
||||
//#recovery-completed
|
||||
|
||||
@Override
|
||||
public void onReceiveRecover(Object message) {
|
||||
if (message instanceof RecoveryCompleted) {
|
||||
recoveryCompleted();
|
||||
}
|
||||
class MyPersistentActor5 extends UntypedPersistentActor {
|
||||
@Override
|
||||
public String persistenceId() {
|
||||
return "persistence-id";
|
||||
}
|
||||
|
||||
//#recovery-completed
|
||||
@Override
|
||||
public void onReceiveRecover(Object message) {
|
||||
if (message instanceof RecoveryCompleted) {
|
||||
// perform init after recovery, before any other messages
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReceiveCommand(Object message) throws Exception {
|
||||
if (message instanceof String) {
|
||||
// ...
|
||||
} else {
|
||||
unhandled(message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReceiveCommand(Object message) throws Exception {
|
||||
if (message instanceof String) {
|
||||
// ...
|
||||
} else {
|
||||
unhandled(message);
|
||||
}
|
||||
}
|
||||
|
||||
private void recoveryCompleted() {
|
||||
// perform init after recovery, before any other messages
|
||||
// ...
|
||||
}
|
||||
|
||||
//#recovery-completed
|
||||
}
|
||||
//#recovery-completed
|
||||
}
|
||||
};
|
||||
|
||||
static Object fullyDisabledRecoveyExample = new Object() {
|
||||
abstract class MyProcessor1 extends UntypedPersistentActor {
|
||||
static Object fullyDisabledRecoveryExample = new Object() {
|
||||
abstract class MyPersistentActor1 extends UntypedPersistentActor {
|
||||
//#recover-fully-disabled
|
||||
@Override
|
||||
public void preStart() { getSelf().tell(Recover.create(0L), getSelf()); }
|
||||
|
|
@ -215,11 +158,15 @@ public class PersistenceDocTest {
|
|||
|
||||
class MyPersistentActor extends UntypedPersistentActorWithAtLeastOnceDelivery {
|
||||
private final ActorPath destination;
|
||||
|
||||
@Override
|
||||
public String persistenceId() { return "persistence-id"; }
|
||||
|
||||
public MyPersistentActor(ActorPath destination) {
|
||||
this.destination = destination;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReceiveCommand(Object message) {
|
||||
if (message instanceof String) {
|
||||
String s = (String) message;
|
||||
|
|
@ -240,6 +187,7 @@ public class PersistenceDocTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReceiveRecover(Object event) {
|
||||
updateState(event);
|
||||
}
|
||||
|
|
@ -273,97 +221,16 @@ public class PersistenceDocTest {
|
|||
//#at-least-once-example
|
||||
};
|
||||
|
||||
static Object o3 = new Object() {
|
||||
//#channel-example
|
||||
class MyProcessor extends UntypedProcessor {
|
||||
private final ActorRef destination;
|
||||
private final ActorRef channel;
|
||||
|
||||
public MyProcessor() {
|
||||
this.destination = getContext().actorOf(Props.create(MyDestination.class));
|
||||
this.channel = getContext().actorOf(Channel.props(), "myChannel");
|
||||
}
|
||||
|
||||
public void onReceive(Object message) throws Exception {
|
||||
if (message instanceof Persistent) {
|
||||
Persistent p = (Persistent)message;
|
||||
Persistent out = p.withPayload("done " + p.payload());
|
||||
channel.tell(Deliver.create(out, destination.path()), getSelf());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class MyDestination extends UntypedActor {
|
||||
public void onReceive(Object message) throws Exception {
|
||||
if (message instanceof ConfirmablePersistent) {
|
||||
ConfirmablePersistent p = (ConfirmablePersistent)message;
|
||||
Object payload = p.payload();
|
||||
Long sequenceNr = p.sequenceNr();
|
||||
int redeliveries = p.redeliveries();
|
||||
// ...
|
||||
p.confirm();
|
||||
}
|
||||
}
|
||||
}
|
||||
//#channel-example
|
||||
|
||||
class MyProcessor2 extends UntypedProcessor {
|
||||
private final ActorRef destination;
|
||||
private final ActorRef channel;
|
||||
|
||||
public MyProcessor2(ActorRef destination) {
|
||||
this.destination = getContext().actorOf(Props.create(MyDestination.class));
|
||||
//#channel-id-override
|
||||
this.channel = getContext().actorOf(Channel.props("my-stable-channel-id"));
|
||||
//#channel-id-override
|
||||
|
||||
//#channel-custom-settings
|
||||
getContext().actorOf(Channel.props(
|
||||
ChannelSettings.create()
|
||||
.withRedeliverInterval(Duration.create(30, TimeUnit.SECONDS))
|
||||
.withRedeliverMax(15)));
|
||||
//#channel-custom-settings
|
||||
|
||||
//#channel-custom-listener
|
||||
class MyListener extends UntypedActor {
|
||||
@Override
|
||||
public void onReceive(Object message) throws Exception {
|
||||
if (message instanceof RedeliverFailure) {
|
||||
Iterable<ConfirmablePersistent> messages =
|
||||
((RedeliverFailure)message).getMessages();
|
||||
// ...
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final ActorRef myListener = getContext().actorOf(Props.create(MyListener.class));
|
||||
getContext().actorOf(Channel.props(
|
||||
ChannelSettings.create().withRedeliverFailureListener(null)));
|
||||
//#channel-custom-listener
|
||||
|
||||
}
|
||||
|
||||
public void onReceive(Object message) throws Exception {
|
||||
if (message instanceof Persistent) {
|
||||
Persistent p = (Persistent)message;
|
||||
Persistent out = p.withPayload("done " + p.payload());
|
||||
channel.tell(Deliver.create(out, destination.path()), getSelf());
|
||||
|
||||
//#channel-example-reply
|
||||
channel.tell(Deliver.create(out, getSender().path()), getSelf());
|
||||
//#channel-example-reply
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
static Object o4 = new Object() {
|
||||
//#save-snapshot
|
||||
class MyProcessor extends UntypedProcessor {
|
||||
class MyPersistentActor extends UntypedPersistentActor {
|
||||
@Override
|
||||
public String persistenceId() { return "persistence-id"; }
|
||||
|
||||
//#save-snapshot
|
||||
private Object state;
|
||||
|
||||
@Override
|
||||
public void onReceive(Object message) throws Exception {
|
||||
public void onReceiveCommand(Object message) {
|
||||
if (message.equals("snap")) {
|
||||
saveSnapshot(state);
|
||||
} else if (message instanceof SaveSnapshotSuccess) {
|
||||
|
|
@ -374,32 +241,46 @@ public class PersistenceDocTest {
|
|||
// ...
|
||||
}
|
||||
}
|
||||
//#save-snapshot
|
||||
|
||||
@Override
|
||||
public void onReceiveRecover(Object event) {
|
||||
}
|
||||
}
|
||||
//#save-snapshot
|
||||
|
||||
};
|
||||
|
||||
static Object o5 = new Object() {
|
||||
//#snapshot-offer
|
||||
class MyProcessor extends UntypedProcessor {
|
||||
class MyPersistentActor extends UntypedPersistentActor {
|
||||
@Override
|
||||
public String persistenceId() { return "persistence-id"; }
|
||||
|
||||
//#snapshot-offer
|
||||
private Object state;
|
||||
|
||||
@Override
|
||||
public void onReceive(Object message) throws Exception {
|
||||
public void onReceiveRecover(Object message) {
|
||||
if (message instanceof SnapshotOffer) {
|
||||
state = ((SnapshotOffer)message).snapshot();
|
||||
// ...
|
||||
} else if (message instanceof Persistent) {
|
||||
} else if (message instanceof RecoveryCompleted) {
|
||||
// ...
|
||||
} else {
|
||||
// ...
|
||||
}
|
||||
}
|
||||
//#snapshot-offer
|
||||
|
||||
@Override
|
||||
public void onReceiveCommand(Object message) {
|
||||
}
|
||||
}
|
||||
//#snapshot-offer
|
||||
|
||||
class MyActor extends UntypedActor {
|
||||
ActorRef processor;
|
||||
ActorRef persistentActor;
|
||||
|
||||
public MyActor() {
|
||||
processor = getContext().actorOf(Props.create(MyProcessor.class));
|
||||
persistentActor = getContext().actorOf(Props.create(MyPersistentActor.class));
|
||||
}
|
||||
|
||||
public void onReceive(Object message) throws Exception {
|
||||
|
|
@ -408,105 +289,13 @@ public class PersistenceDocTest {
|
|||
|
||||
private void recover() {
|
||||
//#snapshot-criteria
|
||||
processor.tell(Recover.create(SnapshotSelectionCriteria.create(457L, System.currentTimeMillis())), null);
|
||||
persistentActor.tell(Recover.create(SnapshotSelectionCriteria.create(457L,
|
||||
System.currentTimeMillis())), null);
|
||||
//#snapshot-criteria
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
static Object o6 = new Object() {
|
||||
//#batch-write
|
||||
class MyProcessor extends UntypedProcessor {
|
||||
public void onReceive(Object message) throws Exception {
|
||||
if (message instanceof Persistent) {
|
||||
Persistent p = (Persistent)message;
|
||||
if (p.payload().equals("a")) { /* ... */ }
|
||||
if (p.payload().equals("b")) { /* ... */ }
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class Example {
|
||||
final ActorSystem system = ActorSystem.create("example");
|
||||
final ActorRef processor = system.actorOf(Props.create(MyProcessor.class));
|
||||
|
||||
public void batchWrite() {
|
||||
processor.tell(PersistentBatch.create(asList(
|
||||
Persistent.create("a"),
|
||||
Persistent.create("b"))), null);
|
||||
}
|
||||
|
||||
// ...
|
||||
}
|
||||
//#batch-write
|
||||
};
|
||||
|
||||
static Object o7 = new Object() {
|
||||
abstract class MyProcessor extends UntypedProcessor {
|
||||
ActorRef destination;
|
||||
|
||||
public void foo() {
|
||||
//#persistent-channel-example
|
||||
final ActorRef channel = getContext().actorOf(PersistentChannel.props(
|
||||
PersistentChannelSettings.create()
|
||||
.withRedeliverInterval(Duration.create(30, TimeUnit.SECONDS))
|
||||
.withRedeliverMax(15)), "myPersistentChannel");
|
||||
|
||||
channel.tell(Deliver.create(Persistent.create("example"), destination.path()), getSelf());
|
||||
//#persistent-channel-example
|
||||
//#persistent-channel-watermarks
|
||||
PersistentChannelSettings.create()
|
||||
.withPendingConfirmationsMax(10000)
|
||||
.withPendingConfirmationsMin(2000);
|
||||
//#persistent-channel-watermarks
|
||||
//#persistent-channel-reply
|
||||
PersistentChannelSettings.create().withReplyPersistent(true);
|
||||
//#persistent-channel-reply
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
static Object o8 = new Object() {
|
||||
//#reliable-event-delivery
|
||||
class MyPersistentActor extends UntypedPersistentActor {
|
||||
@Override
|
||||
public String persistenceId() { return "some-persistence-id"; }
|
||||
|
||||
private ActorRef destination;
|
||||
private ActorRef channel;
|
||||
|
||||
public MyPersistentActor(ActorRef destination) {
|
||||
this.destination = destination;
|
||||
this.channel = getContext().actorOf(Channel.props(), "channel");
|
||||
}
|
||||
|
||||
private void handleEvent(String event) {
|
||||
// update state
|
||||
// ...
|
||||
// reliably deliver events
|
||||
channel.tell(Deliver.create(Persistent.create(
|
||||
event, getCurrentPersistentMessage()), destination.path()), getSelf());
|
||||
}
|
||||
|
||||
public void onReceiveRecover(Object msg) {
|
||||
if (msg instanceof String) {
|
||||
handleEvent((String)msg);
|
||||
}
|
||||
}
|
||||
|
||||
public void onReceiveCommand(Object msg) {
|
||||
if (msg.equals("cmd")) {
|
||||
persist("evt", new Procedure<String>() {
|
||||
public void apply(String event) {
|
||||
handleEvent(event);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
//#reliable-event-delivery
|
||||
};
|
||||
|
||||
static Object o9 = new Object() {
|
||||
//#persist-async
|
||||
class MyPersistentActor extends UntypedPersistentActor {
|
||||
|
|
@ -541,9 +330,9 @@ public class PersistenceDocTest {
|
|||
public void usage() {
|
||||
final ActorSystem system = ActorSystem.create("example");
|
||||
//#persist-async-usage
|
||||
final ActorRef processor = system.actorOf(Props.create(MyPersistentActor.class));
|
||||
processor.tell("a", null);
|
||||
processor.tell("b", null);
|
||||
final ActorRef persistentActor = system.actorOf(Props.create(MyPersistentActor.class));
|
||||
persistentActor.tell("a", null);
|
||||
persistentActor.tell("b", null);
|
||||
|
||||
// possible order of received messages:
|
||||
// a
|
||||
|
|
@ -586,9 +375,9 @@ public class PersistenceDocTest {
|
|||
public void usage() {
|
||||
final ActorSystem system = ActorSystem.create("example");
|
||||
//#defer-caller
|
||||
final ActorRef processor = system.actorOf(Props.create(MyPersistentActor.class));
|
||||
processor.tell("a", null);
|
||||
processor.tell("b", null);
|
||||
final ActorRef persistentActor = system.actorOf(Props.create(MyPersistentActor.class));
|
||||
persistentActor.tell("a", null);
|
||||
persistentActor.tell("b", null);
|
||||
|
||||
// order of received messages:
|
||||
// a
|
||||
|
|
|
|||
|
|
@ -4,26 +4,25 @@
|
|||
|
||||
package docs.persistence;
|
||||
|
||||
//#plugin-imports
|
||||
|
||||
import akka.actor.*;
|
||||
import akka.japi.Option;
|
||||
import akka.japi.Procedure;
|
||||
import akka.persistence.*;
|
||||
import akka.persistence.japi.journal.JavaJournalSpec;
|
||||
import akka.persistence.japi.snapshot.JavaSnapshotStoreSpec;
|
||||
import akka.persistence.journal.japi.AsyncWriteJournal;
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbJournal;
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbStore;
|
||||
import akka.persistence.snapshot.japi.SnapshotStore;
|
||||
import com.typesafe.config.Config;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
import org.iq80.leveldb.util.FileUtils;
|
||||
import scala.concurrent.Future;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import akka.actor.*;
|
||||
import com.typesafe.config.Config;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
import org.iq80.leveldb.util.FileUtils;
|
||||
import akka.persistence.japi.journal.JavaJournalSpec;
|
||||
import akka.persistence.japi.snapshot.JavaSnapshotStoreSpec;
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbJournal;
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbStore;
|
||||
import scala.concurrent.Future;
|
||||
import akka.japi.Option;
|
||||
import akka.japi.Procedure;
|
||||
|
||||
//#plugin-imports
|
||||
import akka.persistence.*;
|
||||
import akka.persistence.journal.japi.AsyncWriteJournal;
|
||||
import akka.persistence.snapshot.japi.SnapshotStore;
|
||||
|
||||
//#plugin-imports
|
||||
|
||||
|
|
@ -91,23 +90,14 @@ public class PersistencePluginDocTest {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<Void> doAsyncWriteConfirmations(Iterable<PersistentConfirmation> confirmations) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<Void> doAsyncDeleteMessages(Iterable<PersistentId> messageIds, boolean permanent) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<Void> doAsyncDeleteMessagesTo(String persistenceId, long toSequenceNr, boolean permanent) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<Void> doAsyncReplayMessages(String persistenceId, long fromSequenceNr, long toSequenceNr, long max, Procedure<PersistentRepr> replayCallback) {
|
||||
public Future<Void> doAsyncReplayMessages(String persistenceId, long fromSequenceNr,
|
||||
long toSequenceNr, long max, Procedure<PersistentRepr> replayCallback) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -130,6 +130,8 @@ It contains instructions on how to run the ``PersistentActorExample``.
|
|||
with ``context().become()`` and ``context().unbecome()``. To get the actor into the same state after
|
||||
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.
|
||||
Note that when using ``become`` from ``receiveRecover`` it will still only use the ``receiveRecover``
|
||||
behavior when replaying the events. When replay is completed it will use the new behavior.
|
||||
|
||||
Identifiers
|
||||
-----------
|
||||
|
|
@ -159,6 +161,15 @@ In this case, a persistent actor must be recovered explicitly by sending it a ``
|
|||
|
||||
.. includecode:: ../../../akka-samples/akka-sample-persistence-java-lambda/src/main/java/doc/LambdaPersistenceDocTest.java#recover-explicit
|
||||
|
||||
.. warning::
|
||||
|
||||
If ``preStart`` is overriden by an empty implementation, incoming commands will not be processed by the
|
||||
``PersistentActor`` until it receives a ``Recover`` and finishes recovery.
|
||||
|
||||
In order to completely skip recovery, you can signal it with ``Recover.create(0L)``
|
||||
|
||||
.. includecode:: ../../../akka-samples/akka-sample-persistence-java-lambda/src/main/java/doc/LambdaPersistenceDocTest.java#recover-fully-disabled
|
||||
|
||||
If not overridden, ``preStart`` sends a ``Recover`` message to ``self()``. Applications may also override
|
||||
``preStart`` to define further ``Recover`` parameters such as an upper sequence number bound, for example.
|
||||
|
||||
|
|
@ -206,7 +217,7 @@ The ordering between events is still guaranteed ("evt-b-1" will be sent after "e
|
|||
.. includecode:: ../../../akka-samples/akka-sample-persistence-java-lambda/src/main/java/doc/LambdaPersistenceDocTest.java#persist-async
|
||||
|
||||
.. note::
|
||||
In order to implement the pattern known as "*command sourcing*" simply ``persistAsync`` all incoming events right away,
|
||||
In order to implement the pattern known as "*command sourcing*" simply call ``persistAsync`` on all incoming messages right away,
|
||||
and handle them in the callback.
|
||||
|
||||
.. warning::
|
||||
|
|
|
|||
|
|
@ -132,6 +132,8 @@ It contains instructions on how to run the ``PersistentActorExample``.
|
|||
with ``getContext().become()`` and ``getContext().unbecome()``. To get the actor into the same state after
|
||||
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.
|
||||
Note that when using ``become`` from ``receiveRecover`` it will still only use the ``receiveRecover``
|
||||
behavior when replaying the events. When replay is completed it will use the new behavior.
|
||||
|
||||
Identifiers
|
||||
-----------
|
||||
|
|
@ -163,8 +165,9 @@ In this case, a persistent actor must be recovered explicitly by sending it a ``
|
|||
.. includecode:: code/docs/persistence/PersistenceDocTest.java#recover-explicit
|
||||
|
||||
.. warning::
|
||||
If ``preStart`` is overriden by an empty implementation, incoming commands will not be processed by the
|
||||
``PersistentActor`` until it receives a ``Recover`` and finishes recovery.
|
||||
|
||||
If ``preStart`` is overriden by an empty implementation, incoming commands will not be processed by the
|
||||
``PersistentActor`` until it receives a ``Recover`` and finishes recovery.
|
||||
|
||||
In order to completely skip recovery, you can signal it with ``Recover.create(0L)``
|
||||
|
||||
|
|
@ -219,7 +222,7 @@ The ordering between events is still guaranteed ("evt-b-1" will be sent after "e
|
|||
.. 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,
|
||||
In order to implement the pattern known as "*command sourcing*" simply ``persistAsync`` all incoming messages right away,
|
||||
and handle them in the callback.
|
||||
|
||||
.. warning::
|
||||
|
|
|
|||
|
|
@ -147,4 +147,17 @@ Default interval for TestKit.awaitAssert changed to 100 ms
|
|||
Default check interval changed from 800 ms to 100 ms. You can define the interval explicitly if you need a
|
||||
longer interval.
|
||||
|
||||
persistenceId
|
||||
=============
|
||||
|
||||
It is now mandatory to define the ``persistenceId`` in subclasses of ``PersistentActor``, ``UntypedPersistentActor``
|
||||
and ``AbstractPersistentId``.
|
||||
|
||||
The rationale behind this change being stricter de-coupling of your Actor hierarchy and the logical
|
||||
"which persistent entity this actor represents".
|
||||
|
||||
In case you want to perserve the old behavior of providing the actor's path as the default ``persistenceId``, you can easily
|
||||
implement it yourself either as a helper trait or simply by overriding ``persistenceId`` as follows::
|
||||
|
||||
override def persistenceId = self.path.toStringWithoutAddress
|
||||
|
||||
|
|
|
|||
|
|
@ -98,6 +98,20 @@ Using the ``PersistentActor`` instead of ``Processor`` also shifts the responsib
|
|||
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.
|
||||
|
||||
How to migrate data from Processor to PersistentActor
|
||||
-----------------------------------------------------
|
||||
|
||||
The recommended approach for migrating persisted messages from a ``Processor`` to events that can be replayed by
|
||||
a ``PersistentActor`` is to write a custom migration tool with a ``PersistentView`` and a ``PersistentActor``.
|
||||
Connect the ``PersistentView`` to the ``persistenceId`` of the old ``Processor`` to replay the stored persistent
|
||||
messages. Send the messages from the view to a ``PersistentActor`` with another ``persistenceId``. There you can
|
||||
transform the old messages to domain events that the real ``PersistentActor`` will be able to understand. Store
|
||||
the events with ``persistAsync``.
|
||||
|
||||
Note that you can implement back-pressure between the writing ``PersistentActor`` and the reading ``PersistentView``
|
||||
by turning off auto-update in the view and send custom ``Update`` messages to the view with a limited `replayMax`
|
||||
value.
|
||||
|
||||
Removed deleteMessage
|
||||
=====================
|
||||
|
||||
|
|
|
|||
|
|
@ -28,37 +28,7 @@ trait PersistenceDocSpec {
|
|||
import system._
|
||||
|
||||
new AnyRef {
|
||||
//#definition
|
||||
import akka.persistence.{ PersistenceFailure, Persistent, Processor }
|
||||
|
||||
class MyProcessor extends Processor {
|
||||
def receive = {
|
||||
case Persistent(payload, sequenceNr) =>
|
||||
// message successfully written to journal
|
||||
case PersistenceFailure(payload, sequenceNr, cause) =>
|
||||
// message failed to be written to journal
|
||||
case m: SomeOtherMessage =>
|
||||
// message not written to journal
|
||||
}
|
||||
}
|
||||
//#definition
|
||||
|
||||
//#usage
|
||||
import akka.actor.Props
|
||||
|
||||
val processor = actorOf(Props[MyProcessor], name = "myProcessor")
|
||||
|
||||
processor ! Persistent("foo") // will be journaled
|
||||
processor ! "bar" // will not be journaled
|
||||
//#usage
|
||||
|
||||
//#recover-explicit
|
||||
processor ! Recover()
|
||||
//#recover-explicit
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
trait MyProcessor1 extends PersistentActor {
|
||||
trait MyPersistentActor1 extends PersistentActor {
|
||||
//#recover-on-start-disabled
|
||||
override def preStart() = ()
|
||||
//#recover-on-start-disabled
|
||||
|
|
@ -67,7 +37,7 @@ trait PersistenceDocSpec {
|
|||
//#recover-on-restart-disabled
|
||||
}
|
||||
|
||||
trait MyProcessor2 extends PersistentActor {
|
||||
trait MyPersistentActor2 extends PersistentActor {
|
||||
//#recover-on-start-custom
|
||||
override def preStart() {
|
||||
self ! Recover(toSequenceNr = 457L)
|
||||
|
|
@ -75,51 +45,40 @@ trait PersistenceDocSpec {
|
|||
//#recover-on-start-custom
|
||||
}
|
||||
|
||||
trait MyProcessor3 extends PersistentActor {
|
||||
//#deletion
|
||||
override def preRestart(reason: Throwable, message: Option[Any]) {
|
||||
message match {
|
||||
case Some(p: Persistent) => deleteMessage(p.sequenceNr)
|
||||
case _ =>
|
||||
}
|
||||
super.preRestart(reason, message)
|
||||
}
|
||||
//#deletion
|
||||
}
|
||||
val persistentActor = system.deadLetters
|
||||
//#recover-explicit
|
||||
persistentActor ! Recover()
|
||||
//#recover-explicit
|
||||
|
||||
class MyProcessor4 extends PersistentActor {
|
||||
class MyPersistentActor4 extends PersistentActor {
|
||||
override def persistenceId = "my-stable-persistence-id"
|
||||
|
||||
//#recovery-completed
|
||||
|
||||
def receiveRecover: Receive = {
|
||||
case RecoveryCompleted => recoveryCompleted()
|
||||
override def receiveRecover: Receive = {
|
||||
case RecoveryCompleted =>
|
||||
// perform init after recovery, before any other messages
|
||||
//...
|
||||
case evt => //...
|
||||
}
|
||||
|
||||
def receiveCommand: Receive = {
|
||||
override def receiveCommand: Receive = {
|
||||
case msg => //...
|
||||
}
|
||||
|
||||
def recoveryCompleted(): Unit = {
|
||||
// perform init after recovery, before any other messages
|
||||
// ...
|
||||
}
|
||||
|
||||
//#recovery-completed
|
||||
}
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
trait MyProcessor1 extends PersistentActor {
|
||||
//#recover-fully-disabled
|
||||
override def preStart() = self ! Recover(toSequenceNr = 0L)
|
||||
//#recover-fully-disabled
|
||||
}
|
||||
trait MyPersistentActor1 extends PersistentActor {
|
||||
//#recover-fully-disabled
|
||||
override def preStart() = self ! Recover(toSequenceNr = 0L)
|
||||
//#recover-fully-disabled
|
||||
}
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
trait ProcessorMethods {
|
||||
trait PersistentActorMethods {
|
||||
//#persistence-id
|
||||
def persistenceId: String
|
||||
//#persistence-id
|
||||
|
|
@ -127,15 +86,16 @@ trait PersistenceDocSpec {
|
|||
def recoveryRunning: Boolean
|
||||
def recoveryFinished: Boolean
|
||||
//#recovery-status
|
||||
//#current-message
|
||||
implicit def currentPersistentMessage: Option[Persistent]
|
||||
//#current-message
|
||||
}
|
||||
class MyProcessor1 extends Processor with ProcessorMethods {
|
||||
class MyPersistentActor1 extends PersistentActor with PersistentActorMethods {
|
||||
//#persistence-id-override
|
||||
override def persistenceId = "my-stable-persistence-id"
|
||||
//#persistence-id-override
|
||||
def receive = {
|
||||
|
||||
override def receiveRecover: Receive = {
|
||||
case _ =>
|
||||
}
|
||||
override def receiveCommand: Receive = {
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
|
|
@ -156,12 +116,14 @@ trait PersistenceDocSpec {
|
|||
class MyPersistentActor(destination: ActorPath)
|
||||
extends PersistentActor with AtLeastOnceDelivery {
|
||||
|
||||
def receiveCommand: Receive = {
|
||||
override def persistenceId: String = "persistence-id"
|
||||
|
||||
override def receiveCommand: Receive = {
|
||||
case s: String => persist(MsgSent(s))(updateState)
|
||||
case Confirm(deliveryId) => persist(MsgConfirmed(deliveryId))(updateState)
|
||||
}
|
||||
|
||||
def receiveRecover: Receive = {
|
||||
override def receiveRecover: Receive = {
|
||||
case evt: Evt => updateState(evt)
|
||||
}
|
||||
|
||||
|
|
@ -184,221 +146,66 @@ trait PersistenceDocSpec {
|
|||
}
|
||||
|
||||
new AnyRef {
|
||||
//#channel-example
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.persistence.{ Channel, Deliver, Persistent, Processor }
|
||||
|
||||
class MyProcessor extends Processor {
|
||||
val destination = context.actorOf(Props[MyDestination])
|
||||
val channel = context.actorOf(Channel.props(), name = "myChannel")
|
||||
class MyPersistentActor extends PersistentActor {
|
||||
override def persistenceId = "my-stable-persistence-id"
|
||||
|
||||
def receive = {
|
||||
case p @ Persistent(payload, _) =>
|
||||
channel ! Deliver(p.withPayload(s"processed ${payload}"), destination.path)
|
||||
}
|
||||
}
|
||||
|
||||
class MyDestination extends Actor {
|
||||
def receive = {
|
||||
case p @ ConfirmablePersistent(payload, sequenceNr, redeliveries) =>
|
||||
// ...
|
||||
p.confirm()
|
||||
}
|
||||
}
|
||||
//#channel-example
|
||||
|
||||
class MyProcessor2 extends Processor {
|
||||
val destination = context.actorOf(Props[MyDestination])
|
||||
val channel =
|
||||
//#channel-id-override
|
||||
context.actorOf(Channel.props("my-stable-channel-id"))
|
||||
//#channel-id-override
|
||||
|
||||
//#channel-custom-settings
|
||||
context.actorOf(Channel.props(
|
||||
ChannelSettings(redeliverInterval = 30 seconds, redeliverMax = 15)),
|
||||
name = "myChannel")
|
||||
//#channel-custom-settings
|
||||
|
||||
def receive = {
|
||||
case p @ Persistent(payload, _) =>
|
||||
//#channel-example-reply
|
||||
channel ! Deliver(p.withPayload(s"processed ${payload}"), sender().path)
|
||||
//#channel-example-reply
|
||||
}
|
||||
|
||||
//#channel-custom-listener
|
||||
class MyListener extends Actor {
|
||||
def receive = {
|
||||
case RedeliverFailure(messages) => // ...
|
||||
}
|
||||
}
|
||||
|
||||
val myListener = context.actorOf(Props[MyListener])
|
||||
val myChannel = context.actorOf(Channel.props(
|
||||
ChannelSettings(redeliverFailureListener = Some(myListener))))
|
||||
//#channel-custom-listener
|
||||
}
|
||||
|
||||
class MyProcessor3 extends Processor {
|
||||
def receive = {
|
||||
//#payload-pattern-matching
|
||||
case Persistent(payload, _) =>
|
||||
//#payload-pattern-matching
|
||||
}
|
||||
}
|
||||
|
||||
class MyProcessor4 extends Processor {
|
||||
def receive = {
|
||||
//#sequence-nr-pattern-matching
|
||||
case Persistent(_, sequenceNr) =>
|
||||
//#sequence-nr-pattern-matching
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
//#fsm-example
|
||||
import akka.actor.FSM
|
||||
import akka.persistence.{ Persistent, Processor }
|
||||
|
||||
class PersistentDoor extends Processor with FSM[String, Int] {
|
||||
startWith("closed", 0)
|
||||
|
||||
when("closed") {
|
||||
case Event(Persistent("open", _), counter) =>
|
||||
goto("open") using (counter + 1) replying (counter)
|
||||
}
|
||||
|
||||
when("open") {
|
||||
case Event(Persistent("close", _), counter) =>
|
||||
goto("closed") using (counter + 1) replying (counter)
|
||||
}
|
||||
}
|
||||
//#fsm-example
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
//#save-snapshot
|
||||
class MyProcessor extends Processor {
|
||||
//#save-snapshot
|
||||
var state: Any = _
|
||||
|
||||
def receive = {
|
||||
override def receiveCommand: Receive = {
|
||||
case "snap" => saveSnapshot(state)
|
||||
case SaveSnapshotSuccess(metadata) => // ...
|
||||
case SaveSnapshotFailure(metadata, reason) => // ...
|
||||
}
|
||||
//#save-snapshot
|
||||
|
||||
override def receiveRecover: Receive = ???
|
||||
}
|
||||
//#save-snapshot
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
//#snapshot-offer
|
||||
class MyProcessor extends Processor {
|
||||
class MyPersistentActor extends PersistentActor {
|
||||
override def persistenceId = "my-stable-persistence-id"
|
||||
|
||||
//#snapshot-offer
|
||||
var state: Any = _
|
||||
|
||||
def receive = {
|
||||
override def receiveRecover: Receive = {
|
||||
case SnapshotOffer(metadata, offeredSnapshot) => state = offeredSnapshot
|
||||
case Persistent(payload, sequenceNr) => // ...
|
||||
case RecoveryCompleted =>
|
||||
case event => // ...
|
||||
}
|
||||
//#snapshot-offer
|
||||
|
||||
override def receiveCommand: Receive = ???
|
||||
}
|
||||
//#snapshot-offer
|
||||
|
||||
import akka.actor.Props
|
||||
|
||||
val processor = system.actorOf(Props[MyProcessor])
|
||||
val persistentActor = system.actorOf(Props[MyPersistentActor])
|
||||
|
||||
//#snapshot-criteria
|
||||
processor ! Recover(fromSnapshot = SnapshotSelectionCriteria(
|
||||
persistentActor ! Recover(fromSnapshot = SnapshotSelectionCriteria(
|
||||
maxSequenceNr = 457L,
|
||||
maxTimestamp = System.currentTimeMillis))
|
||||
//#snapshot-criteria
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
import akka.actor.Props
|
||||
//#batch-write
|
||||
class MyProcessor extends Processor {
|
||||
def receive = {
|
||||
case Persistent("a", _) => // ...
|
||||
case Persistent("b", _) => // ...
|
||||
}
|
||||
}
|
||||
|
||||
val system = ActorSystem("example")
|
||||
val processor = system.actorOf(Props[MyProcessor])
|
||||
|
||||
processor ! PersistentBatch(List(Persistent("a"), Persistent("b")))
|
||||
//#batch-write
|
||||
system.terminate()
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
import akka.actor._
|
||||
trait MyActor extends Actor {
|
||||
val destination: ActorRef = null
|
||||
//#persistent-channel-example
|
||||
val channel = context.actorOf(PersistentChannel.props(
|
||||
PersistentChannelSettings(redeliverInterval = 30 seconds, redeliverMax = 15)),
|
||||
name = "myPersistentChannel")
|
||||
|
||||
channel ! Deliver(Persistent("example"), destination.path)
|
||||
//#persistent-channel-example
|
||||
//#persistent-channel-watermarks
|
||||
PersistentChannelSettings(
|
||||
pendingConfirmationsMax = 10000,
|
||||
pendingConfirmationsMin = 2000)
|
||||
//#persistent-channel-watermarks
|
||||
//#persistent-channel-reply
|
||||
PersistentChannelSettings(replyPersistent = true)
|
||||
//#persistent-channel-reply
|
||||
}
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
import akka.actor.ActorRef
|
||||
|
||||
//#reliable-event-delivery
|
||||
class MyPersistentActor(destination: ActorRef) extends PersistentActor {
|
||||
val channel = context.actorOf(Channel.props("channel"))
|
||||
|
||||
override def persistenceId = "my-stable-persistence-id"
|
||||
|
||||
def handleEvent(event: String) = {
|
||||
// update state
|
||||
// ...
|
||||
// reliably deliver events
|
||||
channel ! Deliver(Persistent(event), destination.path)
|
||||
}
|
||||
|
||||
def receiveRecover: Receive = {
|
||||
case event: String => handleEvent(event)
|
||||
}
|
||||
|
||||
def receiveCommand: Receive = {
|
||||
case "cmd" => {
|
||||
// ...
|
||||
persist("evt")(handleEvent)
|
||||
}
|
||||
}
|
||||
}
|
||||
//#reliable-event-delivery
|
||||
}
|
||||
|
||||
new AnyRef {
|
||||
|
||||
val processor = system.actorOf(Props[MyPersistentActor]())
|
||||
val persistentActor = system.actorOf(Props[MyPersistentActor]())
|
||||
|
||||
//#persist-async
|
||||
class MyPersistentActor extends PersistentActor {
|
||||
|
||||
override def persistenceId = "my-stable-persistence-id"
|
||||
|
||||
def receiveRecover: Receive = {
|
||||
override def receiveRecover: Receive = {
|
||||
case _ => // handle recovery here
|
||||
}
|
||||
|
||||
def receiveCommand: Receive = {
|
||||
override def receiveCommand: Receive = {
|
||||
case c: String => {
|
||||
sender() ! c
|
||||
persistAsync(s"evt-$c-1") { e => sender() ! e }
|
||||
|
|
@ -408,8 +215,8 @@ trait PersistenceDocSpec {
|
|||
}
|
||||
|
||||
// usage
|
||||
processor ! "a"
|
||||
processor ! "b"
|
||||
persistentActor ! "a"
|
||||
persistentActor ! "b"
|
||||
|
||||
// possible order of received messages:
|
||||
// a
|
||||
|
|
@ -423,18 +230,18 @@ trait PersistenceDocSpec {
|
|||
}
|
||||
new AnyRef {
|
||||
|
||||
val processor = system.actorOf(Props[MyPersistentActor]())
|
||||
val persistentActor = system.actorOf(Props[MyPersistentActor]())
|
||||
|
||||
//#defer
|
||||
class MyPersistentActor extends PersistentActor {
|
||||
|
||||
override def persistenceId = "my-stable-persistence-id"
|
||||
|
||||
def receiveRecover: Receive = {
|
||||
override def receiveRecover: Receive = {
|
||||
case _ => // handle recovery here
|
||||
}
|
||||
|
||||
def receiveCommand: Receive = {
|
||||
override def receiveCommand: Receive = {
|
||||
case c: String => {
|
||||
sender() ! c
|
||||
persistAsync(s"evt-$c-1") { e => sender() ! e }
|
||||
|
|
@ -446,8 +253,8 @@ trait PersistenceDocSpec {
|
|||
//#defer
|
||||
|
||||
//#defer-caller
|
||||
processor ! "a"
|
||||
processor ! "b"
|
||||
persistentActor ! "a"
|
||||
persistentActor ! "b"
|
||||
|
||||
// order of received messages:
|
||||
// a
|
||||
|
|
@ -469,7 +276,7 @@ trait PersistenceDocSpec {
|
|||
override def persistenceId: String = "some-persistence-id"
|
||||
override def viewId: String = "some-persistence-id-view"
|
||||
|
||||
def receive: Actor.Receive = {
|
||||
def receive: Receive = {
|
||||
case payload if isPersistent =>
|
||||
// handle message from journal...
|
||||
case payload =>
|
||||
|
|
|
|||
|
|
@ -4,19 +4,19 @@
|
|||
|
||||
package docs.persistence
|
||||
|
||||
//#plugin-imports
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.persistence._
|
||||
import akka.persistence.journal._
|
||||
import akka.persistence.snapshot._
|
||||
import akka.testkit.TestKit
|
||||
import com.typesafe.config._
|
||||
import org.scalatest.WordSpec
|
||||
|
||||
import scala.collection.immutable.Seq
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
|
||||
//#plugin-imports
|
||||
import akka.persistence._
|
||||
import akka.persistence.journal._
|
||||
import akka.persistence.snapshot._
|
||||
|
||||
//#plugin-imports
|
||||
|
||||
object PersistencePluginDocSpec {
|
||||
|
|
@ -122,15 +122,18 @@ trait SharedLeveldbPluginDocSpec {
|
|||
|
||||
class MyJournal extends AsyncWriteJournal {
|
||||
def asyncWriteMessages(messages: Seq[PersistentRepr]): Future[Unit] = ???
|
||||
def asyncWriteConfirmations(confirmations: Seq[PersistentConfirmation]): Future[Unit] = ???
|
||||
def asyncDeleteMessages(messageIds: Seq[PersistentId], permanent: Boolean): Future[Unit] = ???
|
||||
def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean): Future[Unit] = ???
|
||||
def asyncReplayMessages(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, max: Long)(replayCallback: (PersistentRepr) => Unit): Future[Unit] = ???
|
||||
def asyncReadHighestSequenceNr(persistenceId: String, fromSequenceNr: Long): Future[Long] = ???
|
||||
def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long,
|
||||
permanent: Boolean): Future[Unit] = ???
|
||||
def asyncReplayMessages(persistenceId: String, fromSequenceNr: Long,
|
||||
toSequenceNr: Long, max: Long)(
|
||||
replayCallback: (PersistentRepr) => Unit): Future[Unit] = ???
|
||||
def asyncReadHighestSequenceNr(persistenceId: String,
|
||||
fromSequenceNr: Long): Future[Long] = ???
|
||||
}
|
||||
|
||||
class MySnapshotStore extends SnapshotStore {
|
||||
def loadAsync(persistenceId: String, criteria: SnapshotSelectionCriteria): Future[Option[SelectedSnapshot]] = ???
|
||||
def loadAsync(persistenceId: String,
|
||||
criteria: SnapshotSelectionCriteria): Future[Option[SelectedSnapshot]] = ???
|
||||
def saveAsync(metadata: SnapshotMetadata, snapshot: Any): Future[Unit] = ???
|
||||
def saved(metadata: SnapshotMetadata): Unit = ???
|
||||
def delete(metadata: SnapshotMetadata): Unit = ???
|
||||
|
|
@ -145,8 +148,8 @@ object PersistenceTCKDoc {
|
|||
class MyJournalSpec extends JournalSpec {
|
||||
override val config = ConfigFactory.parseString(
|
||||
"""
|
||||
|akka.persistence.journal.plugin = "my.journal.plugin"
|
||||
""".stripMargin)
|
||||
akka.persistence.journal.plugin = "my.journal.plugin"
|
||||
""")
|
||||
}
|
||||
//#journal-tck-scala
|
||||
}
|
||||
|
|
@ -157,8 +160,8 @@ object PersistenceTCKDoc {
|
|||
class MySnapshotStoreSpec extends SnapshotStoreSpec {
|
||||
override val config = ConfigFactory.parseString(
|
||||
"""
|
||||
|akka.persistence.snapshot-store.plugin = "my.snapshot-store.plugin"
|
||||
""".stripMargin)
|
||||
akka.persistence.snapshot-store.plugin = "my.snapshot-store.plugin"
|
||||
""")
|
||||
}
|
||||
//#snapshot-store-tck-scala
|
||||
}
|
||||
|
|
@ -172,8 +175,8 @@ object PersistenceTCKDoc {
|
|||
class MyJournalSpec extends JournalSpec {
|
||||
override val config = ConfigFactory.parseString(
|
||||
"""
|
||||
|akka.persistence.journal.plugin = "my.journal.plugin"
|
||||
""".stripMargin)
|
||||
akka.persistence.journal.plugin = "my.journal.plugin"
|
||||
""")
|
||||
|
||||
val storageLocations = List(
|
||||
new File(system.settings.config.getString("akka.persistence.journal.leveldb.dir")),
|
||||
|
|
@ -192,4 +195,4 @@ object PersistenceTCKDoc {
|
|||
}
|
||||
//#journal-tck-before-after-scala
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ class PersistenceSerializerDocSpec extends WordSpec {
|
|||
}
|
||||
}
|
||||
//#custom-serializer-config
|
||||
""".stripMargin
|
||||
"""
|
||||
|
||||
val system = ActorSystem("PersistenceSerializerDocSpec", ConfigFactory.parseString(customSerializerConfig))
|
||||
try {
|
||||
|
|
|
|||
|
|
@ -124,6 +124,8 @@ It contains instructions on how to run the ``PersistentActorExample``.
|
|||
with ``context.become()`` and ``context.unbecome()``. To get the actor into the same state after
|
||||
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.
|
||||
Note that when using ``become`` from ``receiveRecover`` it will still only use the ``receiveRecover``
|
||||
behavior when replaying the events. When replay is completed it will use the new behavior.
|
||||
|
||||
Identifiers
|
||||
-----------
|
||||
|
|
@ -154,8 +156,9 @@ In this case, a persistent actor must be recovered explicitly by sending it a ``
|
|||
.. includecode:: code/docs/persistence/PersistenceDocSpec.scala#recover-explicit
|
||||
|
||||
.. warning::
|
||||
If ``preStart`` is overriden by an empty implementation, incoming commands will not be processed by the
|
||||
``PersistentActor`` until it receives a ``Recover`` and finishes recovery.
|
||||
|
||||
If ``preStart`` is overriden by an empty implementation, incoming commands will not be processed by the
|
||||
``PersistentActor`` until it receives a ``Recover`` and finishes recovery.
|
||||
|
||||
In order to completely skip recovery, you can signal it with ``Recover(toSequenceNr = OL)``
|
||||
|
||||
|
|
@ -211,7 +214,7 @@ The ordering between events is still guaranteed ("evt-b-1" will be sent after "e
|
|||
|
||||
.. note::
|
||||
In order to implement the pattern known as "*command sourcing*" simply call ``persistAsync(cmd)(...)`` right away on all incomming
|
||||
messages right away, and handle them in the callback.
|
||||
messages, and handle them in the callback.
|
||||
|
||||
.. warning::
|
||||
The callback will not be invoked if the actor is restarted (or stopped) in between the call to
|
||||
|
|
@ -656,16 +659,6 @@ or
|
|||
|
||||
in your Akka configuration. The LevelDB Java port is for testing purposes only.
|
||||
|
||||
Miscellaneous
|
||||
=============
|
||||
|
||||
State machines
|
||||
--------------
|
||||
|
||||
State machines can be persisted by mixing in the ``FSM`` trait into persistent actors.
|
||||
|
||||
.. includecode:: code/docs/persistence/PersistenceDocSpec.scala#fsm-example
|
||||
|
||||
Configuration
|
||||
=============
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue