!per Change for journal plugin compatibility

* A few more adjustments, found when verifying source compatibility of a few
  journal plugins
* Note that PersistentId will be removed with deleteMessage and we should
  not break plugins because of that
* Add missing section of at-least-once delivery in migration guide

(cherry picked from commit 6727eac6d07280d277968e2e25db44e02be3b102)

Conflicts:
	akka-persistence/src/main/scala/akka/persistence/JournalProtocol.scala
	akka-persistence/src/main/scala/akka/persistence/Persistent.scala
	akka-persistence/src/main/scala/akka/persistence/Snapshot.scala
	akka-persistence/src/main/scala/akka/persistence/journal/AsyncWriteProxy.scala
This commit is contained in:
Patrik Nordwall 2014-06-27 11:51:58 +02:00
parent f6aa491ef0
commit 33c7f6bb4f
20 changed files with 81 additions and 74 deletions

View file

@ -87,7 +87,7 @@ public class PersistencePluginDocTest {
} }
@Override @Override
public Future<Void> doAsyncDeleteMessages(Iterable<PersistenceId> messageIds, boolean permanent) { public Future<Void> doAsyncDeleteMessages(Iterable<PersistentId> messageIds, boolean permanent) {
return null; return null;
} }

View file

@ -27,6 +27,8 @@ To extend ``PersistentActor``::
/*...*/ /*...*/
} }
Read more about the persistent actor in the :ref:`documentation for Scala <event-sourcing>` and
:ref:`documentation for Java <event-sourcing-java>`.
Changed processorId to (abstract) persistenceId Changed processorId to (abstract) persistenceId
=============================================== ===============================================
@ -36,7 +38,7 @@ Persistent messages, as well as processors implemented the ``processorId`` metho
This concept remains the same in Akka ``2.3.4``, yet we rename ``processorId`` to ``persistenceId`` because Processors will be removed, This concept remains the same in Akka ``2.3.4``, yet we rename ``processorId`` to ``persistenceId`` because Processors will be removed,
and persistent messages can be used from different classes not only ``PersistentActor`` (Views, directly from Journals etc). and persistent messages can be used from different classes not only ``PersistentActor`` (Views, directly from Journals etc).
Please note that ``processorId`` is **abstract** in the new API classes (``PersistentActor`` and ``PersistentView``), Please note that ``persistenceId`` is **abstract** in the new API classes (``PersistentActor`` and ``PersistentView``),
and we do **not** provide a default (actor-path derrived) value for it like we did for ``processorId``. and we do **not** provide a default (actor-path derrived) value for it like we did for ``processorId``.
The rationale behind this change being stricter de-coupling of your Actor hierarchy and the logical "which persistent entity this actor represents". The rationale behind this change being stricter de-coupling of your Actor hierarchy and the logical "which persistent entity this actor represents".
A longer discussion on this subject can be found on `issue #15436 <https://github.com/akka/akka/issues/15436>`_ on github. A longer discussion on this subject can be found on `issue #15436 <https://github.com/akka/akka/issues/15436>`_ on github.
@ -49,32 +51,6 @@ implement it yourself either as a helper trait or simply by overriding ``persist
We provided the renamed method also on already deprecated classes (Channels), We provided the renamed method also on already deprecated classes (Channels),
so you can simply apply a global rename of ``processorId`` to ``persistenceId``. so you can simply apply a global rename of ``processorId`` to ``persistenceId``.
Plugin APIs: Renamed PersistentId to PersistenceId
==================================================
Following the removal of Processors and moving to ``persistenceId``, the plugin SPI visible type has changed.
The move from ``2.3.3`` to ``2.3.4`` should be relatively painless, and plugins will work even when using the deprecated ``PersistentId`` type.
Change your implementations from::
def asyncWriteMessages(messages: immutable.Seq[PersistentRepr]): Future[Unit] = // ...
def asyncDeleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean): Future[Unit] = {
val p = messageIds.head.processorId // old
// ...
}
to::
def asyncWriteMessages(messages: immutable.Seq[PersistentRepr]): Future[Unit] = // ...
def asyncDeleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean): Future[Unit] = {
val p = messageIds.head.persistenceId // new
// ...
}
Plugins written for ``2.3.3`` are source level compatible with ``2.3.4``, using the deprecated types, but will not work with future releases.
Plugin maintainers are asked to update their plugins to ``2.3.4`` as soon as possible.
Removed Processor in favour of extending PersistentActor with persistAsync Removed Processor in favour of extending PersistentActor with persistAsync
========================================================================== ==========================================================================
@ -110,7 +86,7 @@ Replacement code, with the same semantics, using PersistentActor::
persistAsync(cmd) { e => sender() ! e } persistAsync(cmd) { e => sender() ! e }
} }
def receiveEvent = { def receiveRecover = {
case _ => // logic for handling replay case _ => // logic for handling replay
} }
} }
@ -118,7 +94,7 @@ Replacement code, with the same semantics, using PersistentActor::
It is worth pointing out that using ``sender()`` inside the persistAsync callback block is **valid**, and does *not* suffer 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. 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 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)`` 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. 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.
@ -172,3 +148,24 @@ You should update it to extend ``PersistentView`` instead::
In case you need to obtain the current sequence number the view is looking at, you can use the ``lastSequenceNr`` method. In case you need to obtain the current sequence number the view is looking at, you can use the ``lastSequenceNr`` method.
It is equivalent to "current sequence number", when ``isPersistent`` returns true, otherwise it yields the sequence number It is equivalent to "current sequence number", when ``isPersistent`` returns true, otherwise it yields the sequence number
of the last persistent message that this view was updated with. of the last persistent message that this view was updated with.
Removed Channel and PersistentChannel in favour of AtLeastOnceDelivery trait
============================================================================
One of the primary tasks of a ``Channel`` was to de-duplicate messages that were sent from a
``Processor`` during recovery. Performing external side effects during recovery is not
encouraged with event sourcing and therefore the ``Channel`` is not needed for this purpose.
The ``Channel`` and ``PersistentChannel`` also performed at-least-once delivery of messages,
but it did not free a sending actor from implementing retransmission or timeouts, since the
acknowledgement from the channel is needed to guarantee safe hand-off. Therefore at-least-once
delivery is provided in a new ``AtLeastOnceDelivery`` trait that is mixed-in to the
persistent actor on the sending side.
Read more about at-least-once delivery in the :ref:`documentation for Scala <at-least-once-delivery>` and
:ref:`documentation for Java <at-least-once-delivery-java>`.

View file

@ -126,7 +126,7 @@ trait SharedLeveldbPluginDocSpec {
class MyJournal extends AsyncWriteJournal { class MyJournal extends AsyncWriteJournal {
def asyncWriteMessages(messages: Seq[PersistentRepr]): Future[Unit] = ??? def asyncWriteMessages(messages: Seq[PersistentRepr]): Future[Unit] = ???
def asyncWriteConfirmations(confirmations: Seq[PersistentConfirmation]): Future[Unit] = ??? def asyncWriteConfirmations(confirmations: Seq[PersistentConfirmation]): Future[Unit] = ???
def asyncDeleteMessages(messageIds: Seq[PersistenceId], permanent: Boolean): Future[Unit] = ??? def asyncDeleteMessages(messageIds: Seq[PersistentId], permanent: Boolean): Future[Unit] = ???
def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long, 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 asyncReplayMessages(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, max: Long)(replayCallback: (PersistentRepr) => Unit): Future[Unit] = ???
def asyncReadHighestSequenceNr(persistenceId: String, fromSequenceNr: Long): Future[Long] = ??? def asyncReadHighestSequenceNr(persistenceId: String, fromSequenceNr: Long): Future[Long] = ???

View file

@ -32,7 +32,7 @@ interface AsyncWritePlugin {
* *
* @deprecated doAsyncDeleteMessages will be removed (since 2.3.4) * @deprecated doAsyncDeleteMessages will be removed (since 2.3.4)
*/ */
@Deprecated Future<Void> doAsyncDeleteMessages(Iterable<PersistenceId> messageIds, boolean permanent); @Deprecated Future<Void> doAsyncDeleteMessages(Iterable<PersistentId> messageIds, boolean permanent);
/** /**
* Java API, Plugin API: synchronously deletes all persistent messages up to * Java API, Plugin API: synchronously deletes all persistent messages up to

View file

@ -30,7 +30,7 @@ interface SyncWritePlugin {
* *
* @deprecated doDeleteMessages will be removed (since 2.3.4) * @deprecated doDeleteMessages will be removed (since 2.3.4)
*/ */
@Deprecated void doDeleteMessages(Iterable<PersistenceId> messageIds, boolean permanent); @Deprecated void doDeleteMessages(Iterable<PersistentId> messageIds, boolean permanent);
/** /**
* Java API, Plugin API: synchronously deletes all persistent messages up to * Java API, Plugin API: synchronously deletes all persistent messages up to

View file

@ -18,12 +18,12 @@ private[persistence] object JournalProtocol {
* Request to delete messages identified by `messageIds`. If `permanent` is set to `false`, * Request to delete messages identified by `messageIds`. If `permanent` is set to `false`,
* the persistent messages are marked as deleted, otherwise they are permanently deleted. * the persistent messages are marked as deleted, otherwise they are permanently deleted.
*/ */
final case class DeleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean, requestor: Option[ActorRef] = None) final case class DeleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean, requestor: Option[ActorRef] = None)
/** /**
* Reply message to a successful [[DeleteMessages]] request. * Reply message to a successful [[DeleteMessages]] request.
*/ */
final case class DeleteMessagesSuccess(messageIds: immutable.Seq[PersistenceId]) final case class DeleteMessagesSuccess(messageIds: immutable.Seq[PersistentId])
/** /**
* Reply message to a failed [[DeleteMessages]] request. * Reply message to a failed [[DeleteMessages]] request.

View file

@ -130,7 +130,7 @@ object ConfirmablePersistent {
* During recovery, they are also replayed individually. * During recovery, they are also replayed individually.
*/ */
@deprecated("Use akka.persistence.PersistentActor instead", since = "2.3.4") @deprecated("Use akka.persistence.PersistentActor instead", since = "2.3.4")
case class PersistentBatch(batch: immutable.Seq[Resequenceable]) extends Message final case class PersistentBatch(batch: immutable.Seq[Resequenceable]) extends Message
@deprecated("Use akka.persistence.PersistentActor instead", since = "2.3.4") @deprecated("Use akka.persistence.PersistentActor instead", since = "2.3.4")
object PersistentBatch { object PersistentBatch {
@ -155,26 +155,19 @@ trait PersistentConfirmation {
/** /**
* Plugin API: persistent message identifier. * Plugin API: persistent message identifier.
*
* Deprecated, please use [[PersistenceId]].
*/ */
@deprecated("Use PersistenceId instead.", since = "2.3.4") @deprecated("deleteMessages will be removed.", since = "2.3.4")
trait PersistentId extends PersistenceId { trait PersistentId {
/**
* Persistent id that journals a persistent message
*/
@deprecated("Use `persistenceId` instead.", since = "2.3.4")
def processorId: String = persistenceId
}
/**
* Plugin API: persistent message identifier.
*/
trait PersistenceId {
/** /**
* Persistent id that journals a persistent message * Persistent id that journals a persistent message
*/ */
def persistenceId: String def processorId: String
/**
* Persistent id that journals a persistent message
*/
def persistenceId: String = processorId
/** /**
* A persistent message's sequence number. * A persistent message's sequence number.
@ -185,7 +178,8 @@ trait PersistenceId {
/** /**
* INTERNAL API. * INTERNAL API.
*/ */
private[persistence] final case class PersistenceIdImpl(persistenceId: String, sequenceNr: Long) extends PersistenceId @deprecated("deleteMessages will be removed.", since = "2.3.4")
private[persistence] final case class PersistentIdImpl(processorId: String, sequenceNr: Long) extends PersistentId
/** /**
* Plugin API: representation of a persistent message in the journal plugin API. * Plugin API: representation of a persistent message in the journal plugin API.
@ -194,7 +188,7 @@ private[persistence] final case class PersistenceIdImpl(persistenceId: String, s
* @see [[journal.AsyncWriteJournal]] * @see [[journal.AsyncWriteJournal]]
* @see [[journal.AsyncRecovery]] * @see [[journal.AsyncRecovery]]
*/ */
trait PersistentRepr extends Persistent with Resequenceable with PersistenceId with Message { trait PersistentRepr extends Persistent with Resequenceable with PersistentId with Message {
// todo we want to get rid of the Persistent() wrapper from user land; PersistentRepr is here to stay. #15230 // todo we want to get rid of the Persistent() wrapper from user land; PersistentRepr is here to stay. #15230
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
@ -313,7 +307,7 @@ object PersistentRepr {
private[persistence] final case class PersistentImpl( private[persistence] final case class PersistentImpl(
payload: Any, payload: Any,
sequenceNr: Long, sequenceNr: Long,
@deprecatedName('processorId) persistenceId: String, @deprecatedName('processorId) override val persistenceId: String,
deleted: Boolean, deleted: Boolean,
confirms: immutable.Seq[String], confirms: immutable.Seq[String],
sender: ActorRef) extends Persistent with PersistentRepr { sender: ActorRef) extends Persistent with PersistentRepr {
@ -339,6 +333,9 @@ private[persistence] final case class PersistentImpl(
val confirmable: Boolean = false val confirmable: Boolean = false
val confirmMessage: Delivered = null val confirmMessage: Delivered = null
val confirmTarget: ActorRef = null val confirmTarget: ActorRef = null
@deprecated("Use persistenceId.", since = "2.3.4")
override def processorId = persistenceId
} }
/** /**
@ -348,7 +345,7 @@ private[persistence] final case class PersistentImpl(
private[persistence] final case class ConfirmablePersistentImpl( private[persistence] final case class ConfirmablePersistentImpl(
payload: Any, payload: Any,
sequenceNr: Long, sequenceNr: Long,
@deprecatedName('processorId) persistenceId: String, @deprecatedName('processorId) override val persistenceId: String,
deleted: Boolean, deleted: Boolean,
redeliveries: Int, redeliveries: Int,
confirms: immutable.Seq[String], confirms: immutable.Seq[String],
@ -369,6 +366,9 @@ private[persistence] final case class ConfirmablePersistentImpl(
def update(sequenceNr: Long, @deprecatedName('processorId) persistenceId: String, deleted: Boolean, redeliveries: Int, confirms: immutable.Seq[String], confirmMessage: Delivered, confirmTarget: ActorRef, sender: ActorRef) = def update(sequenceNr: Long, @deprecatedName('processorId) persistenceId: String, deleted: Boolean, redeliveries: Int, confirms: immutable.Seq[String], confirmMessage: Delivered, confirmTarget: ActorRef, sender: ActorRef) =
copy(sequenceNr = sequenceNr, persistenceId = persistenceId, deleted = deleted, redeliveries = redeliveries, confirms = confirms, confirmMessage = confirmMessage, confirmTarget = confirmTarget, sender = sender) copy(sequenceNr = sequenceNr, persistenceId = persistenceId, deleted = deleted, redeliveries = redeliveries, confirms = confirms, confirmMessage = confirmMessage, confirmTarget = confirmTarget, sender = sender)
@deprecated("Use persistenceId.", since = "2.3.4")
override def processorId = persistenceId
} }
/** /**

View file

@ -205,9 +205,11 @@ final case class DeliveredByPersistentChannel(
channelId: String, channelId: String,
persistentSequenceNr: Long, persistentSequenceNr: Long,
deliverySequenceNr: Long = 0L, deliverySequenceNr: Long = 0L,
channel: ActorRef = null) extends Delivered with PersistenceId { channel: ActorRef = null) extends Delivered with PersistentId {
def persistenceId: String = channelId override def persistenceId: String = channelId
@deprecated("Use persistenceId.", since = "2.3.4")
override def processorId = persistenceId
def sequenceNr: Long = persistentSequenceNr def sequenceNr: Long = persistentSequenceNr
def update(deliverySequenceNr: Long, channel: ActorRef): DeliveredByPersistentChannel = def update(deliverySequenceNr: Long, channel: ActorRef): DeliveredByPersistentChannel =
copy(deliverySequenceNr = deliverySequenceNr, channel = channel) copy(deliverySequenceNr = deliverySequenceNr, channel = channel)

View file

@ -223,7 +223,7 @@ private[akka] trait ProcessorImpl extends Actor with Recovery {
*/ */
@deprecated("deleteMessage(sequenceNr) will be removed. Instead, validate before persist, and use deleteMessages for pruning.", since = "2.3.4") @deprecated("deleteMessage(sequenceNr) will be removed. Instead, validate before persist, and use deleteMessages for pruning.", since = "2.3.4")
def deleteMessage(sequenceNr: Long, permanent: Boolean): Unit = { def deleteMessage(sequenceNr: Long, permanent: Boolean): Unit = {
journal ! DeleteMessages(List(PersistenceIdImpl(persistenceId, sequenceNr)), permanent) journal ! DeleteMessages(List(PersistentIdImpl(persistenceId, sequenceNr)), permanent)
} }
/** /**

View file

@ -13,7 +13,10 @@ package akka.persistence
* @param timestamp time at which the snapshot was saved. * @param timestamp time at which the snapshot was saved.
*/ */
@SerialVersionUID(1L) //#snapshot-metadata @SerialVersionUID(1L) //#snapshot-metadata
final case class SnapshotMetadata(@deprecatedName('processorId) persistenceId: String, sequenceNr: Long, timestamp: Long = 0L) final case class SnapshotMetadata(@deprecatedName('processorId) persistenceId: String, sequenceNr: Long, timestamp: Long = 0L) {
@deprecated("Use persistenceId instead.", since = "2.3.4")
def processorId: String = persistenceId
}
//#snapshot-metadata //#snapshot-metadata
/** /**
@ -120,7 +123,10 @@ private[persistence] object SnapshotProtocol {
* @param criteria criteria for selecting a snapshot from which recovery should start. * @param criteria criteria for selecting a snapshot from which recovery should start.
* @param toSequenceNr upper sequence number bound (inclusive) for recovery. * @param toSequenceNr upper sequence number bound (inclusive) for recovery.
*/ */
final case class LoadSnapshot(@deprecatedName('processorId) persistenceId: String, criteria: SnapshotSelectionCriteria, toSequenceNr: Long) final case class LoadSnapshot(@deprecatedName('processorId) persistenceId: String, criteria: SnapshotSelectionCriteria, toSequenceNr: Long) {
@deprecated("Use persistenceId instead.", since = "2.3.4")
def processorId: String = persistenceId
}
/** /**
* Response message to a [[LoadSnapshot]] message. * Response message to a [[LoadSnapshot]] message.
@ -150,5 +156,8 @@ private[persistence] object SnapshotProtocol {
* @param persistenceId persistent actor id. * @param persistenceId persistent actor id.
* @param criteria criteria for selecting snapshots to be deleted. * @param criteria criteria for selecting snapshots to be deleted.
*/ */
final case class DeleteSnapshots(@deprecatedName('processorId) persistenceId: String, criteria: SnapshotSelectionCriteria) final case class DeleteSnapshots(@deprecatedName('processorId) persistenceId: String, criteria: SnapshotSelectionCriteria) {
@deprecated("Use persistenceId instead.", since = "2.3.4")
def processorId: String = persistenceId
}
} }

View file

@ -105,7 +105,7 @@ trait AsyncWriteJournal extends Actor with WriteJournalBase with AsyncRecovery {
* deleted, otherwise they are permanently deleted. * deleted, otherwise they are permanently deleted.
*/ */
@deprecated("asyncDeleteMessages will be removed.", since = "2.3.4") @deprecated("asyncDeleteMessages will be removed.", since = "2.3.4")
def asyncDeleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean): Future[Unit] def asyncDeleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean): Future[Unit]
/** /**
* Plugin API: asynchronously deletes all persistent messages up to `toSequenceNr` * Plugin API: asynchronously deletes all persistent messages up to `toSequenceNr`

View file

@ -43,7 +43,7 @@ private[persistence] trait AsyncWriteProxy extends AsyncWriteJournal with Stash
def asyncWriteConfirmations(confirmations: immutable.Seq[PersistentConfirmation]): Future[Unit] = def asyncWriteConfirmations(confirmations: immutable.Seq[PersistentConfirmation]): Future[Unit] =
(store ? WriteConfirmations(confirmations)).mapTo[Unit] (store ? WriteConfirmations(confirmations)).mapTo[Unit]
def asyncDeleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean): Future[Unit] = def asyncDeleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean): Future[Unit] =
(store ? DeleteMessages(messageIds, permanent)).mapTo[Unit] (store ? DeleteMessages(messageIds, permanent)).mapTo[Unit]
def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean): Future[Unit] = def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean): Future[Unit] =
@ -78,7 +78,7 @@ private[persistence] object AsyncWriteTarget {
final case class WriteConfirmations(confirmations: immutable.Seq[PersistentConfirmation]) final case class WriteConfirmations(confirmations: immutable.Seq[PersistentConfirmation])
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class DeleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean) final case class DeleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean)
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class DeleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean) final case class DeleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean)

View file

@ -97,7 +97,7 @@ trait SyncWriteJournal extends Actor with WriteJournalBase with AsyncRecovery {
* deleted, otherwise they are permanently deleted. * deleted, otherwise they are permanently deleted.
*/ */
@deprecated("deleteMessages will be removed.", since = "2.3.4") @deprecated("deleteMessages will be removed.", since = "2.3.4")
def deleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean): Unit def deleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean): Unit
/** /**
* Plugin API: synchronously deletes all persistent messages up to `toSequenceNr` * Plugin API: synchronously deletes all persistent messages up to `toSequenceNr`

View file

@ -34,7 +34,7 @@ private[persistence] class InmemJournal extends AsyncWriteProxy {
* INTERNAL API. * INTERNAL API.
*/ */
private[persistence] trait InmemMessages { private[persistence] trait InmemMessages {
// persistentActorId -> persistent message // persistenceId -> persistent message
var messages = Map.empty[String, Vector[PersistentRepr]] var messages = Map.empty[String, Vector[PersistentRepr]]
def add(p: PersistentRepr) = messages = messages + (messages.get(p.persistenceId) match { def add(p: PersistentRepr) = messages = messages + (messages.get(p.persistenceId) match {

View file

@ -22,7 +22,7 @@ abstract class AsyncWriteJournal extends AsyncRecovery with SAsyncWriteJournal w
final def asyncWriteConfirmations(confirmations: immutable.Seq[PersistentConfirmation]) = final def asyncWriteConfirmations(confirmations: immutable.Seq[PersistentConfirmation]) =
doAsyncWriteConfirmations(confirmations.asJava).map(Unit.unbox) doAsyncWriteConfirmations(confirmations.asJava).map(Unit.unbox)
final def asyncDeleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean) = final def asyncDeleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean) =
doAsyncDeleteMessages(messageIds.asJava, permanent).map(Unit.unbox) doAsyncDeleteMessages(messageIds.asJava, permanent).map(Unit.unbox)
final def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean) = final def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean) =

View file

@ -20,7 +20,7 @@ abstract class SyncWriteJournal extends AsyncRecovery with SSyncWriteJournal wit
final def writeConfirmations(confirmations: immutable.Seq[PersistentConfirmation]) = final def writeConfirmations(confirmations: immutable.Seq[PersistentConfirmation]) =
doWriteConfirmations(confirmations.asJava) doWriteConfirmations(confirmations.asJava)
final def deleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean) = final def deleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean) =
doDeleteMessages(messageIds.asJava, permanent) doDeleteMessages(messageIds.asJava, permanent)
final def deleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean) = final def deleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean) =

View file

@ -46,7 +46,7 @@ private[persistence] trait LeveldbStore extends Actor with LeveldbIdMapping with
def writeConfirmations(confirmations: immutable.Seq[PersistentConfirmation]) = def writeConfirmations(confirmations: immutable.Seq[PersistentConfirmation]) =
withBatch(batch confirmations.foreach(confirmation addToConfirmationBatch(confirmation, batch))) withBatch(batch confirmations.foreach(confirmation addToConfirmationBatch(confirmation, batch)))
def deleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean) = withBatch { batch def deleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean) = withBatch { batch
messageIds foreach { messageId messageIds foreach { messageId
if (permanent) batch.delete(keyToBytes(Key(numericId(messageId.persistenceId), messageId.sequenceNr, 0))) if (permanent) batch.delete(keyToBytes(Key(numericId(messageId.persistenceId), messageId.sequenceNr, 0)))
else batch.put(keyToBytes(deletionKey(numericId(messageId.persistenceId), messageId.sequenceNr)), Array.emptyByteArray) else batch.put(keyToBytes(deletionKey(numericId(messageId.persistenceId), messageId.sequenceNr)), Array.emptyByteArray)

View file

@ -24,7 +24,7 @@ class ReplayFailedException(ps: Seq[PersistentRepr])
class ReadHighestFailedException class ReadHighestFailedException
extends TestException(s"recovery failed when reading highest sequence number") extends TestException(s"recovery failed when reading highest sequence number")
class DeleteFailedException(messageIds: immutable.Seq[PersistenceId]) class DeleteFailedException(messageIds: immutable.Seq[PersistentId])
extends TestException(s"delete failed for message ids = [${messageIds}]") extends TestException(s"delete failed for message ids = [${messageIds}]")
/** /**
@ -53,13 +53,13 @@ class ChaosJournal extends SyncWriteJournal {
if (shouldFail(confirmFailureRate)) throw new ConfirmFailedException(confirmations) if (shouldFail(confirmFailureRate)) throw new ConfirmFailedException(confirmations)
else confirmations.foreach(cnf update(cnf.persistenceId, cnf.sequenceNr)(p p.update(confirms = cnf.channelId +: p.confirms))) else confirmations.foreach(cnf update(cnf.persistenceId, cnf.sequenceNr)(p p.update(confirms = cnf.channelId +: p.confirms)))
def deleteMessages(messageIds: immutable.Seq[PersistenceId], permanent: Boolean): Unit = def deleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean): Unit =
if (shouldFail(deleteFailureRate)) throw new DeleteFailedException(messageIds) if (shouldFail(deleteFailureRate)) throw new DeleteFailedException(messageIds)
else if (permanent) messageIds.foreach(mid update(mid.persistenceId, mid.sequenceNr)(_.update(deleted = true))) else if (permanent) messageIds.foreach(mid update(mid.persistenceId, mid.sequenceNr)(_.update(deleted = true)))
else messageIds.foreach(mid del(mid.persistenceId, mid.sequenceNr)) else messageIds.foreach(mid del(mid.persistenceId, mid.sequenceNr))
def deleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean): Unit = def deleteMessagesTo(persistenceId: String, toSequenceNr: Long, permanent: Boolean): Unit =
(1L to toSequenceNr).map(PersistenceIdImpl(persistenceId, _)) (1L to toSequenceNr).map(PersistentIdImpl(persistenceId, _))
def asyncReplayMessages(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, max: Long)(replayCallback: (PersistentRepr) Unit): Future[Unit] = def asyncReplayMessages(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, max: Long)(replayCallback: (PersistentRepr) Unit): Future[Unit] =
if (shouldFail(replayFailureRate)) { if (shouldFail(replayFailureRate)) {

View file

@ -87,7 +87,7 @@ public class LambdaPersistencePluginDocTest {
} }
@Override @Override
public Future<Void> doAsyncDeleteMessages(Iterable<PersistenceId> messageIds, boolean permanent) { public Future<Void> doAsyncDeleteMessages(Iterable<PersistentId> messageIds, boolean permanent) {
return null; return null;
} }

View file

@ -9,7 +9,6 @@ import akka.actor.ActorSystem;
import akka.actor.Props; import akka.actor.Props;
import akka.japi.pf.ReceiveBuilder; import akka.japi.pf.ReceiveBuilder;
import akka.persistence.AbstractPersistentActor; import akka.persistence.AbstractPersistentActor;
import akka.persistence.Persistent;
import akka.persistence.SnapshotOffer; import akka.persistence.SnapshotOffer;
import scala.PartialFunction; import scala.PartialFunction;
import scala.runtime.BoxedUnit; import scala.runtime.BoxedUnit;