!str,htc replace and remove OneBoundedInterpreter

main work by @drewhk with contributions from @2m and @rkuhn

This work uncovered many well-hidden bugs in existing Stages, in
particular StatefulStage. These were hidden by the behavior of
OneBoundedInterpreter that normally behaves more orderly than it
guarantees in general, especially with respect to the timeliness of
delivery of upstream termination signals; the bugs were then that
internal state was not flushed when onComplete arrived “too early”.
This commit is contained in:
Roland Kuhn 2015-10-31 14:46:10 +01:00
parent 20f54435f1
commit 556012b7ee
107 changed files with 2456 additions and 3061 deletions

View file

@ -10,7 +10,7 @@ import akka.dispatch.Dispatchers
import akka.pattern.ask
import akka.stream.actor.ActorSubscriber
import akka.stream.impl.StreamLayout.Module
import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule, ActorInterpreter }
import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule }
import akka.stream.impl.io.SslTlsCipherActor
import akka.stream._
import akka.stream.io.SslTls.TlsModule
@ -24,14 +24,13 @@ import scala.concurrent.{ Await, ExecutionContextExecutor }
/**
* INTERNAL API
*/
private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
private[akka] case class ActorMaterializerImpl(system: ActorSystem,
override val settings: ActorMaterializerSettings,
dispatchers: Dispatchers,
val supervisor: ActorRef,
val haveShutDown: AtomicBoolean,
supervisor: ActorRef,
haveShutDown: AtomicBoolean,
flowNameCounter: AtomicLong,
namePrefix: String,
optimizations: Optimizations) extends ActorMaterializer {
namePrefix: String) extends ActorMaterializer {
import akka.stream.impl.Stages._
override def shutdown(): Unit =
@ -45,6 +44,12 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
private[this] def createFlowName(): String = s"$namePrefix-${nextFlowNameCount()}"
private val initialAttributes = Attributes(
Attributes.InputBuffer(settings.initialInputBufferSize, settings.maxInputBufferSize) ::
ActorAttributes.Dispatcher(settings.dispatcher) ::
ActorAttributes.SupervisionStrategy(settings.supervisionDecider) ::
Nil)
override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = {
import Attributes._
import ActorAttributes._
@ -70,7 +75,7 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
throw new IllegalStateException("Attempted to call materialize() after the ActorMaterializer has been shut down.")
if (StreamLayout.Debug) StreamLayout.validate(runnableGraph.module)
val session = new MaterializerSession(runnableGraph.module) {
val session = new MaterializerSession(runnableGraph.module, initialAttributes) {
private val flowName = createFlowName()
private var nextId = 0
private def stageName(attr: Attributes): String = {
@ -93,6 +98,7 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
assignPort(source.shape.outlet, pub.asInstanceOf[Publisher[Any]])
mat
// FIXME: Remove this, only stream-of-stream ops need it
case stage: StageModule
val (processor, mat) = processorFor(stage, effectiveAttributes, effectiveSettings(effectiveAttributes))
assignPort(stage.inPort, processor)
@ -118,7 +124,7 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
case graph: GraphModule
val calculatedSettings = effectiveSettings(effectiveAttributes)
val (inHandlers, outHandlers, logics, mat) = graph.assembly.materialize()
val (inHandlers, outHandlers, logics, mat) = graph.assembly.materialize(effectiveAttributes)
val props = ActorGraphInterpreter.props(
graph.assembly, inHandlers, outHandlers, logics, graph.shape, calculatedSettings, ActorMaterializerImpl.this)
@ -137,11 +143,11 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
}
}
// FIXME: Remove this, only stream-of-stream ops need it
private def processorFor(op: StageModule,
effectiveAttributes: Attributes,
effectiveSettings: ActorMaterializerSettings): (Processor[Any, Any], Any) = op match {
case DirectProcessor(processorFactory, _) processorFactory()
case Identity(attr) (new VirtualProcessor, ())
case _
val (opprops, mat) = ActorProcessorFactory.props(ActorMaterializerImpl.this, op, effectiveAttributes)
ActorProcessorFactory[Any, Any](
@ -248,47 +254,12 @@ private[akka] object ActorProcessorFactory {
// USE THIS TO AVOID CLOSING OVER THE MATERIALIZER BELOW
// Also, otherwise the attributes will not affect the settings properly!
val settings = materializer.effectiveSettings(att)
def interp(s: Stage[_, _]): (Props, Unit) = (ActorInterpreter.props(settings, List(s), materializer, att), ())
def interpAttr(s: Stage[_, _], newAttributes: Attributes): (Props, Unit) = (ActorInterpreter.props(settings, List(s), materializer, newAttributes), ())
def inputSizeAttr(n: Long) = {
if (n <= 0)
inputBuffer(initial = 1, max = 1) and att
else if (n <= materializer.settings.maxInputBufferSize)
inputBuffer(initial = n.toInt, max = n.toInt) and att
else
att
}
op match {
case Map(f, _) interp(fusing.Map(f, settings.supervisionDecider))
case Filter(p, _) interp(fusing.Filter(p, settings.supervisionDecider))
case Drop(n, _) interp(fusing.Drop(n))
case Take(n, _) interpAttr(fusing.Take(n), inputSizeAttr(n))
case TakeWhile(p, _) interp(fusing.TakeWhile(p, settings.supervisionDecider))
case DropWhile(p, _) interp(fusing.DropWhile(p, settings.supervisionDecider))
case Collect(pf, _) interp(fusing.Collect(pf, settings.supervisionDecider))
case Scan(z, f, _) interp(fusing.Scan(z, f, settings.supervisionDecider))
case Fold(z, f, _) interp(fusing.Fold(z, f, settings.supervisionDecider))
case Intersperse(s, i, e, _) interp(fusing.Intersperse(s, i, e))
case Recover(pf, _) interp(fusing.Recover(pf))
case Expand(s, f, _) interp(fusing.Expand(s, f))
case Conflate(s, f, _) interp(fusing.Conflate(s, f, settings.supervisionDecider))
case Buffer(n, s, _) interp(fusing.Buffer(n, s))
case MapConcat(f, _) interp(fusing.MapConcat(f, settings.supervisionDecider))
case MapAsync(p, f, _) interp(fusing.MapAsync(p, f, settings.supervisionDecider))
case MapAsyncUnordered(p, f, _) interp(fusing.MapAsyncUnordered(p, f, settings.supervisionDecider))
case Grouped(n, _) interp(fusing.Grouped(n))
case Sliding(n, step, _) interp(fusing.Sliding(n, step))
case Log(n, e, l, _) interp(fusing.Log(n, e, l))
case GroupBy(f, _) (GroupByProcessorImpl.props(settings, f), ())
case PrefixAndTail(n, _) (PrefixAndTailImpl.props(settings, n), ())
case Split(d, _) (SplitWhereProcessorImpl.props(settings, d), ())
case ConcatAll(_) (ConcatAllImpl.props(materializer), ())
case StageFactory(mkStage, _) interp(mkStage())
case MaterializingStageFactory(mkStageAndMat, _)
val s_m = mkStageAndMat()
(ActorInterpreter.props(settings, List(s_m._1), materializer, att), s_m._2)
case GroupBy(f, _) (GroupByProcessorImpl.props(settings, f), ())
case PrefixAndTail(n, _) (PrefixAndTailImpl.props(settings, n), ())
case Split(d, _) (SplitWhereProcessorImpl.props(settings, d), ())
case ConcatAll(_) (ConcatAllImpl.props(materializer), ())
case DirectProcessor(p, m) throw new AssertionError("DirectProcessor cannot end up in ActorProcessorFactory")
case Identity(_) throw new AssertionError("Identity cannot end up in ActorProcessorFactory")
}
}