=str #15152 operation transformations
This commit is contained in:
parent
92432cc8b3
commit
64feb041be
46 changed files with 877 additions and 373 deletions
|
|
@ -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] = {
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue