!per #3704 Persistence improvements
- Channel enhancements (#3773): - Live read models (#3776): - Batch-oriented journal plugin API (#3804): - Batching of confirmations and deletions - Message deletion enhancements (more efficient range deletions)
This commit is contained in:
parent
32b76adb9a
commit
f327e1e357
55 changed files with 3474 additions and 2191 deletions
|
|
@ -4,64 +4,67 @@
|
|||
|
||||
package akka.persistence
|
||||
|
||||
import java.lang.{ Iterable ⇒ JIterable }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.concurrent.duration._
|
||||
import scala.language.postfixOps
|
||||
|
||||
import akka.actor._
|
||||
import akka.dispatch.Envelope
|
||||
|
||||
import akka.persistence.JournalProtocol.Confirm
|
||||
import akka.persistence.serialization.Message
|
||||
import akka.persistence.JournalProtocol._
|
||||
|
||||
/**
|
||||
* A [[Channel]] configuration object.
|
||||
*
|
||||
* @param redeliverMax maximum number of redeliveries (default is 5).
|
||||
* @param redeliverInterval interval between redeliveries (default is 5 seconds).
|
||||
* @param redeliverMax Maximum number of redelivery attempts.
|
||||
* @param redeliverInterval Interval between redelivery attempts.
|
||||
* @param redeliverFailureListener Receiver of [[RedeliverFailure]] notifications which are sent when the number
|
||||
* of redeliveries reaches `redeliverMax` for a sequence of messages. To enforce
|
||||
* a redelivery of these messages, the listener has to restart the sending processor.
|
||||
* Alternatively, it can also confirm these messages, preventing further redeliveries.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
class ChannelSettings(
|
||||
val redeliverMax: Int,
|
||||
val redeliverInterval: FiniteDuration) extends Serializable {
|
||||
case class ChannelSettings(
|
||||
val redeliverMax: Int = 5,
|
||||
val redeliverInterval: FiniteDuration = 5.seconds,
|
||||
val redeliverFailureListener: Option[ActorRef] = None) {
|
||||
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def withRedeliverMax(redeliverMax: Int): ChannelSettings =
|
||||
update(redeliverMax = redeliverMax)
|
||||
copy(redeliverMax = redeliverMax)
|
||||
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def withRedeliverInterval(redeliverInterval: FiniteDuration): ChannelSettings =
|
||||
update(redeliverInterval = redeliverInterval)
|
||||
|
||||
private def update(
|
||||
redeliverMax: Int = redeliverMax,
|
||||
redeliverInterval: FiniteDuration = redeliverInterval): ChannelSettings =
|
||||
new ChannelSettings(redeliverMax, redeliverInterval)
|
||||
}
|
||||
|
||||
object ChannelSettings {
|
||||
def apply(
|
||||
redeliverMax: Int = 5,
|
||||
redeliverInterval: FiniteDuration = 5 seconds): ChannelSettings =
|
||||
new ChannelSettings(redeliverMax, redeliverInterval)
|
||||
copy(redeliverInterval = redeliverInterval)
|
||||
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def create() = apply()
|
||||
def withRedeliverFailureListener(redeliverFailureListener: ActorRef): ChannelSettings =
|
||||
copy(redeliverFailureListener = Option(redeliverFailureListener))
|
||||
}
|
||||
|
||||
object ChannelSettings {
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def create() = ChannelSettings.apply()
|
||||
}
|
||||
|
||||
/**
|
||||
* A channel is used by [[Processor]]s for sending [[Persistent]] messages to destinations. The main
|
||||
* responsibility of a channel is to prevent redundant delivery of replayed messages to destinations
|
||||
* A channel is used by [[Processor]]s (and [[View]]s) for sending [[Persistent]] messages to destinations.
|
||||
* The main responsibility of a channel is to prevent redundant delivery of replayed messages to destinations
|
||||
* when a processor is recovered.
|
||||
*
|
||||
* A channel is instructed to deliver a persistent message to a `destination` with the [[Deliver]]
|
||||
* command.
|
||||
* A channel is instructed to deliver a persistent message to a destination with the [[Deliver]] command. A
|
||||
* destination is provided as `ActorPath` and messages are sent via that path's `ActorSelection`.
|
||||
*
|
||||
* {{{
|
||||
* class ForwardExample extends Processor {
|
||||
|
|
@ -71,7 +74,7 @@ object ChannelSettings {
|
|||
* def receive = {
|
||||
* case m @ Persistent(payload, _) =>
|
||||
* // forward modified message to destination
|
||||
* channel forward Deliver(m.withPayload(s"fw: ${payload}"), destination)
|
||||
* channel forward Deliver(m.withPayload(s"fw: ${payload}"), destination.path)
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
|
|
@ -86,7 +89,7 @@ object ChannelSettings {
|
|||
* def receive = {
|
||||
* case m @ Persistent(payload, _) =>
|
||||
* // reply modified message to sender
|
||||
* channel ! Deliver(m.withPayload(s"re: ${payload}"), sender)
|
||||
* channel ! Deliver(m.withPayload(s"re: ${payload}"), sender.path)
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
|
|
@ -105,39 +108,38 @@ object ChannelSettings {
|
|||
* }}}
|
||||
*
|
||||
* If a destination does not confirm the receipt of a `ConfirmablePersistent` message, it will be redelivered
|
||||
* by the channel according to the parameters in [[ChannelSettings]]. Message redelivery is done out of order
|
||||
* with regards to normal delivery i.e. redelivered messages may arrive later than newer normally delivered
|
||||
* messages. Redelivered messages have a `redeliveries` value greater than zero.
|
||||
* by the channel according to the parameters in [[ChannelSettings]]. Redelivered messages have a `redeliveries`
|
||||
* value greater than zero.
|
||||
*
|
||||
* If the maximum number of redeliveries for a certain message is reached and there is still no confirmation
|
||||
* from the destination, then this message is removed from the channel. In order to deliver that message to
|
||||
* the destination again, the processor must replay its stored messages to the channel (during start or restart).
|
||||
* Replayed, unconfirmed messages are then processed and delivered by the channel again. These messages are now
|
||||
* duplicates (with a `redeliveries` counter starting from zero). Duplicates can be detected by destinations
|
||||
* by tracking message sequence numbers.
|
||||
* If the maximum number of redeliveries is reached for certain messages, they are removed from the channel and
|
||||
* a `redeliverFailureListener` (if specified, see [[ChannelSettings]]) is notified about these messages with a
|
||||
* [[RedeliverFailure]] message. Besides other application-specific tasks, this listener can restart the sending
|
||||
* processor to enforce a redelivery of these messages or confirm these messages to prevent further redeliveries.
|
||||
*
|
||||
* @see [[Deliver]]
|
||||
*/
|
||||
final class Channel private[akka] (_channelId: Option[String], channelSettings: ChannelSettings) extends Actor {
|
||||
import channelSettings._
|
||||
|
||||
private val id = _channelId match {
|
||||
case Some(cid) ⇒ cid
|
||||
case None ⇒ Persistence(context.system).channelId(self)
|
||||
}
|
||||
|
||||
private val journal = Persistence(context.system).journalFor(id)
|
||||
|
||||
private val reliableDelivery = context.actorOf(Props(classOf[ReliableDelivery], channelSettings))
|
||||
private val resolvedDelivery = context.actorOf(Props(classOf[ResolvedDelivery], reliableDelivery))
|
||||
private val journal = Persistence(context.system).confirmationBatchingJournalForChannel(id)
|
||||
private val delivery = context.actorOf(Props(classOf[ReliableDelivery], channelSettings))
|
||||
|
||||
def receive = {
|
||||
case d @ Deliver(persistent: PersistentRepr, _, _) ⇒
|
||||
if (!persistent.confirms.contains(id)) resolvedDelivery forward d.copy(prepareDelivery(persistent))
|
||||
case d @ Deliver(persistent: PersistentRepr, _) ⇒
|
||||
if (!persistent.confirms.contains(id)) delivery forward d.copy(prepareDelivery(persistent))
|
||||
case d: RedeliverFailure ⇒ redeliverFailureListener.foreach(_ ! d)
|
||||
case d: Delivered ⇒ delivery forward d
|
||||
}
|
||||
|
||||
private def prepareDelivery(persistent: PersistentRepr): PersistentRepr =
|
||||
ConfirmablePersistentImpl(persistent,
|
||||
confirmTarget = journal,
|
||||
confirmMessage = Confirm(persistent.processorId, persistent.sequenceNr, id))
|
||||
confirmMessage = DeliveredByChannel(persistent.processorId, id, persistent.sequenceNr, channel = self))
|
||||
}
|
||||
|
||||
object Channel {
|
||||
|
|
@ -178,189 +180,144 @@ object Channel {
|
|||
}
|
||||
|
||||
/**
|
||||
* Instructs a [[Channel]] or [[PersistentChannel]] to deliver `persistent` message to
|
||||
* destination `destination`. The `resolve` parameter can be:
|
||||
*
|
||||
* - `Resolve.Destination`: will resolve a new destination reference from the specified
|
||||
* `destination`s path. The `persistent` message will be sent to the newly resolved
|
||||
* destination.
|
||||
* - `Resolve.Sender`: will resolve a new sender reference from this `Deliver` message's
|
||||
* `sender` path. The `persistent` message will be sent to the specified `destination`
|
||||
* using the newly resolved sender.
|
||||
* - `Resolve.Off`: will not do any resolution (default).
|
||||
*
|
||||
* Resolving an actor reference means first obtaining an `ActorSelection` from the path of
|
||||
* the reference to be resolved and then obtaining a new actor reference via an `Identify`
|
||||
* - `ActorIdentity` conversation. Actor reference resolution does not change the original
|
||||
* order of messages.
|
||||
*
|
||||
* Resolving actor references may become necessary when using the stored sender references
|
||||
* of replayed messages. A stored sender reference may become invalid (for example, it may
|
||||
* reference a previous sender incarnation, after a JVM restart). Depending on how a processor
|
||||
* uses sender references, two resolution strategies are relevant.
|
||||
*
|
||||
* - `Resolve.Sender` when a processor forwards a replayed message to a destination.
|
||||
*
|
||||
* {{{
|
||||
* channel forward Deliver(message, destination, Resolve.Sender)
|
||||
* }}}
|
||||
*
|
||||
* - `Resolve.Destination` when a processor replies to the sender of a replayed message. In
|
||||
* this case the sender is used as channel destination.
|
||||
*
|
||||
* {{{
|
||||
* channel ! Deliver(message, sender, Resolve.Destination)
|
||||
* }}}
|
||||
*
|
||||
* A destination or sender reference will only be resolved by a channel if
|
||||
*
|
||||
* - the `resolve` parameter is set to `Resolve.Destination` or `Resolve.Channel`
|
||||
* - the message is replayed
|
||||
* - the message is not retained by the channel and
|
||||
* - there was no previous successful resolve action for that message
|
||||
* Instructs a [[Channel]] or [[PersistentChannel]] to deliver a `persistent` message to
|
||||
* a `destination`.
|
||||
*
|
||||
* @param persistent persistent message.
|
||||
* @param destination persistent message destination.
|
||||
* @param resolve resolve strategy.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class Deliver(persistent: Persistent, destination: ActorRef, resolve: Resolve.ResolveStrategy = Resolve.Off) extends Message
|
||||
case class Deliver(persistent: Persistent, destination: ActorPath) extends Message
|
||||
|
||||
object Deliver {
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def create(persistent: Persistent, destination: ActorRef) = Deliver(persistent, destination)
|
||||
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def create(persistent: Persistent, destination: ActorRef, resolve: Resolve.ResolveStrategy) = Deliver(persistent, destination, resolve)
|
||||
def create(persistent: Persistent, destination: ActorPath) = Deliver(persistent, destination)
|
||||
}
|
||||
|
||||
/**
|
||||
* Actor reference resolution strategy.
|
||||
*
|
||||
* @see [[Deliver]]
|
||||
* Plugin API: confirmation message generated by receivers of [[ConfirmablePersistent]] messages
|
||||
* by calling `ConfirmablePersistent.confirm()`.
|
||||
*/
|
||||
object Resolve {
|
||||
sealed abstract class ResolveStrategy
|
||||
trait Delivered extends Message {
|
||||
def channelId: String
|
||||
def persistentSequenceNr: Long
|
||||
def deliverySequenceNr: Long
|
||||
def channel: ActorRef
|
||||
|
||||
/**
|
||||
* No resolution.
|
||||
* INTERNAL API.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case object Off extends ResolveStrategy
|
||||
|
||||
/**
|
||||
* [[Channel]] should resolve the `sender` of a [[Deliver]] message.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case object Sender extends ResolveStrategy
|
||||
|
||||
/**
|
||||
* [[Channel]] should resolve the `destination` of a [[Deliver]] message.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case object Destination extends ResolveStrategy
|
||||
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def off() = Off
|
||||
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def sender() = Sender
|
||||
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def destination() = Destination
|
||||
private[persistence] def update(deliverySequenceNr: Long = deliverySequenceNr, channel: ActorRef = channel): Delivered
|
||||
}
|
||||
|
||||
/**
|
||||
* Resolves actor references as specified by [[Deliver]] requests and then delegates delivery
|
||||
* to `next`.
|
||||
* Plugin API.
|
||||
*/
|
||||
private class ResolvedDelivery(next: ActorRef) extends Actor with Stash {
|
||||
private var currentResolution: Envelope = _
|
||||
case class DeliveredByChannel(
|
||||
processorId: String,
|
||||
channelId: String,
|
||||
persistentSequenceNr: Long,
|
||||
deliverySequenceNr: Long = 0L,
|
||||
channel: ActorRef = null) extends Delivered with PersistentConfirmation {
|
||||
|
||||
private val delivering: Receive = {
|
||||
case d @ Deliver(persistent: PersistentRepr, destination, resolve) ⇒
|
||||
resolve match {
|
||||
case Resolve.Sender if !persistent.resolved ⇒
|
||||
context.actorSelection(sender.path) ! Identify(1)
|
||||
context.become(resolving, discardOld = false)
|
||||
currentResolution = Envelope(d, sender, context.system)
|
||||
case Resolve.Destination if !persistent.resolved ⇒
|
||||
context.actorSelection(destination.path) ! Identify(1)
|
||||
context.become(resolving, discardOld = false)
|
||||
currentResolution = Envelope(d, sender, context.system)
|
||||
case _ ⇒ next forward d
|
||||
def sequenceNr: Long = persistentSequenceNr
|
||||
def update(deliverySequenceNr: Long, channel: ActorRef): DeliveredByChannel =
|
||||
copy(deliverySequenceNr = deliverySequenceNr, channel = channel)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private[persistence] class DeliveredByChannelBatching(journal: ActorRef, settings: PersistenceSettings) extends Actor {
|
||||
private val publish = settings.internal.publishConfirmations
|
||||
private val batchMax = settings.journal.maxConfirmationBatchSize
|
||||
|
||||
private var batching = false
|
||||
private var batch = Vector.empty[DeliveredByChannel]
|
||||
|
||||
def receive = {
|
||||
case WriteConfirmationsSuccess(confirmations) ⇒
|
||||
if (batch.isEmpty) batching = false else journalBatch()
|
||||
confirmations.foreach { c ⇒
|
||||
val dbc = c.asInstanceOf[DeliveredByChannel]
|
||||
if (dbc.channel != null) dbc.channel ! c
|
||||
if (publish) context.system.eventStream.publish(c)
|
||||
}
|
||||
unstash()
|
||||
case WriteConfirmationsFailure(_) ⇒
|
||||
if (batch.isEmpty) batching = false else journalBatch()
|
||||
case d: DeliveredByChannel ⇒
|
||||
addToBatch(d)
|
||||
if (!batching || maxBatchSizeReached) journalBatch()
|
||||
case m ⇒ journal forward m
|
||||
}
|
||||
|
||||
private val resolving: Receive = {
|
||||
case ActorIdentity(1, resolvedOption) ⇒
|
||||
val Envelope(d: Deliver, sender) = currentResolution
|
||||
if (d.resolve == Resolve.Sender) {
|
||||
next tell (d, resolvedOption.getOrElse(sender))
|
||||
} else if (d.resolve == Resolve.Destination) {
|
||||
next tell (d.copy(destination = resolvedOption.getOrElse(d.destination)), sender)
|
||||
}
|
||||
context.unbecome()
|
||||
unstash()
|
||||
case _: Deliver ⇒ stash()
|
||||
}
|
||||
def addToBatch(pc: DeliveredByChannel): Unit =
|
||||
batch = batch :+ pc
|
||||
|
||||
def receive = delivering
|
||||
def maxBatchSizeReached: Boolean =
|
||||
batch.length >= batchMax
|
||||
|
||||
def journalBatch(): Unit = {
|
||||
journal ! WriteConfirmations(batch, self)
|
||||
batch = Vector.empty
|
||||
batching = true
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Notification message to inform channel listeners about messages that have reached the maximum
|
||||
* number of redeliveries.
|
||||
*/
|
||||
case class RedeliverFailure(messages: immutable.Seq[ConfirmablePersistent]) {
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
def getMessages: JIterable[ConfirmablePersistent] = messages.asJava
|
||||
}
|
||||
|
||||
/**
|
||||
* Reliably deliver messages contained in [[Deliver]] requests to their destinations. Unconfirmed
|
||||
* messages are redelivered according to the parameters in [[ChannelSettings]].
|
||||
*/
|
||||
private class ReliableDelivery(channelSettings: ChannelSettings) extends Actor {
|
||||
import channelSettings._
|
||||
private class ReliableDelivery(redeliverSettings: ChannelSettings) extends Actor {
|
||||
import redeliverSettings._
|
||||
import ReliableDelivery._
|
||||
|
||||
private val redelivery = context.actorOf(Props(classOf[Redelivery], channelSettings))
|
||||
private var attempts: DeliveryAttempts = Map.empty
|
||||
private var sequenceNr: Long = 0L
|
||||
private val redelivery = context.actorOf(Props(classOf[Redelivery], redeliverSettings))
|
||||
private var deliveryAttempts: DeliveryAttempts = immutable.SortedMap.empty
|
||||
private var deliverySequenceNr: Long = 0L
|
||||
|
||||
def receive = {
|
||||
case d @ Deliver(persistent: PersistentRepr, destination, _) ⇒
|
||||
val dsnr = nextSequenceNr()
|
||||
case d @ Deliver(persistent: ConfirmablePersistentImpl, destination) ⇒
|
||||
val dsnr = nextDeliverySequenceNr()
|
||||
val psnr = persistent.sequenceNr
|
||||
val confirm = persistent.confirmMessage.copy(channelEndpoint = self)
|
||||
val confirm = persistent.confirmMessage.update(deliverySequenceNr = dsnr)
|
||||
val updated = persistent.update(confirmMessage = confirm, sequenceNr = if (psnr == 0) dsnr else psnr)
|
||||
destination forward updated
|
||||
attempts += ((updated.processorId, updated.sequenceNr) -> DeliveryAttempt(updated, destination, sender, dsnr))
|
||||
case c @ Confirm(processorId, messageSequenceNr, _, _, _) ⇒
|
||||
attempts -= ((processorId, messageSequenceNr))
|
||||
context.actorSelection(destination).tell(updated, sender)
|
||||
deliveryAttempts += (dsnr -> DeliveryAttempt(updated, destination, sender))
|
||||
case d: Delivered ⇒
|
||||
deliveryAttempts -= d.deliverySequenceNr
|
||||
redelivery forward d
|
||||
case Redeliver ⇒
|
||||
val limit = System.nanoTime - redeliverInterval.toNanos
|
||||
val (older, younger) = attempts.partition { case (_, a) ⇒ a.timestamp < limit }
|
||||
val (older, younger) = deliveryAttempts.span { case (_, a) ⇒ a.timestamp < limit }
|
||||
redelivery ! Redeliver(older, redeliverMax)
|
||||
attempts = younger
|
||||
deliveryAttempts = younger
|
||||
}
|
||||
|
||||
private def nextSequenceNr(): Long = {
|
||||
sequenceNr += 1
|
||||
sequenceNr
|
||||
private def nextDeliverySequenceNr(): Long = {
|
||||
deliverySequenceNr += 1
|
||||
deliverySequenceNr
|
||||
}
|
||||
}
|
||||
|
||||
private object ReliableDelivery {
|
||||
type DeliveryAttempts = immutable.Map[(String, Long), DeliveryAttempt]
|
||||
|
||||
case class DeliveryAttempt(persistent: PersistentRepr, destination: ActorRef, sender: ActorRef, deliverySequenceNr: Long, timestamp: Long = System.nanoTime) {
|
||||
def withChannelEndpoint(channelEndpoint: ActorRef) =
|
||||
copy(persistent.update(confirmMessage = persistent.confirmMessage.copy(channelEndpoint = channelEndpoint)))
|
||||
type DeliveryAttempts = immutable.SortedMap[Long, DeliveryAttempt]
|
||||
type FailedAttempts = Vector[ConfirmablePersistentImpl]
|
||||
|
||||
case class DeliveryAttempt(persistent: ConfirmablePersistentImpl, destination: ActorPath, sender: ActorRef, timestamp: Long = System.nanoTime) {
|
||||
def incrementRedeliveryCount =
|
||||
copy(persistent.update(redeliveries = persistent.redeliveries + 1))
|
||||
}
|
||||
|
|
@ -371,40 +328,42 @@ private object ReliableDelivery {
|
|||
/**
|
||||
* Redelivery process used by [[ReliableDelivery]].
|
||||
*/
|
||||
private class Redelivery(channelSettings: ChannelSettings) extends Actor {
|
||||
private class Redelivery(redeliverSettings: ChannelSettings) extends Actor {
|
||||
import context.dispatcher
|
||||
import channelSettings._
|
||||
import redeliverSettings._
|
||||
import ReliableDelivery._
|
||||
|
||||
private var attempts: DeliveryAttempts = Map.empty
|
||||
private var schedule: Cancellable = _
|
||||
private var redeliveryAttempts: DeliveryAttempts = immutable.SortedMap.empty
|
||||
private var redeliverySchedule: Cancellable = _
|
||||
|
||||
def receive = {
|
||||
case Redeliver(as, max) ⇒
|
||||
attempts ++= as.map { case (k, a) ⇒ (k, a.withChannelEndpoint(self)) }
|
||||
attempts = attempts.foldLeft[DeliveryAttempts](Map.empty) {
|
||||
case (acc, (k, attempt)) ⇒
|
||||
// drop redelivery attempts that exceed redeliveryMax
|
||||
if (attempt.persistent.redeliveries >= redeliverMax) acc
|
||||
// increase redelivery count of attempt
|
||||
else acc + (k -> attempt.incrementRedeliveryCount)
|
||||
val (attempts, failed) = (redeliveryAttempts ++ as).foldLeft[(DeliveryAttempts, FailedAttempts)]((immutable.SortedMap.empty, Vector.empty)) {
|
||||
case ((attempts, failed), (k, attempt)) ⇒
|
||||
val persistent = attempt.persistent
|
||||
if (persistent.redeliveries >= redeliverMax) {
|
||||
(attempts, failed :+ persistent)
|
||||
} else {
|
||||
val updated = attempt.incrementRedeliveryCount
|
||||
context.actorSelection(updated.destination).tell(updated.persistent, updated.sender)
|
||||
(attempts.updated(k, updated), failed)
|
||||
|
||||
}
|
||||
}
|
||||
redeliver(attempts)
|
||||
redeliveryAttempts = attempts
|
||||
scheduleRedelivery()
|
||||
case c @ Confirm(processorId, messageSequenceNr, _, _, _) ⇒
|
||||
attempts -= ((processorId, messageSequenceNr))
|
||||
failed.headOption.foreach(_.confirmMessage.channel ! RedeliverFailure(failed))
|
||||
case c: Delivered ⇒
|
||||
redeliveryAttempts -= c.deliverySequenceNr
|
||||
}
|
||||
|
||||
override def preStart(): Unit =
|
||||
scheduleRedelivery()
|
||||
|
||||
override def postStop(): Unit =
|
||||
schedule.cancel()
|
||||
redeliverySchedule.cancel()
|
||||
|
||||
private def scheduleRedelivery(): Unit =
|
||||
schedule = context.system.scheduler.scheduleOnce(redeliverInterval, context.parent, Redeliver)
|
||||
|
||||
private def redeliver(attempts: DeliveryAttempts): Unit =
|
||||
attempts.values.toSeq.sortBy(_.deliverySequenceNr).foreach(ad ⇒ ad.destination tell (ad.persistent, ad.sender))
|
||||
redeliverySchedule = context.system.scheduler.scheduleOnce(redeliverInterval, context.parent, Redeliver)
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue