Revert "=str #16280 Make flows sharable again" since the problem is solved differently in this branch.

This reverts commit 2af568eb7a0c8d65f3dd2bb04852d05d5cbda416.
This commit is contained in:
Viktor Klang 2014-11-12 14:26:10 +01:00
parent af3d9860dd
commit efe9331b69
4 changed files with 21 additions and 26 deletions

View file

@ -22,7 +22,7 @@ class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet() val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet()
val processor = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode( val processor = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode(
Ast.OpFactory(() akka.stream.impl.fusing.Map[Int, Int](identity), "identity"), flowName, 1) Ast.Fusable(Vector(akka.stream.impl.fusing.Map[Int, Int](identity)), "identity"), flowName, 1)
processor.asInstanceOf[Processor[Int, Int]] processor.asInstanceOf[Processor[Int, Int]]
} }

View file

@ -14,7 +14,7 @@ import akka.actor.{ Props, ActorRefFactory, ActorRef }
import akka.stream.{ TransformerLike, MaterializerSettings } import akka.stream.{ TransformerLike, MaterializerSettings }
import akka.stream.FlowMaterializer import akka.stream.FlowMaterializer
import akka.stream.impl.{ ActorProcessorFactory, TransformProcessorImpl, StreamSupervisor, ActorBasedFlowMaterializer } import akka.stream.impl.{ ActorProcessorFactory, TransformProcessorImpl, StreamSupervisor, ActorBasedFlowMaterializer }
import akka.stream.impl.Ast.{ Transform, OpFactory, AstNode } import akka.stream.impl.Ast.{ Transform, Fusable, AstNode }
import akka.stream.testkit.{ StreamTestKit, AkkaSpec } import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit.ChainSetup import akka.stream.testkit.ChainSetup
import akka.testkit._ import akka.testkit._
@ -73,7 +73,7 @@ object FlowSpec {
override def processorForNode(op: AstNode, flowName: String, n: Int): Processor[Any, Any] = { override def processorForNode(op: AstNode, flowName: String, n: Int): Processor[Any, Any] = {
val props = op match { val props = op match {
case t: Transform Props(new BrokenTransformProcessorImpl(settings, t.mkTransformer(), brokenMessage)) case t: Transform Props(new BrokenTransformProcessorImpl(settings, t.mkTransformer(), brokenMessage))
case OpFactory(mkOps, _) Props(new BrokenActorInterpreter(settings, mkOps.map(_.apply()), brokenMessage)).withDispatcher(settings.dispatcher) case f: Fusable Props(new BrokenActorInterpreter(settings, f.ops, brokenMessage)).withDispatcher(settings.dispatcher)
case o ActorProcessorFactory.props(this, o) case o ActorProcessorFactory.props(this, o)
} }
val impl = actorOf(props, s"$flowName-$n-${op.name}") val impl = actorOf(props, s"$flowName-$n-${op.name}")

View file

@ -43,12 +43,7 @@ private[akka] object Ast {
case class TimerTransform(name: String, mkTransformer: () TimerTransformer[Any, Any]) extends AstNode case class TimerTransform(name: String, mkTransformer: () TimerTransformer[Any, Any]) extends AstNode
object OpFactory { case class Fusable(ops: immutable.Seq[Op[_, _, _, _, _]], name: String) extends AstNode
def apply(mkOp: () Op[_, _, _, _, _], name: String): OpFactory =
OpFactory(List(mkOp), name)
}
case class OpFactory(mkOps: List[() Op[_, _, _, _, _]], name: String) extends AstNode
case class MapAsync(f: Any Future[Any]) extends AstNode { case class MapAsync(f: Any Future[Any]) extends AstNode {
override def name = "mapAsync" override def name = "mapAsync"
@ -331,7 +326,7 @@ private[akka] object ActorProcessorFactory {
def props(materializer: FlowMaterializer, op: AstNode): Props = { def props(materializer: FlowMaterializer, op: AstNode): Props = {
val settings = materializer.settings val settings = materializer.settings
(op match { (op match {
case OpFactory(mkOps, _) Props(new ActorInterpreter(materializer.settings, mkOps.map(_.apply()))) case Fusable(ops, _) Props(new ActorInterpreter(materializer.settings, ops))
case t: Transform Props(new TransformProcessorImpl(settings, t.mkTransformer())) case t: Transform Props(new TransformProcessorImpl(settings, t.mkTransformer()))
case t: TimerTransform Props(new TimerTransformerProcessorsImpl(settings, t.mkTransformer())) case t: TimerTransform Props(new TimerTransformerProcessorsImpl(settings, t.mkTransformer()))
case m: MapAsync Props(new MapAsyncProcessorImpl(settings, m.f)) case m: MapAsync Props(new MapAsyncProcessorImpl(settings, m.f))

View file

@ -108,13 +108,13 @@ trait FlowOps[+Out] {
* Transform this stream by applying the given function to each of the elements * Transform this stream by applying the given function to each of the elements
* as they pass through this processing step. * as they pass through this processing step.
*/ */
def map[T](f: Out T): Repr[T] = andThen(OpFactory(() fusing.Map(f), "map")) def map[T](f: Out T): Repr[T] = andThen(Fusable(Vector(fusing.Map(f)), "map"))
/** /**
* Transform each input element into a sequence of output elements that is * Transform each input element into a sequence of output elements that is
* then flattened into the output stream. * then flattened into the output stream.
*/ */
def mapConcat[T](f: Out immutable.Seq[T]): Repr[T] = andThen(OpFactory(() fusing.MapConcat(f), "mapConcat")) def mapConcat[T](f: Out immutable.Seq[T]): Repr[T] = andThen(Fusable(Vector(fusing.MapConcat(f)), "mapConcat"))
/** /**
* Transform this stream by applying the given function to each of the elements * Transform this stream by applying the given function to each of the elements
@ -144,16 +144,16 @@ trait FlowOps[+Out] {
/** /**
* Only pass on those elements that satisfy the given predicate. * Only pass on those elements that satisfy the given predicate.
*/ */
def filter(p: Out Boolean): Repr[Out] = andThen(OpFactory(() fusing.Filter(p), "filter")) def filter(p: Out Boolean): Repr[Out] = andThen(Fusable(Vector(fusing.Filter(p)), "filter"))
/** /**
* Transform this stream by applying the given partial function to each of the elements * Transform this stream by applying the given partial function to each of the elements
* on which the function is defined as they pass through this processing step. * on which the function is defined as they pass through this processing step.
* Non-matching elements are filtered out. * Non-matching elements are filtered out.
*/ */
def collect[T](pf: PartialFunction[Out, T]): Repr[T] = andThen(OpFactory(List( def collect[T](pf: PartialFunction[Out, T]): Repr[T] = andThen(Fusable(Vector(
() fusing.Filter(pf.isDefinedAt), fusing.Filter(pf.isDefinedAt),
() fusing.Map(pf.apply)), "filter")) fusing.Map(pf.apply)), "filter"))
/** /**
* Chunk up this stream into groups of the given size, with the last group * Chunk up this stream into groups of the given size, with the last group
@ -163,7 +163,7 @@ trait FlowOps[+Out] {
*/ */
def grouped(n: Int): Repr[immutable.Seq[Out]] = { def grouped(n: Int): Repr[immutable.Seq[Out]] = {
require(n > 0, "n must be greater than 0") require(n > 0, "n must be greater than 0")
andThen(OpFactory(() fusing.Grouped(n), "grouped")) andThen(Fusable(Vector(fusing.Grouped(n)), "grouped"))
} }
/** /**
@ -208,8 +208,8 @@ trait FlowOps[+Out] {
* No elements will be dropped if `n` is zero or negative. * No elements will be dropped if `n` is zero or negative.
*/ */
def drop(n: Int): Repr[Out] = def drop(n: Int): Repr[Out] =
if (n <= 0) andThen(OpFactory(Nil, "drop")) if (n <= 0) andThen(Fusable(Vector.empty, "drop"))
else andThen(OpFactory(() fusing.Drop(n), "drop")) else andThen(Fusable(Vector(fusing.Drop(n)), "drop"))
/** /**
* Discard the elements received within the given duration at beginning of the stream. * Discard the elements received within the given duration at beginning of the stream.
@ -240,8 +240,8 @@ trait FlowOps[+Out] {
* or negative. * or negative.
*/ */
def take(n: Int): Repr[Out] = def take(n: Int): Repr[Out] =
if (n <= 0) andThen(OpFactory(() fusing.Completed(), "take")) if (n <= 0) andThen(Fusable(Vector(fusing.Completed()), "take"))
else andThen(OpFactory(() fusing.Take(n), "take")) else andThen(Fusable(Vector(fusing.Take(n)), "take"))
/** /**
* Terminate processing (and cancel the upstream publisher) after the given * Terminate processing (and cancel the upstream publisher) after the given
@ -278,7 +278,7 @@ trait FlowOps[+Out] {
* @param aggregate Takes the currently aggregated value and the current pending element to produce a new aggregate * @param aggregate Takes the currently aggregated value and the current pending element to produce a new aggregate
*/ */
def conflate[S](seed: Out S)(aggregate: (S, Out) S): Repr[S] = def conflate[S](seed: Out S)(aggregate: (S, Out) S): Repr[S] =
andThen(OpFactory(() fusing.Conflate(seed, aggregate), "conflate")) andThen(Fusable(Vector(fusing.Conflate(seed, aggregate)), "conflate"))
/** /**
* Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older * Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older
@ -294,7 +294,7 @@ trait FlowOps[+Out] {
* state. * state.
*/ */
def expand[S, U](seed: Out S)(extrapolate: S (U, S)): Repr[U] = def expand[S, U](seed: Out S)(extrapolate: S (U, S)): Repr[U] =
andThen(OpFactory(() fusing.Expand(seed, extrapolate), "expand")) andThen(Fusable(Vector(fusing.Expand(seed, extrapolate)), "expand"))
/** /**
* Adds a fixed size buffer in the flow that allows to store elements from a faster upstream until it becomes full. * Adds a fixed size buffer in the flow that allows to store elements from a faster upstream until it becomes full.
@ -306,7 +306,7 @@ trait FlowOps[+Out] {
*/ */
def buffer(size: Int, overflowStrategy: OverflowStrategy): Repr[Out] = { def buffer(size: Int, overflowStrategy: OverflowStrategy): Repr[Out] = {
require(size > 0, s"Buffer size must be larger than zero but was [$size]") require(size > 0, s"Buffer size must be larger than zero but was [$size]")
andThen(OpFactory(() fusing.Buffer(size, overflowStrategy), "buffer")) andThen(Fusable(Vector(fusing.Buffer(size, overflowStrategy)), "buffer"))
} }
/** /**