Snapshot when stages stopped incorrect and can cause crash #26902

This commit is contained in:
Johan Andrén 2019-05-20 13:56:23 +02:00 committed by GitHub
parent a7c3e2b014
commit 7c6025b06a
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
4 changed files with 123 additions and 23 deletions

View file

@ -4,8 +4,9 @@
package akka.stream.snapshot package akka.stream.snapshot
import akka.stream.ActorMaterializer import akka.stream.{ ActorMaterializer, FlowShape }
import akka.stream.scaladsl.{ Sink, Source } import akka.stream.scaladsl.{ Flow, GraphDSL, Keep, Merge, Partition, Sink, Source }
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.StreamSpec import akka.stream.testkit.StreamSpec
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -16,6 +17,7 @@ class MaterializerStateSpec extends StreamSpec {
"snapshot a running stream" in { "snapshot a running stream" in {
implicit val mat = ActorMaterializer() implicit val mat = ActorMaterializer()
try {
Source.maybe[Int].map(_.toString).zipWithIndex.runWith(Sink.seq) Source.maybe[Int].map(_.toString).zipWithIndex.runWith(Sink.seq)
awaitAssert({ awaitAssert({
@ -24,8 +26,76 @@ class MaterializerStateSpec extends StreamSpec {
snapshot should have size (1) snapshot should have size (1)
snapshot.head.activeInterpreters should have size (1) snapshot.head.activeInterpreters should have size (1)
snapshot.head.activeInterpreters.head.logics should have size (4) // all 4 operators snapshot.head.activeInterpreters.head.logics should have size (4) // all 4 operators
}, 3.seconds) }, remainingOrDefault)
} finally {
mat.shutdown()
} }
} }
"snapshot a stream that has a stopped stage" in {
implicit val mat = ActorMaterializer()
try {
val probe = TestSink.probe[String](system)
val out = Source
.single("one")
.concat(Source.maybe[String]) // make sure we leave it running
.runWith(probe)
out.requestNext("one")
awaitAssert({
val snapshot = MaterializerState.streamSnapshots(mat).futureValue
snapshot should have size (1)
snapshot.head.activeInterpreters should have size (1)
snapshot.head.activeInterpreters.head.stoppedLogics should have size (2) // Source.single and a detach
}, remainingOrDefault)
} finally {
mat.shutdown()
}
}
"snapshot a more complicated graph" in {
implicit val mat = ActorMaterializer()
try {
// snapshot before anything is running
MaterializerState.streamSnapshots(mat).futureValue
val graph = Flow.fromGraph(GraphDSL.create() { implicit b =>
import GraphDSL.Implicits._
val partition = b.add(Partition[String](4, {
case "green" => 0
case "red" => 1
case "blue" => 2
case _ => 3
}))
val merge = b.add(Merge[String](4, eagerComplete = false))
val discard = b.add(Sink.ignore.async)
val one = b.add(Source.single("purple"))
partition.out(0) ~> merge.in(0)
partition.out(1).via(Flow[String].map(_.toUpperCase()).async) ~> merge.in(1)
partition.out(2).groupBy(2, identity).mergeSubstreams ~> merge.in(2)
partition.out(3) ~> discard
one ~> merge.in(3)
FlowShape(partition.in, merge.out)
})
val callMeMaybe =
Source.maybe[String].viaMat(graph)(Keep.left).toMat(Sink.ignore)(Keep.left).run()
// just check that we can snapshot without errors
MaterializerState.streamSnapshots(mat).futureValue
callMeMaybe.success(Some("green"))
MaterializerState.streamSnapshots(mat).futureValue
Thread.sleep(100) // just to give it a bigger chance to cover different states of shutting down
MaterializerState.streamSnapshots(mat).futureValue
} finally {
mat.shutdown()
}
}
}
} }

View file

