pekko/akka-persistence/src/test/scala/akka/persistence/PersistentActorFailureSpec.scala
Patrik Nordwall 8c47e01e9d !per #15377 Mandate atomic writes for persistAll, and support rejections
* changing Plugin API for asyncWriteMessages and writeMessages
* passing explicit AtomicWrite that represents the events of
  persistAll, or a single event from persist
* journal may reject events before storing them, and that
  will result in onPersistRejected (logging) and continue in the
  persistent actor
* clarified the semantics with regards to batches and atomic writes,
  and failures and rejections in the api docs of asyncWriteMessages
  and writeMessages
* adjust the Java plugin API, asyncReplayMessages, doLoadAsync
2015-06-25 15:28:37 +02:00

265 lines
9.4 KiB
Scala

/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.persistence
import scala.collection.immutable
import akka.actor.{ OneForOneStrategy, _ }
import akka.persistence.journal.AsyncWriteProxy
import akka.persistence.journal.AsyncWriteTarget.{ ReplayFailure, ReplayMessages, ReplaySuccess, WriteMessages }
import akka.persistence.journal.inmem.InmemStore
import akka.testkit.{ ImplicitSender, TestProbe }
import akka.util.Timeout
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.control.NoStackTrace
import scala.util.Try
import akka.persistence.journal.AsyncWriteJournal
import scala.util.Failure
object PersistentActorFailureSpec {
import PersistentActorSpec.{ Cmd, Evt, ExamplePersistentActor }
class SimulatedException(msg: String) extends RuntimeException(msg) with NoStackTrace
class SimulatedSerializationException(msg: String) extends RuntimeException(msg) with NoStackTrace
class FailingInmemJournal extends AsyncWriteProxy {
import AsyncWriteProxy.SetStore
val timeout = Timeout(3 seconds)
override def preStart(): Unit = {
super.preStart()
self ! SetStore(context.actorOf(Props[FailingInmemStore]))
}
}
class FailingInmemStore extends InmemStore {
def failingReceive: Receive = {
case w: WriteMessages if isWrong(w)
throw new SimulatedException("Simulated Store failure")
case w: WriteMessages if checkSerializable(w).exists(_.isFailure)
sender() ! checkSerializable(w)
case ReplayMessages(pid, fromSnr, toSnr, max)
val readFromStore = read(pid, fromSnr, toSnr, max)
if (readFromStore.length == 0)
sender() ! ReplaySuccess
else if (isCorrupt(readFromStore))
sender() ! ReplayFailure(new SimulatedException(s"blahonga $fromSnr $toSnr"))
else {
readFromStore.foreach(sender() ! _)
sender() ! ReplaySuccess
}
}
def isWrong(w: WriteMessages): Boolean =
w.messages.exists {
case a: AtomicWrite
a.payload.exists { case PersistentRepr(Evt(s: String), _) s.contains("wrong") }
case _ false
}
def checkSerializable(w: WriteMessages): immutable.Seq[Try[Unit]] =
w.messages.collect {
case a: AtomicWrite
(a.payload.collectFirst {
case PersistentRepr(Evt(s: String), _) if s.contains("not serializable") s
}) match {
case Some(s) Failure(new SimulatedSerializationException(s))
case None AsyncWriteJournal.successUnit
}
}
def isCorrupt(events: Seq[PersistentRepr]): Boolean =
events.exists { case PersistentRepr(Evt(s: String), _) s.contains("corrupt") }
override def receive = failingReceive.orElse(super.receive)
}
class Supervisor(testActor: ActorRef) extends Actor {
override def supervisorStrategy = OneForOneStrategy(loggingEnabled = false) {
case e
testActor ! e
SupervisorStrategy.Restart
}
def receive = {
case props: Props sender() ! context.actorOf(props)
case m sender() ! m
}
}
class ResumingSupervisor(testActor: ActorRef) extends Supervisor(testActor) {
override def supervisorStrategy = OneForOneStrategy(loggingEnabled = false) {
case e
testActor ! e
SupervisorStrategy.Resume
}
}
class FailingRecovery(name: String, recoveryFailureProbe: Option[ActorRef]) extends ExamplePersistentActor(name) {
def this(name: String) = this(name, None)
override val receiveCommand: Receive = commonBehavior orElse {
case Cmd(data) persist(Evt(s"${data}"))(updateState)
}
val failingRecover: Receive = {
case Evt(data) if data == "bad"
throw new SimulatedException("Simulated exception from receiveRecover")
}
override def receiveRecover: Receive = failingRecover.orElse[Any, Unit](super.receiveRecover)
}
class ThrowingActor1(name: String) extends ExamplePersistentActor(name) {
override val receiveCommand: Receive = commonBehavior orElse {
case Cmd(data)
persist(Evt(s"${data}"))(updateState)
if (data == "err")
throw new SimulatedException("Simulated exception 1")
}
}
class ThrowingActor2(name: String) extends ExamplePersistentActor(name) {
override val receiveCommand: Receive = commonBehavior orElse {
case Cmd(data)
persist(Evt(s"${data}")) { evt
if (data == "err")
throw new SimulatedException("Simulated exception 1")
updateState(evt)
}
}
}
}
class PersistentActorFailureSpec extends PersistenceSpec(PersistenceSpec.config("inmem", "SnapshotFailureRobustnessSpec", extraConfig = Some(
"""
akka.persistence.journal.inmem.class = "akka.persistence.PersistentActorFailureSpec$FailingInmemJournal"
"""))) with ImplicitSender {
import PersistentActorFailureSpec._
import PersistentActorSpec._
def prepareFailingRecovery(): Unit = {
val persistentActor = namedPersistentActor[FailingRecovery]
persistentActor ! Cmd("a")
persistentActor ! Cmd("b")
persistentActor ! Cmd("bad")
persistentActor ! Cmd("c")
persistentActor ! GetState
expectMsg(List("a", "b", "bad", "c"))
}
"A persistent actor" must {
"stop if recovery from persisted events fail" in {
val persistentActor = namedPersistentActor[Behavior1PersistentActor]
persistentActor ! Cmd("corrupt")
persistentActor ! GetState
expectMsg(List("corrupt-1", "corrupt-2"))
// recover by creating another with same name
system.actorOf(Props(classOf[Supervisor], testActor)) ! Props(classOf[Behavior1PersistentActor], name)
val ref = expectMsgType[ActorRef]
watch(ref)
expectTerminated(ref)
}
"stop if persist fails" in {
system.actorOf(Props(classOf[Supervisor], testActor)) ! Props(classOf[Behavior1PersistentActor], name)
val persistentActor = expectMsgType[ActorRef]
watch(persistentActor)
persistentActor ! Cmd("wrong")
expectTerminated(persistentActor)
}
"stop if persistAsync fails" in {
system.actorOf(Props(classOf[Supervisor], testActor)) ! Props(classOf[AsyncPersistPersistentActor], name)
val persistentActor = expectMsgType[ActorRef]
persistentActor ! Cmd("a")
watch(persistentActor)
expectMsg("a") // reply before persistAsync
expectMsg("a-1") // reply after successful persistAsync
persistentActor ! Cmd("wrong")
expectMsg("wrong") // reply before persistAsync
expectTerminated(persistentActor)
}
"call onPersistRejected and continue if persist rejected" in {
system.actorOf(Props(classOf[Supervisor], testActor)) ! Props(classOf[Behavior1PersistentActor], name)
val persistentActor = expectMsgType[ActorRef]
persistentActor ! Cmd("not serializable")
expectMsg("Rejected: not serializable-1")
expectMsg("Rejected: not serializable-2")
persistentActor ! Cmd("a")
persistentActor ! GetState
expectMsg(List("a-1", "a-2"))
}
"stop if receiveRecover fails" in {
prepareFailingRecovery()
// recover by creating another with same name
system.actorOf(Props(classOf[Supervisor], testActor)) ! Props(classOf[FailingRecovery], name)
val ref = expectMsgType[ActorRef]
watch(ref)
expectTerminated(ref)
}
"support resume when persist followed by exception" in {
system.actorOf(Props(classOf[ResumingSupervisor], testActor)) ! Props(classOf[ThrowingActor1], name)
val persistentActor = expectMsgType[ActorRef]
persistentActor ! Cmd("a")
persistentActor ! Cmd("err")
persistentActor ! Cmd("b")
expectMsgType[SimulatedException] // from supervisor
persistentActor ! Cmd("c")
persistentActor ! GetState
expectMsg(List("a", "err", "b", "c"))
}
"support restart when persist followed by exception" in {
system.actorOf(Props(classOf[Supervisor], testActor)) ! Props(classOf[ThrowingActor1], name)
val persistentActor = expectMsgType[ActorRef]
persistentActor ! Cmd("a")
persistentActor ! Cmd("err")
persistentActor ! Cmd("b")
expectMsgType[SimulatedException] // from supervisor
persistentActor ! Cmd("c")
persistentActor ! GetState
expectMsg(List("a", "err", "b", "c"))
}
"support resume when persist handler throws exception" in {
system.actorOf(Props(classOf[ResumingSupervisor], testActor)) ! Props(classOf[ThrowingActor2], name)
val persistentActor = expectMsgType[ActorRef]
persistentActor ! Cmd("a")
persistentActor ! Cmd("b")
persistentActor ! Cmd("err")
persistentActor ! Cmd("c")
expectMsgType[SimulatedException] // from supervisor
persistentActor ! Cmd("d")
persistentActor ! GetState
expectMsg(List("a", "b", "c", "d"))
}
"support restart when persist handler throws exception" in {
system.actorOf(Props(classOf[Supervisor], testActor)) ! Props(classOf[ThrowingActor2], name)
val persistentActor = expectMsgType[ActorRef]
persistentActor ! Cmd("a")
persistentActor ! Cmd("b")
persistentActor ! Cmd("err")
persistentActor ! Cmd("c")
expectMsgType[SimulatedException] // from supervisor
persistentActor ! Cmd("d")
persistentActor ! GetState
// err was stored, and was be replayed
expectMsg(List("a", "b", "err", "c", "d"))
}
}
}