2013-10-15 09:01:07 +02:00
|
|
|
/**
|
2014-02-02 19:05:45 -06:00
|
|
|
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
package akka.persistence
|
|
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
import java.util.concurrent.atomic.AtomicInteger
|
|
|
|
|
import scala.collection.immutable
|
|
|
|
|
import scala.util.control.NonFatal
|
|
|
|
|
import akka.actor.ActorKilledException
|
|
|
|
|
import akka.actor.Stash
|
|
|
|
|
import akka.actor.StashFactory
|
2014-12-14 21:45:22 +01:00
|
|
|
import akka.event.Logging
|
|
|
|
|
import akka.event.LoggingAdapter
|
2015-02-08 19:12:02 -06:00
|
|
|
import akka.actor.ActorRef
|
2013-10-15 09:01:07 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[persistence] object Eventsourced {
|
|
|
|
|
// ok to wrap around (2*Int.MaxValue restarts will not happen within a journal roundtrip)
|
|
|
|
|
private val instanceIdCounter = new AtomicInteger(1)
|
2014-06-24 16:57:33 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private sealed trait PendingHandlerInvocation {
|
|
|
|
|
def evt: Any
|
|
|
|
|
def handler: Any ⇒ Unit
|
|
|
|
|
}
|
|
|
|
|
/** forces actor to stash incoming commands untill all these invocations are handled */
|
|
|
|
|
private final case class StashingHandlerInvocation(evt: Any, handler: Any ⇒ Unit) extends PendingHandlerInvocation
|
|
|
|
|
/** does not force the actor to stash commands; Originates from either `persistAsync` or `defer` calls */
|
|
|
|
|
private final case class AsyncHandlerInvocation(evt: Any, handler: Any ⇒ Unit) extends PendingHandlerInvocation
|
|
|
|
|
}
|
2013-10-15 09:01:07 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API.
|
|
|
|
|
*
|
2014-12-08 11:02:14 +01:00
|
|
|
* Scala API and implementation details of [[PersistentActor]], [[AbstractPersistentActor]] and
|
|
|
|
|
* [[UntypedPersistentActor]].
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2015-02-08 19:12:02 -06:00
|
|
|
private[persistence] trait Eventsourced extends Snapshotter with Stash with StashFactory with PersistenceIdentity {
|
2014-12-08 11:02:14 +01:00
|
|
|
import JournalProtocol._
|
|
|
|
|
import SnapshotProtocol.LoadSnapshotResult
|
|
|
|
|
import Eventsourced._
|
2014-05-21 01:35:21 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private val extension = Persistence(context.system)
|
2015-02-08 19:12:02 -06:00
|
|
|
|
|
|
|
|
private[persistence] lazy val journal = extension.journalFor(journalPluginId)
|
|
|
|
|
private[persistence] lazy val snapshotStore = extension.snapshotStoreFor(snapshotPluginId)
|
2014-03-19 11:59:16 +01:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private val instanceId: Int = Eventsourced.instanceIdCounter.getAndIncrement()
|
2013-11-20 13:47:42 +01:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private var journalBatch = Vector.empty[PersistentEnvelope]
|
|
|
|
|
private val maxMessageBatchSize = extension.settings.journal.maxMessageBatchSize
|
|
|
|
|
private var writeInProgress = false
|
|
|
|
|
private var sequenceNr: Long = 0L
|
|
|
|
|
private var _lastSequenceNr: Long = 0L
|
2014-04-03 11:29:01 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private var currentState: State = recoveryPending
|
2014-05-21 01:35:21 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
// Used instead of iterating `pendingInvocations` 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 pendingInvocations = new java.util.LinkedList[PendingHandlerInvocation]() // we only append / isEmpty / get(0) on it
|
|
|
|
|
private var eventBatch: List[PersistentEnvelope] = Nil
|
2014-05-21 01:35:21 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private val internalStash = createStash()
|
2014-04-03 11:29:01 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private val unstashFilterPredicate: Any ⇒ Boolean = {
|
|
|
|
|
case _: WriteMessageSuccess ⇒ false
|
|
|
|
|
case _: ReplayedMessage ⇒ false
|
|
|
|
|
case _ ⇒ true
|
2013-10-15 09:01:07 +02:00
|
|
|
}
|
|
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
/**
|
|
|
|
|
* Returns `persistenceId`.
|
|
|
|
|
*/
|
|
|
|
|
override def snapshotterId: String = persistenceId
|
2014-06-03 16:40:44 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
/**
|
|
|
|
|
* Highest received sequence number so far or `0L` if this actor hasn't replayed
|
|
|
|
|
* or stored any persistent events yet.
|
|
|
|
|
*/
|
|
|
|
|
def lastSequenceNr: Long = _lastSequenceNr
|
2014-06-03 16:40:44 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
/**
|
|
|
|
|
* Returns `lastSequenceNr`.
|
|
|
|
|
*/
|
|
|
|
|
def snapshotSequenceNr: Long = lastSequenceNr
|
2013-10-15 09:01:07 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API.
|
|
|
|
|
* Called whenever a message replay succeeds.
|
|
|
|
|
* May be implemented by subclass.
|
|
|
|
|
*/
|
|
|
|
|
private[persistence] def onReplaySuccess(): Unit = ()
|
2014-05-21 01:35:21 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API.
|
|
|
|
|
* Called whenever a message replay fails.
|
|
|
|
|
* May be implemented by subclass.
|
|
|
|
|
* @param cause failure cause.
|
|
|
|
|
*/
|
|
|
|
|
private[persistence] def onReplayFailure(cause: Throwable): Unit = ()
|
|
|
|
|
|
|
|
|
|
/**
|
2014-12-14 21:45:22 +01:00
|
|
|
* User-overridable callback. Called when a persistent actor is started or restarted.
|
|
|
|
|
* Default implementation sends a `Recover()` to `self`. Note that if you override
|
|
|
|
|
* `preStart` (or `preRestart`) and not call `super.preStart` you must send
|
|
|
|
|
* a `Recover()` message to `self` to activate the persistent actor.
|
2014-12-08 11:02:14 +01:00
|
|
|
*/
|
|
|
|
|
@throws(classOf[Exception])
|
|
|
|
|
override def preStart(): Unit =
|
|
|
|
|
self ! Recover()
|
|
|
|
|
|
2015-02-08 19:12:02 -06:00
|
|
|
/** INTERNAL API. */
|
2014-12-08 11:02:14 +01:00
|
|
|
override protected[akka] def aroundReceive(receive: Receive, message: Any): Unit =
|
|
|
|
|
currentState.stateReceive(receive, message)
|
|
|
|
|
|
2015-02-08 19:12:02 -06:00
|
|
|
/** INTERNAL API. */
|
|
|
|
|
override protected[akka] def aroundPreStart(): Unit = {
|
|
|
|
|
// Fail fast on missing plugins.
|
|
|
|
|
val j = journal; val s = snapshotStore
|
|
|
|
|
super.aroundPreStart()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/** INTERNAL API. */
|
2014-12-08 11:02:14 +01:00
|
|
|
override protected[akka] def aroundPreRestart(reason: Throwable, message: Option[Any]): Unit = {
|
|
|
|
|
try {
|
|
|
|
|
internalStash.unstashAll()
|
|
|
|
|
unstashAll(unstashFilterPredicate)
|
|
|
|
|
} finally {
|
|
|
|
|
message match {
|
|
|
|
|
case Some(WriteMessageSuccess(m, _)) ⇒
|
|
|
|
|
flushJournalBatch()
|
|
|
|
|
super.aroundPreRestart(reason, Some(m))
|
|
|
|
|
case Some(LoopMessageSuccess(m, _)) ⇒
|
|
|
|
|
flushJournalBatch()
|
|
|
|
|
super.aroundPreRestart(reason, Some(m))
|
|
|
|
|
case Some(ReplayedMessage(m)) ⇒
|
|
|
|
|
flushJournalBatch()
|
|
|
|
|
super.aroundPreRestart(reason, Some(m))
|
|
|
|
|
case mo ⇒
|
|
|
|
|
flushJournalBatch()
|
|
|
|
|
super.aroundPreRestart(reason, None)
|
2013-10-15 09:01:07 +02:00
|
|
|
}
|
|
|
|
|
}
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
2014-05-21 01:35:21 +02:00
|
|
|
|
2015-02-08 19:12:02 -06:00
|
|
|
/** INTERNAL API. */
|
2014-12-08 11:02:14 +01:00
|
|
|
override protected[akka] def aroundPostStop(): Unit =
|
|
|
|
|
try {
|
|
|
|
|
internalStash.unstashAll()
|
|
|
|
|
unstashAll(unstashFilterPredicate)
|
|
|
|
|
} finally super.aroundPostStop()
|
|
|
|
|
|
|
|
|
|
override def unhandled(message: Any): Unit = {
|
|
|
|
|
message match {
|
2014-12-14 21:45:22 +01:00
|
|
|
case RecoveryCompleted | ReadHighestSequenceNrSuccess | ReadHighestSequenceNrFailure ⇒ // mute
|
2014-12-08 11:02:14 +01:00
|
|
|
case RecoveryFailure(cause) ⇒
|
|
|
|
|
val errorMsg = s"PersistentActor killed after recovery failure (persisten id = [${persistenceId}]). " +
|
|
|
|
|
"To avoid killing persistent actors on recovery failure, a PersistentActor must handle RecoveryFailure messages. " +
|
|
|
|
|
"RecoveryFailure was caused by: " + cause
|
|
|
|
|
throw new ActorKilledException(errorMsg)
|
|
|
|
|
case PersistenceFailure(payload, sequenceNumber, cause) ⇒
|
|
|
|
|
val errorMsg = "PersistentActor killed after persistence failure " +
|
|
|
|
|
s"(persistent id = [${persistenceId}], sequence nr = [${sequenceNumber}], payload class = [${payload.getClass.getName}]). " +
|
|
|
|
|
"To avoid killing persistent actors on persistence failure, a PersistentActor must handle PersistenceFailure messages. " +
|
|
|
|
|
"PersistenceFailure was caused by: " + cause
|
|
|
|
|
throw new ActorKilledException(errorMsg)
|
|
|
|
|
case m ⇒ super.unhandled(m)
|
|
|
|
|
}
|
2014-03-19 11:59:16 +01:00
|
|
|
}
|
|
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private def changeState(state: State): Unit = {
|
|
|
|
|
currentState = state
|
2014-05-21 01:35:21 +02:00
|
|
|
}
|
|
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private def updateLastSequenceNr(persistent: PersistentRepr): Unit =
|
|
|
|
|
if (persistent.sequenceNr > _lastSequenceNr) _lastSequenceNr = persistent.sequenceNr
|
|
|
|
|
|
|
|
|
|
private def setLastSequenceNr(value: Long): Unit =
|
|
|
|
|
_lastSequenceNr = value
|
|
|
|
|
|
|
|
|
|
private def nextSequenceNr(): Long = {
|
|
|
|
|
sequenceNr += 1L
|
|
|
|
|
sequenceNr
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def flushJournalBatch(): Unit = {
|
|
|
|
|
journal ! WriteMessages(journalBatch, self, instanceId)
|
|
|
|
|
journalBatch = Vector.empty
|
|
|
|
|
writeInProgress = true
|
2014-06-03 16:40:44 +02:00
|
|
|
}
|
|
|
|
|
|
2014-12-14 21:45:22 +01:00
|
|
|
private def log: LoggingAdapter = Logging(context.system, this)
|
|
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
/**
|
|
|
|
|
* Recovery handler that receives persisted events during recovery. If a state snapshot
|
|
|
|
|
* has been captured and saved, this handler will receive a [[SnapshotOffer]] message
|
|
|
|
|
* followed by events that are younger than the offered snapshot.
|
|
|
|
|
*
|
|
|
|
|
* This handler must not have side-effects other than changing persistent actor state i.e. it
|
|
|
|
|
* should not perform actions that may fail, such as interacting with external services,
|
|
|
|
|
* for example.
|
|
|
|
|
*
|
2014-12-14 21:45:22 +01:00
|
|
|
* If recovery fails, the actor will be stopped by throwing ActorKilledException.
|
|
|
|
|
* This can be customized by handling [[RecoveryFailure]] message in `receiveRecover`
|
|
|
|
|
* and/or defining `supervisorStrategy` in parent actor.
|
2014-12-08 11:02:14 +01:00
|
|
|
*
|
|
|
|
|
* @see [[Recover]]
|
|
|
|
|
*/
|
|
|
|
|
def receiveRecover: Receive
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Command handler. Typically validates commands against current state (and/or by
|
|
|
|
|
* communication with other actors). On successful validation, one or more events are
|
|
|
|
|
* derived from a command and these events are then persisted by calling `persist`.
|
|
|
|
|
*/
|
|
|
|
|
def receiveCommand: Receive
|
|
|
|
|
|
2013-10-15 09:01:07 +02:00
|
|
|
/**
|
|
|
|
|
* Asynchronously persists `event`. On successful persistence, `handler` is called with the
|
2014-06-27 09:44:21 +02:00
|
|
|
* persisted event. It is guaranteed that no new commands will be received by a persistent actor
|
2013-10-15 09:01:07 +02:00
|
|
|
* between a call to `persist` and the execution of its `handler`. This also holds for
|
|
|
|
|
* multiple `persist` calls per received command. Internally, this is achieved by stashing new
|
|
|
|
|
* commands and unstashing them when the `event` has been persisted and handled. The stash used
|
2014-06-27 09:44:21 +02:00
|
|
|
* for that is an internal stash which doesn't interfere with the inherited user stash.
|
2013-10-15 09:01:07 +02:00
|
|
|
*
|
2014-06-27 09:44:21 +02:00
|
|
|
* An event `handler` may close over persistent actor state and modify it. The `sender` of a persisted
|
2013-10-15 09:01:07 +02:00
|
|
|
* event is the sender of the corresponding command. This means that one can reply to a command
|
|
|
|
|
* sender within an event `handler`.
|
|
|
|
|
*
|
2014-06-27 09:44:21 +02:00
|
|
|
* Within an event handler, applications usually update persistent actor state using persisted event
|
2013-10-15 09:01:07 +02:00
|
|
|
* data, notify listeners and reply to command senders.
|
|
|
|
|
*
|
2014-12-14 21:45:22 +01:00
|
|
|
* If persistence of an event fails, the persistent actor will be stopped by throwing ActorKilledException.
|
|
|
|
|
* This can be customized by handling [[PersistenceFailure]] message in [[#receiveCommand]]
|
|
|
|
|
* and/or defining `supervisorStrategy` in parent actor.
|
2013-10-15 09:01:07 +02:00
|
|
|
*
|
2014-05-21 01:35:21 +02:00
|
|
|
* @param event event to be persisted
|
2013-10-15 09:01:07 +02:00
|
|
|
* @param handler handler for each persisted `event`
|
|
|
|
|
*/
|
|
|
|
|
final def persist[A](event: A)(handler: A ⇒ Unit): Unit = {
|
2014-05-21 01:35:21 +02:00
|
|
|
pendingStashingPersistInvocations += 1
|
2014-06-03 16:40:44 +02:00
|
|
|
pendingInvocations addLast StashingHandlerInvocation(event, handler.asInstanceOf[Any ⇒ Unit])
|
2014-12-08 11:02:14 +01:00
|
|
|
eventBatch = PersistentRepr(event) :: eventBatch
|
2013-10-15 09:01:07 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Asynchronously persists `events` in specified order. This is equivalent to calling
|
2013-11-07 10:45:02 +01:00
|
|
|
* `persist[A](event: A)(handler: A => Unit)` multiple times with the same `handler`,
|
|
|
|
|
* except that `events` are persisted atomically with this method.
|
2013-10-15 09:01:07 +02:00
|
|
|
*
|
2014-05-21 01:35:21 +02:00
|
|
|
* @param events events to be persisted
|
2013-10-15 09:01:07 +02:00
|
|
|
* @param handler handler for each persisted `events`
|
|
|
|
|
*/
|
|
|
|
|
final def persist[A](events: immutable.Seq[A])(handler: A ⇒ Unit): Unit =
|
|
|
|
|
events.foreach(persist(_)(handler))
|
|
|
|
|
|
2014-05-21 01:35:21 +02:00
|
|
|
/**
|
|
|
|
|
* Asynchronously persists `event`. On successful persistence, `handler` is called with the
|
|
|
|
|
* persisted event.
|
|
|
|
|
*
|
2014-12-14 21:45:22 +01:00
|
|
|
* Unlike `persist` the persistent actor will continue to receive incoming commands between the
|
2014-05-21 01:35:21 +02:00
|
|
|
* 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.
|
|
|
|
|
*
|
2014-06-27 09:44:21 +02:00
|
|
|
* An event `handler` may close over persistent actor state and modify it. The `sender` of a persisted
|
2014-05-21 01:35:21 +02:00
|
|
|
* event is the sender of the corresponding command. This means that one can reply to a command
|
|
|
|
|
* sender within an event `handler`.
|
|
|
|
|
*
|
2014-12-14 21:45:22 +01:00
|
|
|
* If persistence of an event fails, the persistent actor will be stopped by throwing ActorKilledException.
|
|
|
|
|
* This can be customized by handling [[PersistenceFailure]] message in [[#receiveCommand]]
|
|
|
|
|
* and/or defining `supervisorStrategy` in parent actor.
|
2014-05-21 01:35:21 +02:00
|
|
|
*
|
|
|
|
|
* @param event event to be persisted
|
|
|
|
|
* @param handler handler for each persisted `event`
|
|
|
|
|
*/
|
|
|
|
|
final def persistAsync[A](event: A)(handler: A ⇒ Unit): Unit = {
|
2014-06-03 16:40:44 +02:00
|
|
|
pendingInvocations addLast AsyncHandlerInvocation(event, handler.asInstanceOf[Any ⇒ Unit])
|
2014-12-08 11:02:14 +01:00
|
|
|
eventBatch = PersistentRepr(event) :: eventBatch
|
2014-05-21 01:35:21 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* 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))
|
|
|
|
|
|
2014-06-03 16:40:44 +02:00
|
|
|
/**
|
|
|
|
|
* Defer the handler execution until all pending handlers have been executed.
|
|
|
|
|
* Allows to define logic within the actor, which will respect the invocation-order-guarantee
|
|
|
|
|
* in respect to `persistAsync` calls. That is, if `persistAsync` was invoked before defer,
|
|
|
|
|
* the corresponding handlers will be invoked in the same order as they were registered in.
|
|
|
|
|
*
|
|
|
|
|
* This call will NOT result in `event` being persisted, please use `persist` or `persistAsync`,
|
|
|
|
|
* if the given event should possible to replay.
|
|
|
|
|
*
|
2014-12-14 21:45:22 +01:00
|
|
|
* If there are no pending persist handler calls, the handler will be called immediately.
|
2014-06-03 16:40:44 +02:00
|
|
|
*
|
|
|
|
|
* In the event of persistence failures (indicated by [[PersistenceFailure]] messages being sent to the
|
|
|
|
|
* [[PersistentActor]], you can handle these messages, which in turn will enable the deferred handlers to run afterwards.
|
2014-12-14 21:45:22 +01:00
|
|
|
* If persistence failure messages are left `unhandled`, the default behavior is to stop the Actor by
|
|
|
|
|
* throwing ActorKilledException, thus the handlers will not be run.
|
2014-06-03 16:40:44 +02:00
|
|
|
*
|
2014-12-14 21:45:22 +01:00
|
|
|
* @param event event to be handled in the future, when preceding persist operations have been processes
|
2014-06-03 16:40:44 +02:00
|
|
|
* @param handler handler for the given `event`
|
|
|
|
|
*/
|
|
|
|
|
final def defer[A](event: A)(handler: A ⇒ Unit): Unit = {
|
|
|
|
|
if (pendingInvocations.isEmpty) {
|
|
|
|
|
handler(event)
|
|
|
|
|
} else {
|
|
|
|
|
pendingInvocations addLast AsyncHandlerInvocation(event, handler.asInstanceOf[Any ⇒ Unit])
|
2014-12-08 11:02:14 +01:00
|
|
|
eventBatch = NonPersistentRepr(event, sender()) :: eventBatch
|
2014-06-03 16:40:44 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Defer the handler execution until all pending handlers have been executed.
|
|
|
|
|
* Allows to define logic within the actor, which will respect the invocation-order-guarantee
|
|
|
|
|
* in respect to `persistAsync` calls. That is, if `persistAsync` was invoked before defer,
|
|
|
|
|
* the corresponding handlers will be invoked in the same order as they were registered in.
|
|
|
|
|
*
|
|
|
|
|
* This call will NOT result in `event` being persisted, please use `persist` or `persistAsync`,
|
|
|
|
|
* if the given event should possible to replay.
|
|
|
|
|
*
|
2014-12-14 21:45:22 +01:00
|
|
|
* If there are no pending persist handler calls, the handler will be called immediately.
|
2014-06-03 16:40:44 +02:00
|
|
|
*
|
|
|
|
|
* In the event of persistence failures (indicated by [[PersistenceFailure]] messages being sent to the
|
|
|
|
|
* [[PersistentActor]], you can handle these messages, which in turn will enable the deferred handlers to run afterwards.
|
2014-12-14 21:45:22 +01:00
|
|
|
* If persistence failure messages are left `unhandled`, the default behavior is to stop the Actor by
|
|
|
|
|
* throwing ActorKilledException, thus the handlers will not be run.
|
2014-06-03 16:40:44 +02:00
|
|
|
*
|
2014-12-14 21:45:22 +01:00
|
|
|
* @param events event to be handled in the future, when preceding persist operations have been processes
|
2014-06-03 16:40:44 +02:00
|
|
|
* @param handler handler for each `event`
|
|
|
|
|
*/
|
|
|
|
|
final def defer[A](events: immutable.Seq[A])(handler: A ⇒ Unit): Unit =
|
|
|
|
|
events.foreach(defer(_)(handler))
|
|
|
|
|
|
2013-10-15 09:01:07 +02:00
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Permanently deletes all persistent messages with sequence numbers less than or equal `toSequenceNr`.
|
2013-10-15 09:01:07 +02:00
|
|
|
*
|
2014-12-08 11:02:14 +01:00
|
|
|
* @param toSequenceNr upper sequence number bound of persistent messages to be deleted.
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
def deleteMessages(toSequenceNr: Long): Unit = {
|
|
|
|
|
deleteMessages(toSequenceNr, permanent = true)
|
|
|
|
|
}
|
2013-10-15 09:01:07 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Deletes all persistent messages with sequence numbers less than or equal `toSequenceNr`. If `permanent`
|
|
|
|
|
* is set to `false`, the persistent messages are marked as deleted in the journal, otherwise
|
|
|
|
|
* they permanently deleted from the journal.
|
|
|
|
|
*
|
|
|
|
|
* @param toSequenceNr upper sequence number bound of persistent messages to be deleted.
|
|
|
|
|
* @param permanent if `false`, the message is marked as deleted, otherwise it is permanently deleted.
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
def deleteMessages(toSequenceNr: Long, permanent: Boolean): Unit = {
|
|
|
|
|
journal ! DeleteMessagesTo(persistenceId, toSequenceNr, permanent)
|
2013-10-27 08:01:14 +01:00
|
|
|
}
|
|
|
|
|
|
2013-10-15 09:01:07 +02:00
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Returns `true` if this persistent actor is currently recovering.
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
def recoveryRunning: Boolean = currentState.recoveryRunning
|
2013-10-15 09:01:07 +02:00
|
|
|
|
2014-06-30 11:04:37 +02:00
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Returns `true` if this persistent actor has successfully finished recovery.
|
2014-06-30 11:04:37 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
def recoveryFinished: Boolean = !recoveryRunning
|
|
|
|
|
|
|
|
|
|
override def unstashAll() {
|
|
|
|
|
// Internally, all messages are processed by unstashing them from
|
|
|
|
|
// the internal stash one-by-one. Hence, an unstashAll() from the
|
|
|
|
|
// user stash must be prepended to the internal stash.
|
|
|
|
|
internalStash.prepend(clearStash())
|
2014-06-30 11:04:37 +02:00
|
|
|
}
|
|
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private trait State {
|
|
|
|
|
def stateReceive(receive: Receive, message: Any): Unit
|
|
|
|
|
def recoveryRunning: Boolean
|
2013-10-27 08:01:14 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Initial state, waits for `Recover` request, and then submits a `LoadSnapshot` request to the snapshot
|
|
|
|
|
* store and changes to `recoveryStarted` state. All incoming messages except `Recover` are stashed.
|
2013-10-27 08:01:14 +01:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
private def recoveryPending = new State {
|
|
|
|
|
override def toString: String = "recovery pending"
|
|
|
|
|
override def recoveryRunning: Boolean = true
|
|
|
|
|
|
|
|
|
|
override def stateReceive(receive: Receive, message: Any): Unit = message match {
|
|
|
|
|
case Recover(fromSnap, toSnr, replayMax) ⇒
|
|
|
|
|
changeState(recoveryStarted(replayMax))
|
|
|
|
|
loadSnapshot(snapshotterId, fromSnap, toSnr)
|
|
|
|
|
case _ ⇒ internalStash.stash()
|
|
|
|
|
}
|
2013-10-27 08:01:14 +01:00
|
|
|
}
|
|
|
|
|
|
2013-10-15 09:01:07 +02:00
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Processes a loaded snapshot, if any. A loaded snapshot is offered with a `SnapshotOffer`
|
|
|
|
|
* message to the actor's `receiveRecover`. Then initiates a message replay, either starting
|
|
|
|
|
* from the loaded snapshot or from scratch, and switches to `replayStarted` state.
|
|
|
|
|
* All incoming messages are stashed.
|
2014-03-19 11:59:16 +01:00
|
|
|
*
|
2014-12-08 11:02:14 +01:00
|
|
|
* @param replayMax maximum number of messages to replay.
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
private def recoveryStarted(replayMax: Long) = new State {
|
|
|
|
|
|
|
|
|
|
private val recoveryBehavior: Receive = {
|
|
|
|
|
val _receiveRecover = receiveRecover
|
|
|
|
|
|
|
|
|
|
{
|
|
|
|
|
case PersistentRepr(payload, _) if recoveryRunning && _receiveRecover.isDefinedAt(payload) ⇒
|
|
|
|
|
_receiveRecover(payload)
|
|
|
|
|
case s: SnapshotOffer if _receiveRecover.isDefinedAt(s) ⇒
|
|
|
|
|
_receiveRecover(s)
|
|
|
|
|
case f: RecoveryFailure if _receiveRecover.isDefinedAt(f) ⇒
|
|
|
|
|
_receiveRecover(f)
|
|
|
|
|
case RecoveryCompleted if _receiveRecover.isDefinedAt(RecoveryCompleted) ⇒
|
|
|
|
|
_receiveRecover(RecoveryCompleted)
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-10-15 09:01:07 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
override def toString: String = s"recovery started (replayMax = [${replayMax}])"
|
|
|
|
|
override def recoveryRunning: Boolean = true
|
|
|
|
|
|
|
|
|
|
override def stateReceive(receive: Receive, message: Any) = message match {
|
|
|
|
|
case r: Recover ⇒ // ignore
|
|
|
|
|
case LoadSnapshotResult(sso, toSnr) ⇒
|
|
|
|
|
sso.foreach {
|
|
|
|
|
case SelectedSnapshot(metadata, snapshot) ⇒
|
|
|
|
|
setLastSequenceNr(metadata.sequenceNr)
|
|
|
|
|
// Since we are recovering we can ignore the receive behavior from the stack
|
|
|
|
|
Eventsourced.super.aroundReceive(recoveryBehavior, SnapshotOffer(metadata, snapshot))
|
|
|
|
|
}
|
|
|
|
|
changeState(replayStarted(recoveryBehavior))
|
|
|
|
|
journal ! ReplayMessages(lastSequenceNr + 1L, toSnr, replayMax, persistenceId, self)
|
|
|
|
|
case other ⇒ internalStash.stash()
|
|
|
|
|
}
|
2013-10-15 09:01:07 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Processes replayed messages, if any. The actor's `receiveRecover` is invoked with the replayed
|
|
|
|
|
* events.
|
2013-10-15 09:01:07 +02:00
|
|
|
*
|
2014-12-08 11:02:14 +01:00
|
|
|
* If replay succeeds it switches to `initializing` state and requests the highest stored sequence
|
|
|
|
|
* number from the journal. Otherwise RecoveryFailure is emitted.
|
|
|
|
|
* If replay succeeds the `onReplaySuccess` callback method is called, otherwise `onReplayFailure`.
|
2013-10-15 09:01:07 +02:00
|
|
|
*
|
2014-12-08 11:02:14 +01:00
|
|
|
* If processing of a replayed event fails, the exception is caught and
|
2014-12-13 15:35:12 +01:00
|
|
|
* stored for later `RecoveryFailure` message and state is changed to `recoveryFailed`.
|
2013-10-15 09:01:07 +02:00
|
|
|
*
|
2014-12-08 11:02:14 +01:00
|
|
|
* All incoming messages are stashed.
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
private def replayStarted(recoveryBehavior: Receive) = new State {
|
|
|
|
|
override def toString: String = s"replay started"
|
|
|
|
|
override def recoveryRunning: Boolean = true
|
|
|
|
|
|
|
|
|
|
override def stateReceive(receive: Receive, message: Any) = message match {
|
|
|
|
|
case r: Recover ⇒ // ignore
|
|
|
|
|
case ReplayedMessage(p) ⇒
|
|
|
|
|
try {
|
|
|
|
|
updateLastSequenceNr(p)
|
|
|
|
|
Eventsourced.super.aroundReceive(recoveryBehavior, p)
|
|
|
|
|
} catch {
|
|
|
|
|
case NonFatal(t) ⇒
|
2014-12-13 15:35:12 +01:00
|
|
|
changeState(replayFailed(recoveryBehavior, t, p))
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
|
|
|
|
case ReplayMessagesSuccess ⇒
|
|
|
|
|
onReplaySuccess() // callback for subclass implementation
|
|
|
|
|
changeState(initializing(recoveryBehavior))
|
|
|
|
|
journal ! ReadHighestSequenceNr(lastSequenceNr, persistenceId, self)
|
|
|
|
|
case ReplayMessagesFailure(cause) ⇒
|
2014-12-14 21:45:22 +01:00
|
|
|
// in case the actor resumes the state must be initializing
|
|
|
|
|
changeState(initializing(recoveryBehavior))
|
|
|
|
|
journal ! ReadHighestSequenceNr(lastSequenceNr, persistenceId, self)
|
|
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
onReplayFailure(cause) // callback for subclass implementation
|
2014-12-13 15:35:12 +01:00
|
|
|
Eventsourced.super.aroundReceive(recoveryBehavior, RecoveryFailure(cause)(None))
|
2014-12-08 11:02:14 +01:00
|
|
|
case other ⇒
|
|
|
|
|
internalStash.stash()
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-10-15 09:01:07 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-12-13 15:35:12 +01:00
|
|
|
* Consumes remaining replayed messages and then emits RecoveryFailure to the
|
|
|
|
|
* `receiveRecover` behavior.
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2014-12-13 15:35:12 +01:00
|
|
|
private def replayFailed(recoveryBehavior: Receive, cause: Throwable, failed: PersistentRepr) = new State {
|
2014-12-08 11:02:14 +01:00
|
|
|
|
|
|
|
|
override def toString: String = "replay failed"
|
|
|
|
|
override def recoveryRunning: Boolean = true
|
|
|
|
|
|
|
|
|
|
override def stateReceive(receive: Receive, message: Any) = message match {
|
2014-12-13 15:35:12 +01:00
|
|
|
case ReplayedMessage(p) ⇒ updateLastSequenceNr(p)
|
2014-12-14 21:45:22 +01:00
|
|
|
case ReplayMessagesSuccess | ReplayMessagesFailure(_) ⇒ replayCompleted()
|
|
|
|
|
case r: Recover ⇒ // ignore
|
|
|
|
|
case _ ⇒ internalStash.stash()
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
2013-10-15 09:01:07 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
def replayCompleted(): Unit = {
|
2014-12-14 21:45:22 +01:00
|
|
|
// in case the actor resumes the state must be initializing
|
|
|
|
|
changeState(initializing(recoveryBehavior))
|
|
|
|
|
journal ! ReadHighestSequenceNr(failed.sequenceNr, persistenceId, self)
|
|
|
|
|
|
2014-12-13 15:35:12 +01:00
|
|
|
Eventsourced.super.aroundReceive(recoveryBehavior,
|
|
|
|
|
RecoveryFailure(cause)(Some((failed.sequenceNr, failed.payload))))
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
|
|
|
|
}
|
2014-06-03 16:40:44 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Processes the highest stored sequence number response from the journal and then switches
|
|
|
|
|
* to `processingCommands` state.
|
|
|
|
|
* All incoming messages are stashed.
|
2014-06-03 16:40:44 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
private def initializing(recoveryBehavior: Receive) = new State {
|
|
|
|
|
override def toString: String = "initializing"
|
|
|
|
|
override def recoveryRunning: Boolean = true
|
|
|
|
|
|
|
|
|
|
override def stateReceive(receive: Receive, message: Any) = message match {
|
|
|
|
|
case ReadHighestSequenceNrSuccess(highest) ⇒
|
|
|
|
|
changeState(processingCommands)
|
|
|
|
|
sequenceNr = highest
|
2014-12-14 21:45:22 +01:00
|
|
|
setLastSequenceNr(highest)
|
2014-12-08 11:02:14 +01:00
|
|
|
internalStash.unstashAll()
|
|
|
|
|
Eventsourced.super.aroundReceive(recoveryBehavior, RecoveryCompleted)
|
|
|
|
|
case ReadHighestSequenceNrFailure(cause) ⇒
|
2014-12-14 21:45:22 +01:00
|
|
|
log.error(cause, "PersistentActor could not retrieve highest sequence number and must " +
|
|
|
|
|
"therefore be stopped. (persisten id = [{}]).", persistenceId)
|
|
|
|
|
context.stop(self)
|
2014-12-08 11:02:14 +01:00
|
|
|
case other ⇒
|
|
|
|
|
internalStash.stash()
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-06-03 16:40:44 +02:00
|
|
|
|
2013-10-15 09:01:07 +02:00
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Common receive handler for processingCommands and persistingEvents
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
private abstract class ProcessingState extends State {
|
|
|
|
|
val common: Receive = {
|
|
|
|
|
case WriteMessageSuccess(p, id) ⇒
|
|
|
|
|
// instanceId mismatch can happen for persistAsync and defer in case of actor restart
|
|
|
|
|
// while message is in flight, in that case we ignore the call to the handler
|
|
|
|
|
if (id == instanceId) {
|
|
|
|
|
updateLastSequenceNr(p)
|
2014-12-14 21:45:22 +01:00
|
|
|
try {
|
|
|
|
|
pendingInvocations.peek().handler(p.payload)
|
|
|
|
|
onWriteMessageComplete(err = false)
|
|
|
|
|
} catch { case NonFatal(e) ⇒ onWriteMessageComplete(err = true); throw e }
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
|
|
|
|
case WriteMessageFailure(p, cause, id) ⇒
|
|
|
|
|
// instanceId mismatch can happen for persistAsync and defer in case of actor restart
|
|
|
|
|
// while message is in flight, in that case the handler has already been discarded
|
|
|
|
|
if (id == instanceId) {
|
2014-12-14 21:45:22 +01:00
|
|
|
try {
|
|
|
|
|
Eventsourced.super.aroundReceive(receive, PersistenceFailure(p.payload, p.sequenceNr, cause)) // stops actor by default
|
|
|
|
|
onWriteMessageComplete(err = false)
|
|
|
|
|
} catch { case NonFatal(e) ⇒ onWriteMessageComplete(err = true); throw e }
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
|
|
|
|
case LoopMessageSuccess(l, id) ⇒
|
|
|
|
|
// instanceId mismatch can happen for persistAsync and defer in case of actor restart
|
|
|
|
|
// while message is in flight, in that case we ignore the call to the handler
|
|
|
|
|
if (id == instanceId) {
|
2014-12-14 21:45:22 +01:00
|
|
|
try {
|
|
|
|
|
pendingInvocations.peek().handler(l)
|
|
|
|
|
onWriteMessageComplete(err = false)
|
|
|
|
|
} catch { case NonFatal(e) ⇒ onWriteMessageComplete(err = true); throw e }
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
2014-12-14 21:45:22 +01:00
|
|
|
case WriteMessagesSuccessful | WriteMessagesFailed(_) ⇒ // FIXME PN: WriteMessagesFailed?
|
2014-12-08 11:02:14 +01:00
|
|
|
if (journalBatch.isEmpty) writeInProgress = false else flushJournalBatch()
|
|
|
|
|
}
|
|
|
|
|
|
2014-12-14 21:45:22 +01:00
|
|
|
def onWriteMessageComplete(err: Boolean): Unit =
|
2014-12-08 11:02:14 +01:00
|
|
|
pendingInvocations.pop()
|
|
|
|
|
}
|
2013-10-15 09:01:07 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Command processing state. If event persistence is pending after processing a
|
|
|
|
|
* command, event persistence is triggered and state changes to `persistingEvents`.
|
2013-10-15 09:01:07 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
private val processingCommands: State = new ProcessingState {
|
|
|
|
|
override def toString: String = "processing commands"
|
|
|
|
|
override def recoveryRunning: Boolean = false
|
2014-02-15 23:44:00 -05:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
override def stateReceive(receive: Receive, message: Any) =
|
|
|
|
|
if (common.isDefinedAt(message)) common(message)
|
2014-12-14 21:45:22 +01:00
|
|
|
else try {
|
|
|
|
|
Eventsourced.super.aroundReceive(receive, message)
|
|
|
|
|
aroundReceiveComplete(err = false)
|
|
|
|
|
} catch { case NonFatal(e) ⇒ aroundReceiveComplete(err = true); throw e }
|
2014-02-15 23:44:00 -05:00
|
|
|
|
2014-12-14 21:45:22 +01:00
|
|
|
private def aroundReceiveComplete(err: Boolean): Unit = {
|
2014-12-08 11:02:14 +01:00
|
|
|
if (eventBatch.nonEmpty) flushBatch()
|
2014-03-20 12:05:32 +01:00
|
|
|
|
2014-12-14 21:45:22 +01:00
|
|
|
if (pendingStashingPersistInvocations > 0)
|
|
|
|
|
changeState(persistingEvents)
|
|
|
|
|
else if (err)
|
|
|
|
|
internalStash.unstashAll()
|
|
|
|
|
else
|
|
|
|
|
internalStash.unstash()
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
2014-05-21 01:35:21 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
private def flushBatch() {
|
|
|
|
|
// When using only `persistAsync` and `defer` max throughput is increased by using
|
|
|
|
|
// batching, but when using `persist` we want to use one atomic WriteMessages
|
|
|
|
|
// for the emitted events.
|
|
|
|
|
// Flush previously collected events, if any, separately from the `persist` batch
|
|
|
|
|
if (pendingStashingPersistInvocations > 0 && journalBatch.nonEmpty)
|
|
|
|
|
flushJournalBatch()
|
|
|
|
|
|
|
|
|
|
eventBatch.reverse.foreach { p ⇒
|
|
|
|
|
addToBatch(p)
|
|
|
|
|
if (!writeInProgress || maxBatchSizeReached) flushJournalBatch()
|
|
|
|
|
}
|
2014-06-03 16:40:44 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
eventBatch = Nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def addToBatch(p: PersistentEnvelope): Unit = p match {
|
|
|
|
|
case p: PersistentRepr ⇒
|
|
|
|
|
journalBatch :+= p.update(persistenceId = persistenceId, sequenceNr = nextSequenceNr(), sender = sender())
|
|
|
|
|
case r: PersistentEnvelope ⇒
|
|
|
|
|
journalBatch :+= r
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def maxBatchSizeReached: Boolean =
|
|
|
|
|
journalBatch.size >= maxMessageBatchSize
|
|
|
|
|
|
|
|
|
|
}
|
2014-06-03 16:40:44 +02:00
|
|
|
|
2014-05-21 01:35:21 +02:00
|
|
|
/**
|
2014-12-08 11:02:14 +01:00
|
|
|
* Event persisting state. Remains until pending events are persisted and then changes
|
|
|
|
|
* state to `processingCommands`. Only events to be persisted are processed. All other
|
|
|
|
|
* messages are stashed internally.
|
2014-05-21 01:35:21 +02:00
|
|
|
*/
|
2014-12-08 11:02:14 +01:00
|
|
|
private val persistingEvents: State = new ProcessingState {
|
|
|
|
|
override def toString: String = "persisting events"
|
|
|
|
|
override def recoveryRunning: Boolean = false
|
2014-05-21 01:35:21 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
override def stateReceive(receive: Receive, message: Any) =
|
|
|
|
|
if (common.isDefinedAt(message)) common(message)
|
|
|
|
|
else internalStash.stash()
|
2014-03-20 12:05:32 +01:00
|
|
|
|
2014-12-14 21:45:22 +01:00
|
|
|
override def onWriteMessageComplete(err: Boolean): Unit = {
|
2014-12-08 11:02:14 +01:00
|
|
|
pendingInvocations.pop() match {
|
|
|
|
|
case _: StashingHandlerInvocation ⇒
|
|
|
|
|
// enables an early return to `processingCommands`, because if this counter hits `0`,
|
|
|
|
|
// we know the remaining pendingInvocations 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
|
|
|
|
|
case _ ⇒ // do nothing
|
|
|
|
|
}
|
2014-06-26 13:56:01 +02:00
|
|
|
|
2014-12-08 11:02:14 +01:00
|
|
|
if (pendingStashingPersistInvocations == 0) {
|
|
|
|
|
changeState(processingCommands)
|
2014-12-14 21:45:22 +01:00
|
|
|
if (err) internalStash.unstashAll()
|
|
|
|
|
else internalStash.unstash()
|
2014-12-08 11:02:14 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
2014-06-26 13:56:01 +02:00
|
|
|
|
2014-06-03 15:10:56 +02:00
|
|
|
}
|