!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:
Martin Krasser 2014-01-17 06:58:25 +01:00
parent 32b76adb9a
commit f327e1e357
55 changed files with 3474 additions and 2191 deletions

View file

@ -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)
}