2014-03-30 09:27:19 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.stream.impl
|
|
|
|
|
|
|
|
|
|
import scala.annotation.tailrec
|
|
|
|
|
import scala.collection.immutable
|
|
|
|
|
|
|
|
|
|
import org.reactivestreams.api.{ Consumer, Processor, Producer }
|
|
|
|
|
import org.reactivestreams.spi.Subscriber
|
|
|
|
|
|
|
|
|
|
import akka.actor.ActorRefFactory
|
2014-04-08 13:37:55 +02:00
|
|
|
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
2014-03-30 09:27:19 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object Ast {
|
|
|
|
|
trait AstNode
|
|
|
|
|
|
2014-04-02 11:40:35 +02:00
|
|
|
case class Transform(
|
|
|
|
|
zero: Any,
|
|
|
|
|
f: (Any, Any) ⇒ (Any, immutable.Seq[Any]),
|
|
|
|
|
onComplete: Any ⇒ immutable.Seq[Any],
|
|
|
|
|
isComplete: Any ⇒ Boolean,
|
|
|
|
|
cleanup: Any ⇒ Unit) extends AstNode
|
|
|
|
|
|
2014-03-30 09:27:19 +02:00
|
|
|
case class Recover(t: Transform) extends AstNode
|
2014-03-30 09:27:19 +02:00
|
|
|
case class GroupBy(f: Any ⇒ Any) extends AstNode
|
|
|
|
|
case class SplitWhen(p: Any ⇒ Boolean) extends AstNode
|
|
|
|
|
case class Merge(other: Producer[Any]) extends AstNode
|
|
|
|
|
case class Zip(other: Producer[Any]) extends AstNode
|
2014-04-01 12:45:41 +02:00
|
|
|
case class Concat(next: Producer[Any]) extends AstNode
|
2014-03-28 15:44:18 +01:00
|
|
|
|
|
|
|
|
trait ProducerNode[I] {
|
2014-04-08 13:37:55 +02:00
|
|
|
def createProducer(settings: MaterializerSettings, context: ActorRefFactory): Producer[I]
|
2014-03-28 15:44:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case class ExistingProducer[I](producer: Producer[I]) extends ProducerNode[I] {
|
2014-04-08 13:37:55 +02:00
|
|
|
def createProducer(settings: MaterializerSettings, context: ActorRefFactory) = producer
|
2014-03-28 15:44:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case class IteratorProducerNode[I](iterator: Iterator[I]) extends ProducerNode[I] {
|
2014-04-08 13:37:55 +02:00
|
|
|
def createProducer(settings: MaterializerSettings, context: ActorRefFactory): Producer[I] =
|
2014-03-28 15:44:18 +01:00
|
|
|
if (iterator.isEmpty) EmptyProducer.asInstanceOf[Producer[I]]
|
|
|
|
|
else new ActorProducer[I](context.actorOf(IteratorProducer.props(iterator, settings)))
|
|
|
|
|
}
|
|
|
|
|
case class IterableProducerNode[I](iterable: immutable.Iterable[I]) extends ProducerNode[I] {
|
2014-04-08 13:37:55 +02:00
|
|
|
def createProducer(settings: MaterializerSettings, context: ActorRefFactory): Producer[I] =
|
2014-03-28 15:44:18 +01:00
|
|
|
if (iterable.isEmpty) EmptyProducer.asInstanceOf[Producer[I]]
|
|
|
|
|
else new ActorProducer[I](context.actorOf(IterableProducer.props(iterable, settings)))
|
|
|
|
|
}
|
2014-04-02 08:07:05 +02:00
|
|
|
case class ThunkProducerNode[I](f: () ⇒ I) extends ProducerNode[I] {
|
2014-04-08 13:37:55 +02:00
|
|
|
def createProducer(settings: MaterializerSettings, context: ActorRefFactory): Producer[I] =
|
2014-04-02 08:07:05 +02:00
|
|
|
new ActorProducer(context.actorOf(ActorProducer.props(settings, f)))
|
|
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2014-04-08 13:37:55 +02:00
|
|
|
private[akka] object ActorBasedFlowMaterializer {
|
2014-04-02 09:03:59 +02:00
|
|
|
|
|
|
|
|
val ctx = new ThreadLocal[ActorRefFactory]
|
|
|
|
|
|
|
|
|
|
def withCtx[T](arf: ActorRefFactory)(block: ⇒ T): T = {
|
|
|
|
|
val old = ctx.get()
|
|
|
|
|
ctx.set(arf)
|
|
|
|
|
try block
|
|
|
|
|
finally ctx.set(old)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2014-04-08 13:37:55 +02:00
|
|
|
private[akka] class ActorBasedFlowMaterializer(settings: MaterializerSettings, _context: ActorRefFactory) extends FlowMaterializer {
|
2014-03-30 09:27:19 +02:00
|
|
|
import Ast._
|
2014-04-08 13:37:55 +02:00
|
|
|
import ActorBasedFlowMaterializer._
|
2014-04-02 09:03:59 +02:00
|
|
|
|
|
|
|
|
private def context = ctx.get() match {
|
|
|
|
|
case null ⇒ _context
|
|
|
|
|
case x ⇒ x
|
|
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
|
|
|
|
|
@tailrec private def processorChain(topConsumer: Consumer[_], ops: immutable.Seq[AstNode]): Consumer[_] = {
|
|
|
|
|
ops match {
|
|
|
|
|
case op :: tail ⇒
|
|
|
|
|
val opProcessor: Processor[Any, Any] = processorForNode(op)
|
|
|
|
|
opProcessor.produceTo(topConsumer.asInstanceOf[Consumer[Any]])
|
|
|
|
|
processorChain(opProcessor, tail)
|
|
|
|
|
case _ ⇒ topConsumer
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Ops come in reverse order
|
2014-03-28 15:44:18 +01:00
|
|
|
override def toProducer[I, O](producerNode: ProducerNode[I], ops: List[AstNode]): Producer[O] = {
|
|
|
|
|
if (ops.isEmpty) producerNode.createProducer(settings, context).asInstanceOf[Producer[O]]
|
2014-03-30 09:27:19 +02:00
|
|
|
else {
|
|
|
|
|
val opProcessor = processorForNode(ops.head)
|
|
|
|
|
val topConsumer = processorChain(opProcessor, ops.tail)
|
2014-03-28 15:44:18 +01:00
|
|
|
producerNode.createProducer(settings, context).produceTo(topConsumer.asInstanceOf[Consumer[I]])
|
2014-03-30 09:27:19 +02:00
|
|
|
opProcessor.asInstanceOf[Producer[O]]
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-02 11:40:35 +02:00
|
|
|
private val identityConsumer = Transform((), (_, _) ⇒ () -> Nil, _ ⇒ Nil, _ ⇒ false, _ ⇒ ())
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-03-28 15:44:18 +01:00
|
|
|
override def consume[I](producerNode: ProducerNode[I], ops: List[AstNode]): Unit = {
|
2014-03-30 09:27:19 +02:00
|
|
|
val consumer = ops match {
|
|
|
|
|
case Nil ⇒
|
|
|
|
|
new ActorConsumer[Any](context.actorOf(ActorConsumer.props(settings, identityConsumer)))
|
|
|
|
|
case head :: tail ⇒
|
|
|
|
|
val c = new ActorConsumer[Any](context.actorOf(ActorConsumer.props(settings, head)))
|
|
|
|
|
processorChain(c, tail)
|
|
|
|
|
}
|
2014-03-28 15:44:18 +01:00
|
|
|
producerNode.createProducer(settings, context).produceTo(consumer.asInstanceOf[Consumer[I]])
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def processorForNode(op: AstNode): Processor[Any, Any] = new ActorProcessor(context.actorOf(ActorProcessor.props(settings, op)))
|
|
|
|
|
|
|
|
|
|
}
|