=str #15152 operation transformations

This commit is contained in:
Martynas Mickevicius 2014-12-01 20:07:55 +02:00
parent 92432cc8b3
commit 64feb041be
46 changed files with 877 additions and 373 deletions

View file

@ -8,6 +8,7 @@ import java.util.concurrent.atomic.AtomicLong
import akka.dispatch.Dispatchers
import akka.event.Logging
import akka.stream.impl.fusing.ActorInterpreter
import akka.stream.scaladsl.OperationAttributes._
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.{ Promise, ExecutionContext, Await, Future }
@ -25,82 +26,171 @@ import org.reactivestreams.{ Processor, Publisher, Subscriber }
* INTERNAL API
*/
private[akka] object Ast {
sealed abstract class AstNode {
def name: String
def attributes: OperationAttributes
def withAttributes(attributes: OperationAttributes): AstNode
}
final case class TimerTransform(mkStage: () TimerTransformer[Any, Any], override val name: String) extends AstNode
object Defaults {
val timerTransform = name("timerTransform")
val stageFactory = name("stageFactory")
val fused = name("fused")
val map = name("map")
val filter = name("filter")
val collect = name("collect")
val mapAsync = name("mapAsync")
val mapAsyncUnordered = name("mapAsyncUnordered")
val grouped = name("grouped")
val take = name("take")
val drop = name("drop")
val scan = name("scan")
val buffer = name("buffer")
val conflate = name("conflate")
val expand = name("expand")
val mapConcat = name("mapConcat")
val groupBy = name("groupBy")
val prefixAndTail = name("prefixAndTail")
val splitWhen = name("splitWhen")
val concatAll = name("concatAll")
val processor = name("processor")
val processorWithKey = name("processorWithKey")
val identityOp = name("identityOp")
final case class StageFactory(mkStage: () Stage[_, _], override val name: String) extends AstNode
val merge = name("merge")
val mergePreferred = name("mergePreferred")
val broadcast = name("broadcast")
val balance = name("balance")
val zip = name("zip")
val unzip = name("unzip")
val concat = name("concat")
val flexiMerge = name("flexiMerge")
val flexiRoute = name("flexiRoute")
val identityJunction = name("identityJunction")
}
import Defaults._
final case class TimerTransform(mkStage: () TimerTransformer[Any, Any], attributes: OperationAttributes = timerTransform) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class StageFactory(mkStage: () Stage[_, _], attributes: OperationAttributes = stageFactory) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
object Fused {
def apply(ops: immutable.Seq[Stage[_, _]]): Fused =
Fused(ops, ops.map(x Logging.simpleName(x).toLowerCase).mkString("+")) //FIXME change to something more performant for name
Fused(ops, name(ops.map(x Logging.simpleName(x).toLowerCase).mkString("+"))) //FIXME change to something more performant for name
}
final case class Fused(ops: immutable.Seq[Stage[_, _]], attributes: OperationAttributes = fused) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class Fused(ops: immutable.Seq[Stage[_, _]], override val name: String) extends AstNode
final case class Map(f: Any Any) extends AstNode { override def name = "map" }
final case class Map(f: Any Any, attributes: OperationAttributes = map) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class Filter(p: Any Boolean) extends AstNode { override def name = "filter" }
final case class Filter(p: Any Boolean, attributes: OperationAttributes = filter) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class Collect(pf: PartialFunction[Any, Any]) extends AstNode { override def name = "collect" }
final case class Collect(pf: PartialFunction[Any, Any], attributes: OperationAttributes = collect) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
// FIXME Replace with OperateAsync
final case class MapAsync(f: Any Future[Any]) extends AstNode { override def name = "mapAsync" }
final case class MapAsync(f: Any Future[Any], attributes: OperationAttributes = mapAsync) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
//FIXME Should be OperateUnorderedAsync
final case class MapAsyncUnordered(f: Any Future[Any]) extends AstNode { override def name = "mapAsyncUnordered" }
final case class MapAsyncUnordered(f: Any Future[Any], attributes: OperationAttributes = mapAsyncUnordered) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class Grouped(n: Int) extends AstNode {
final case class Grouped(n: Int, attributes: OperationAttributes = grouped) extends AstNode {
require(n > 0, "n must be greater than 0")
override def name = "grouped"
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
//FIXME should be `n: Long`
final case class Take(n: Int) extends AstNode {
override def name = "take"
final case class Take(n: Int, attributes: OperationAttributes = take) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
//FIXME should be `n: Long`
final case class Drop(n: Int) extends AstNode {
override def name = "drop"
final case class Drop(n: Int, attributes: OperationAttributes = drop) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class Scan(zero: Any, f: (Any, Any) Any) extends AstNode { override def name = "scan" }
final case class Scan(zero: Any, f: (Any, Any) Any, attributes: OperationAttributes = scan) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class Buffer(size: Int, overflowStrategy: OverflowStrategy) extends AstNode {
final case class Buffer(size: Int, overflowStrategy: OverflowStrategy, attributes: OperationAttributes = buffer) extends AstNode {
require(size > 0, s"Buffer size must be larger than zero but was [$size]")
override def name = "buffer"
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class Conflate(seed: Any Any, aggregate: (Any, Any) Any) extends AstNode {
override def name = "conflate"
final case class Conflate(seed: Any Any, aggregate: (Any, Any) Any, attributes: OperationAttributes = conflate) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class Expand(seed: Any Any, extrapolate: Any (Any, Any)) extends AstNode {
override def name = "expand"
final case class Expand(seed: Any Any, extrapolate: Any (Any, Any), attributes: OperationAttributes = expand) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class MapConcat(f: Any immutable.Seq[Any]) extends AstNode {
override def name = "mapConcat"
final case class MapConcat(f: Any immutable.Seq[Any], attributes: OperationAttributes = mapConcat) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class GroupBy(f: Any Any) extends AstNode { override def name = "groupBy" }
final case class PrefixAndTail(n: Int) extends AstNode { override def name = "prefixAndTail" }
final case class SplitWhen(p: Any Boolean) extends AstNode { override def name = "splitWhen" }
final case object ConcatAll extends AstNode {
override def name = "concatFlatten"
final case class GroupBy(f: Any Any, attributes: OperationAttributes = groupBy) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
case class DirectProcessor(p: () Processor[Any, Any]) extends AstNode {
override def name = "processor"
final case class PrefixAndTail(n: Int, attributes: OperationAttributes = prefixAndTail) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
case class DirectProcessorWithKey(p: () (Processor[Any, Any], Any), key: Key) extends AstNode {
override def name = "processorWithKey"
final case class SplitWhen(p: Any Boolean, attributes: OperationAttributes = splitWhen) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
final case class ConcatAll(attributes: OperationAttributes = concatAll) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
case class DirectProcessor(p: () Processor[Any, Any], attributes: OperationAttributes = processor) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
case class DirectProcessorWithKey(p: () (Processor[Any, Any], Any), key: Key, attributes: OperationAttributes = processorWithKey) extends AstNode {
def withAttributes(attributes: OperationAttributes) =
copy(attributes = attributes)
}
sealed trait JunctionAstNode {
def name: String
def attributes: OperationAttributes
}
// FIXME: Try to eliminate these
@ -108,46 +198,21 @@ private[akka] object Ast {
sealed trait FanOutAstNode extends JunctionAstNode
// FIXME Why do we need this?
case class IdentityAstNode(id: Int) extends JunctionAstNode {
override val name = s"identity$id"
}
case class IdentityAstNode(attributes: OperationAttributes) extends JunctionAstNode
case object Merge extends FanInAstNode {
override def name = "merge"
}
final case class Merge(attributes: OperationAttributes) extends FanInAstNode
final case class MergePreferred(attributes: OperationAttributes) extends FanInAstNode
case object MergePreferred extends FanInAstNode {
override def name = "mergePreferred"
}
final case class Broadcast(attributes: OperationAttributes) extends FanOutAstNode
final case class Balance(waitForAllDownstreams: Boolean, attributes: OperationAttributes) extends FanOutAstNode
case object Broadcast extends FanOutAstNode {
override def name = "broadcast"
}
final case class Zip(as: ZipAs, attributes: OperationAttributes) extends FanInAstNode
final case class Unzip(attributes: OperationAttributes) extends FanOutAstNode
case class Balance(waitForAllDownstreams: Boolean) extends FanOutAstNode {
override def name = "balance"
}
final case class Zip(as: ZipAs) extends FanInAstNode {
override def name = "zip"
}
case object Unzip extends FanOutAstNode {
override def name = "unzip"
}
case object Concat extends FanInAstNode {
override def name = "concat"
}
case class FlexiMergeNode(factory: FlexiMergeImpl.MergeLogicFactory[Any]) extends FanInAstNode {
override def name = factory.name.getOrElse("flexiMerge")
}
case class FlexiRouteNode(factory: FlexiRouteImpl.RouteLogicFactory[Any]) extends FanOutAstNode {
override def name = factory.name.getOrElse("flexiRoute")
}
final case class Concat(attributes: OperationAttributes) extends FanInAstNode
final case class FlexiMergeNode(factory: FlexiMergeImpl.MergeLogicFactory[Any], attributes: OperationAttributes) extends FanInAstNode
final case class FlexiRouteNode(factory: FlexiRouteImpl.RouteLogicFactory[Any], attributes: OperationAttributes) extends FanOutAstNode
}
/**
@ -216,12 +281,12 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
rest match {
case noMatch if !optimizations.elision || (noMatch ne orig) orig
//Collapses consecutive Take's into one
case (t1 @ Ast.Take(t1n)) :: (t2 @ Ast.Take(t2n)) :: rest (if (t1n < t2n) t1 else t2) :: rest
case (t1: Ast.Take) :: (t2: Ast.Take) :: rest (if (t1.n < t2.n) t1 else t2) :: rest
//Collapses consecutive Drop's into one
case (d1 @ Ast.Drop(d1n)) :: (d2 @ Ast.Drop(d2n)) :: rest new Ast.Drop(d1n + d2n) :: rest
case (d1: Ast.Drop) :: (d2: Ast.Drop) :: rest new Ast.Drop(d1.n + d2.n, d1.attributes and d2.attributes) :: rest
case Ast.Drop(n) :: rest if n < 1 rest // a 0 or negative drop is a NoOp
case Ast.Drop(n, _) :: rest if n < 1 rest // a 0 or negative drop is a NoOp
case noMatch noMatch
}
@ -231,7 +296,7 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
case noMatch if !optimizations.simplification || (noMatch ne orig) orig
// Two consecutive maps is equivalent to one pipelined map
case Ast.Map(second) :: Ast.Map(first) :: rest Ast.Map(first andThen second) :: rest
case (second: Ast.Map) :: (first: Ast.Map) :: rest Ast.Map(first.f andThen second.f, first.attributes and second.attributes) :: rest
case noMatch noMatch
}
@ -242,7 +307,7 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
case noMatch if !optimizations.collapsing || (noMatch ne orig) orig
// Collapses a filter and a map into a collect
case Ast.Map(f) :: Ast.Filter(p) :: rest Ast.Collect({ case i if p(i) f(i) }) :: rest
case (map: Ast.Map) :: (fil: Ast.Filter) :: rest Ast.Collect({ case i if fil.p(i) map.f(i) }) :: rest
case noMatch noMatch
}
@ -256,17 +321,17 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
// Optimizations below
case noMatch if !optimizations.fusion prev
case Ast.Map(f) fusing.Map(f) :: prev
case Ast.Filter(p) fusing.Filter(p) :: prev
case Ast.Drop(n) fusing.Drop(n) :: prev
case Ast.Take(n) fusing.Take(n) :: prev
case Ast.Collect(pf) fusing.Collect(pf) :: prev
case Ast.Scan(z, f) fusing.Scan(z, f) :: prev
case Ast.Expand(s, f) fusing.Expand(s, f) :: prev
case Ast.Conflate(s, f) fusing.Conflate(s, f) :: prev
case Ast.Buffer(n, s) fusing.Buffer(n, s) :: prev
case Ast.MapConcat(f) fusing.MapConcat(f) :: prev
case Ast.Grouped(n) fusing.Grouped(n) :: prev
case Ast.Map(f, _) fusing.Map(f) :: prev
case Ast.Filter(p, _) fusing.Filter(p) :: prev
case Ast.Drop(n, _) fusing.Drop(n) :: prev
case Ast.Take(n, _) fusing.Take(n) :: prev
case Ast.Collect(pf, _) fusing.Collect(pf) :: prev
case Ast.Scan(z, f, _) fusing.Scan(z, f) :: prev
case Ast.Expand(s, f, _) fusing.Expand(s, f) :: prev
case Ast.Conflate(s, f, _) fusing.Conflate(s, f) :: prev
case Ast.Buffer(n, s, _) fusing.Buffer(n, s) :: prev
case Ast.MapConcat(f, _) fusing.MapConcat(f) :: prev
case Ast.Grouped(n, _) fusing.Grouped(n) :: prev
//FIXME Add more fusion goodies here
case _ prev
}
@ -354,7 +419,7 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
}
}
//FIXME Should this be a dedicated AstNode?
private[this] val identityStageNode = Ast.StageFactory(() FlowOps.identityStage[Any], "identity")
private[this] val identityStageNode = Ast.StageFactory(() FlowOps.identityStage[Any], Ast.Defaults.identityOp)
def executionContext: ExecutionContext = dispatchers.lookup(settings.dispatcher match {
case Deploy.NoDispatcherGiven Dispatchers.DefaultDispatcherId
@ -366,23 +431,32 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
*/
private[akka] def processorForNode[In, Out](op: AstNode, flowName: String, n: Int): (Processor[In, Out], MaterializedMap) = op match {
// FIXME #16376 should probably be replaced with an ActorFlowProcessor similar to ActorFlowSource/Sink
case Ast.DirectProcessor(p) (p().asInstanceOf[Processor[In, Out]], MaterializedMap.empty)
case Ast.DirectProcessorWithKey(p, key)
case Ast.DirectProcessor(p, _) (p().asInstanceOf[Processor[In, Out]], MaterializedMap.empty)
case Ast.DirectProcessorWithKey(p, key, _)
val (processor, value) = p()
(processor.asInstanceOf[Processor[In, Out]], MaterializedMap.empty.updated(key, value))
case _
(ActorProcessorFactory[In, Out](actorOf(ActorProcessorFactory.props(this, op), s"$flowName-$n-${op.name}")), MaterializedMap.empty)
(ActorProcessorFactory[In, Out](actorOf(ActorProcessorFactory.props(this, op), s"$flowName-$n-${op.attributes.name}", op)), MaterializedMap.empty)
}
def actorOf(props: Props, name: String): ActorRef = supervisor match {
private[akka] def actorOf(props: Props, name: String): ActorRef =
actorOf(props, name, settings.dispatcher)
private[akka] def actorOf(props: Props, name: String, ast: Ast.JunctionAstNode): ActorRef =
actorOf(props, name, ast.attributes.settings(settings).dispatcher)
private[akka] def actorOf(props: Props, name: String, ast: AstNode): ActorRef =
actorOf(props, name, ast.attributes.settings(settings).dispatcher)
private[akka] def actorOf(props: Props, name: String, dispatcher: String): ActorRef = supervisor match {
case ref: LocalActorRef
ref.underlying.attachChild(props.withDispatcher(settings.dispatcher), name, systemService = false)
ref.underlying.attachChild(props.withDispatcher(dispatcher), name, systemService = false)
case ref: RepointableActorRef
if (ref.isStarted)
ref.underlying.asInstanceOf[ActorCell].attachChild(props.withDispatcher(settings.dispatcher), name, systemService = false)
ref.underlying.asInstanceOf[ActorCell].attachChild(props.withDispatcher(dispatcher), name, systemService = false)
else {
implicit val timeout = ref.system.settings.CreationTimeout
val f = (supervisor ? StreamSupervisor.Materialize(props.withDispatcher(settings.dispatcher), name)).mapTo[ActorRef]
val f = (supervisor ? StreamSupervisor.Materialize(props.withDispatcher(dispatcher), name)).mapTo[ActorRef]
Await.result(f, timeout.duration)
}
case unknown
@ -390,17 +464,20 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
}
// FIXME Investigate possibility of using `enableOperationsFusion` in `materializeJunction`
override def materializeJunction[In, Out](op: Ast.JunctionAstNode, inputCount: Int, outputCount: Int): (immutable.Seq[Subscriber[In]], immutable.Seq[Publisher[Out]]) = {
val actorName = s"${createFlowName()}-${op.name}"
val actorName = s"${createFlowName()}-${op.attributes.name}"
val transformedSettings = op.attributes.settings(settings)
op match {
case fanin: Ast.FanInAstNode
val impl = fanin match {
case Ast.Merge actorOf(FairMerge.props(settings, inputCount), actorName)
case Ast.MergePreferred actorOf(UnfairMerge.props(settings, inputCount), actorName)
case zip: Ast.Zip actorOf(Zip.props(settings, zip.as), actorName)
case Ast.Concat actorOf(Concat.props(settings), actorName)
case Ast.FlexiMergeNode(merger) actorOf(FlexiMergeImpl.props(settings, inputCount, merger.createMergeLogic()), actorName)
val props = fanin match {
case Ast.Merge(_) FairMerge.props(transformedSettings, inputCount)
case Ast.MergePreferred(_) UnfairMerge.props(transformedSettings, inputCount)
case Ast.Zip(as, _) Zip.props(transformedSettings, as)
case Ast.Concat(_) Concat.props(transformedSettings)
case Ast.FlexiMergeNode(merger, _) FlexiMergeImpl.props(transformedSettings, inputCount, merger.createMergeLogic())
}
val impl = actorOf(props, actorName, fanin)
val publisher = new ActorPublisher[Out](impl)
impl ! ExposedPublisher(publisher.asInstanceOf[ActorPublisher[Any]])
@ -408,12 +485,13 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
(subscribers, List(publisher))
case fanout: Ast.FanOutAstNode
val impl = fanout match {
case Ast.Broadcast actorOf(Broadcast.props(settings, outputCount), actorName)
case Ast.Balance(waitForAllDownstreams) actorOf(Balance.props(settings, outputCount, waitForAllDownstreams), actorName)
case Ast.Unzip actorOf(Unzip.props(settings), actorName)
case Ast.FlexiRouteNode(route) actorOf(FlexiRouteImpl.props(settings, outputCount, route.createRouteLogic()), actorName)
val props = fanout match {
case Ast.Broadcast(_) Broadcast.props(transformedSettings, outputCount)
case Ast.Balance(waitForAllDownstreams, _) Balance.props(transformedSettings, outputCount, waitForAllDownstreams)
case Ast.Unzip(_) Unzip.props(transformedSettings)
case Ast.FlexiRouteNode(route, _) FlexiRouteImpl.props(transformedSettings, outputCount, route.createRouteLogic())
}
val impl = actorOf(props, actorName, fanout)
val publishers = Vector.tabulate(outputCount)(id new ActorPublisher[Out](impl) {
override val wakeUpMsg = FanOut.SubstreamSubscribePending(id)
@ -422,9 +500,9 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
val subscriber = ActorSubscriber[In](impl)
(List(subscriber), publishers)
case identity @ Ast.IdentityAstNode(_) // FIXME Why is IdentityAstNode a JunctionAStNode?
case identity @ Ast.IdentityAstNode(attr) // FIXME Why is IdentityAstNode a JunctionAStNode?
// We can safely ignore the materialized map that gets created here since it will be empty
val id = List(processorForNode[In, Out](identityStageNode, identity.name, 1)._1) // FIXME is `identity.name` appropriate/unique here?
val id = List(processorForNode[In, Out](identityStageNode, attr.name, 1)._1) // FIXME is `identity.name` appropriate/unique here?
(id, id)
}
@ -477,28 +555,28 @@ private[akka] object ActorProcessorFactory {
import Ast._
def props(materializer: FlowMaterializer, op: AstNode): Props = {
val settings = materializer.settings // USE THIS TO AVOID CLOSING OVER THE MATERIALIZER BELOW
(op match {
op match {
case Fused(ops, _) ActorInterpreter.props(settings, ops)
case Map(f) ActorInterpreter.props(settings, List(fusing.Map(f)))
case Filter(p) ActorInterpreter.props(settings, List(fusing.Filter(p)))
case Drop(n) ActorInterpreter.props(settings, List(fusing.Drop(n)))
case Take(n) ActorInterpreter.props(settings, List(fusing.Take(n)))
case Collect(pf) ActorInterpreter.props(settings, List(fusing.Collect(pf)))
case Scan(z, f) ActorInterpreter.props(settings, List(fusing.Scan(z, f)))
case Expand(s, f) ActorInterpreter.props(settings, List(fusing.Expand(s, f)))
case Conflate(s, f) ActorInterpreter.props(settings, List(fusing.Conflate(s, f)))
case Buffer(n, s) ActorInterpreter.props(settings, List(fusing.Buffer(n, s)))
case MapConcat(f) ActorInterpreter.props(settings, List(fusing.MapConcat(f)))
case MapAsync(f) MapAsyncProcessorImpl.props(settings, f)
case MapAsyncUnordered(f) MapAsyncUnorderedProcessorImpl.props(settings, f)
case Grouped(n) ActorInterpreter.props(settings, List(fusing.Grouped(n)))
case GroupBy(f) GroupByProcessorImpl.props(settings, f)
case PrefixAndTail(n) PrefixAndTailImpl.props(settings, n)
case SplitWhen(p) SplitWhenProcessorImpl.props(settings, p)
case ConcatAll ConcatAllImpl.props(materializer) //FIXME closes over the materializer, is this good?
case Map(f, _) ActorInterpreter.props(settings, List(fusing.Map(f)))
case Filter(p, _) ActorInterpreter.props(settings, List(fusing.Filter(p)))
case Drop(n, _) ActorInterpreter.props(settings, List(fusing.Drop(n)))
case Take(n, _) ActorInterpreter.props(settings, List(fusing.Take(n)))
case Collect(pf, _) ActorInterpreter.props(settings, List(fusing.Collect(pf)))
case Scan(z, f, _) ActorInterpreter.props(settings, List(fusing.Scan(z, f)))
case Expand(s, f, _) ActorInterpreter.props(settings, List(fusing.Expand(s, f)))
case Conflate(s, f, _) ActorInterpreter.props(settings, List(fusing.Conflate(s, f)))
case Buffer(n, s, _) ActorInterpreter.props(settings, List(fusing.Buffer(n, s)))
case MapConcat(f, _) ActorInterpreter.props(settings, List(fusing.MapConcat(f)))
case MapAsync(f, _) MapAsyncProcessorImpl.props(settings, f)
case MapAsyncUnordered(f, _) MapAsyncUnorderedProcessorImpl.props(settings, f)
case Grouped(n, _) ActorInterpreter.props(settings, List(fusing.Grouped(n)))
case GroupBy(f, _) GroupByProcessorImpl.props(settings, f)
case PrefixAndTail(n, _) PrefixAndTailImpl.props(settings, n)
case SplitWhen(p, _) SplitWhenProcessorImpl.props(settings, p)
case ConcatAll(_) ConcatAllImpl.props(materializer) //FIXME closes over the materializer, is this good?
case StageFactory(mkStage, _) ActorInterpreter.props(settings, List(mkStage()))
case TimerTransform(mkStage, _) TimerTransformerProcessorsImpl.props(settings, mkStage())
}).withDispatcher(settings.dispatcher)
}
}
def apply[I, O](impl: ActorRef): ActorProcessor[I, O] = {