@ -10,13 +10,15 @@ import akka.actor._
import akka.annotation.{ DoNotInherit, InternalApi } import akka.annotation.{ DoNotInherit, InternalApi }
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.pattern.ask import akka.pattern.{ ask, pipe, retry }
import akka.stream._ import akka.stream._
import akka.stream.impl.fusing.GraphInterpreterShell import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphInterpreterShell }
import akka.util.OptionVal import akka.stream.snapshot.StreamSnapshot
import akka.util.{ OptionVal, Timeout }
import scala.concurrent.duration.FiniteDuration import scala.collection.immutable
import scala.concurrent.{ Await, ExecutionContextExecutor } import scala.concurrent.duration._
import scala.concurrent.{ Await, ExecutionContextExecutor, Future }
/** /**
* ExtendedActorMaterializer used by subtypes which delegates in-island wiring to [[akka.stream.impl.PhaseIsland]]s * ExtendedActorMaterializer used by subtypes which delegates in-island wiring to [[akka.stream.impl.PhaseIsland]]s
@ -177,6 +179,11 @@ private[akka] class SubFusingActorMaterializerImpl(
extends DeadLetterSuppression extends DeadLetterSuppression
with NoSerializationVerificationNeeded with NoSerializationVerificationNeeded
case object GetChildrenSnapshots
final case class ChildrenSnapshots(seq: immutable.Seq[StreamSnapshot])
extends DeadLetterSuppression
with NoSerializationVerificationNeeded
/** Testing purpose */ /** Testing purpose */
case object GetChildren case object GetChildren
@ -195,7 +202,7 @@ private[akka] class SubFusingActorMaterializerImpl(
*/ */
@InternalApi private[akka] class StreamSupervisor(haveShutDown: AtomicBoolean) extends Actor { @InternalApi private[akka] class StreamSupervisor(haveShutDown: AtomicBoolean) extends Actor {
import akka.stream.impl.StreamSupervisor._ import akka.stream.impl.StreamSupervisor._
implicit val ec = context.dispatcher
override def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.stoppingStrategy override def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.stoppingStrategy
def receive = { def receive = {
@ -209,10 +216,29 @@ private[akka] class SubFusingActorMaterializerImpl(
context.asInstanceOf[ActorCell].removeFunctionRef(ref) context.asInstanceOf[ActorCell].removeFunctionRef(ref)
case GetChildren => case GetChildren =>
sender() ! Children(context.children.toSet) sender() ! Children(context.children.toSet)
case GetChildrenSnapshots =>
takeSnapshotsOfChildren().map(ChildrenSnapshots.apply _).pipeTo(sender())
case StopChildren => case StopChildren =>
context.children.foreach(context.stop) context.children.foreach(context.stop)
sender() ! StoppedChildren sender() ! StoppedChildren
} }
def takeSnapshotsOfChildren(): Future[immutable.Seq[StreamSnapshot]] = {
implicit val scheduler = context.system.scheduler
// Arbitrary timeout but should always be quick, the failure scenario is that
// the child/stream stopped, and we do retry below
implicit val timeout: Timeout = 1.second
def takeSnapshot() = {
val futureSnapshots =
context.children.toList.map(child => (child ? ActorGraphInterpreter.Snapshot).mapTo[StreamSnapshot])
Future.sequence(futureSnapshots)
}
// If the timeout hits it is likely because one of the streams stopped between looking at the list
// of children and asking it for a snapshot. We retry the entire snapshot in that case
retry(() => takeSnapshot(), 3, Duration.Zero)
}
override def postStop(): Unit = haveShutDown.set(true) override def postStop(): Unit = haveShutDown.set(true)
} }

View file

@ -680,12 +680,15 @@ import akka.stream.snapshot._
}) })
} }
val stoppedStages: List[LogicSnapshot] = shutdownCounter.zipWithIndex.collect {
case (activeConnections, idx) if activeConnections < 1 => logicSnapshots(idx)
}.toList
RunningInterpreterImpl( RunningInterpreterImpl(
logicSnapshots.toVector, logicSnapshots.toVector,
connectionSnapshots.toVector, connectionSnapshots.toVector,
queueStatus, queueStatus,
runningStages, runningStages,
shutdownCounter.toList.map(n => logicSnapshots(n))) stoppedStages)
} }
} }

View file

@ -8,8 +8,8 @@ import akka.actor.{ ActorPath, ActorRef }
import akka.annotation.{ ApiMayChange, DoNotInherit, InternalApi } import akka.annotation.{ ApiMayChange, DoNotInherit, InternalApi }
import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
import akka.pattern.ask import akka.pattern.ask
import akka.stream.{ Attributes, Materializer }
import akka.stream.impl.fusing.ActorGraphInterpreter import akka.stream.impl.fusing.ActorGraphInterpreter
import akka.stream.{ Attributes, Materializer }
import akka.util.Timeout import akka.util.Timeout
import scala.collection.immutable import scala.collection.immutable
@ -41,11 +41,9 @@ object MaterializerState {
@InternalApi @InternalApi
private[akka] def requestFromSupervisor(supervisor: ActorRef)( private[akka] def requestFromSupervisor(supervisor: ActorRef)(
implicit ec: ExecutionContext): Future[immutable.Seq[StreamSnapshot]] = { implicit ec: ExecutionContext): Future[immutable.Seq[StreamSnapshot]] = {
// FIXME arbitrary timeout // Arbitrary timeout: operation should always be quick, when it times out it will be because the materializer stopped
implicit val timeout: Timeout = 10.seconds implicit val timeout: Timeout = 10.seconds
(supervisor ? StreamSupervisor.GetChildren) (supervisor ? StreamSupervisor.GetChildrenSnapshots).mapTo[StreamSupervisor.ChildrenSnapshots].map(_.seq)
.mapTo[StreamSupervisor.Children]
.flatMap(msg => Future.sequence(msg.children.toVector.map(requestFromChild)))
} }
/** INTERNAL API */ /** INTERNAL API */
@ -188,7 +186,10 @@ private[akka] final case class RunningInterpreterImpl(
@InternalApi @InternalApi
private[akka] final case class LogicSnapshotImpl(index: Int, label: String, attributes: Attributes) private[akka] final case class LogicSnapshotImpl(index: Int, label: String, attributes: Attributes)
extends LogicSnapshot extends LogicSnapshot
with HideImpl with HideImpl {
override def toString: String = s"Logic($label)"
}
/** /**
* INTERNAL API * INTERNAL API