2014-09-03 21:54:18 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
2014-10-27 14:35:41 +01:00
|
|
|
package akka.stream.scaladsl
|
2014-09-03 21:54:18 +02:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.actor.{ ActorRef, Props }
|
|
|
|
|
import akka.stream.impl._
|
|
|
|
|
import akka.stream.{ SinkShape, Inlet, Outlet, Graph }
|
|
|
|
|
import akka.stream.scaladsl.OperationAttributes._
|
|
|
|
|
import akka.stream.stage.{ TerminationDirective, Directive, Context, PushStage }
|
|
|
|
|
import org.reactivestreams.{ Publisher, Subscriber }
|
|
|
|
|
import scala.annotation.unchecked.uncheckedVariance
|
|
|
|
|
import scala.concurrent.{ Promise, Future }
|
|
|
|
|
import scala.util.{ Success, Failure, Try }
|
2015-02-26 22:42:34 +01:00
|
|
|
import akka.stream.FlowMaterializer
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.impl.StreamLayout.Module
|
2014-09-03 21:54:18 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-10-02 17:32:08 +02:00
|
|
|
* A `Sink` is a set of stream processing steps that has one open input and an attached output.
|
|
|
|
|
* Can be used as a `Subscriber`
|
2014-09-03 21:54:18 +02:00
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
final class Sink[-In, +Mat](private[stream] override val module: Module)
|
|
|
|
|
extends Graph[SinkShape[In], Mat] {
|
|
|
|
|
|
|
|
|
|
override val shape: SinkShape[In] = module.shape.asInstanceOf[SinkShape[In]]
|
2014-10-30 14:58:44 +01:00
|
|
|
|
2014-10-02 13:34:27 +02:00
|
|
|
/**
|
2014-10-17 14:05:50 +02:00
|
|
|
* Connect this `Sink` to a `Source` and run it. The returned value is the materialized value
|
|
|
|
|
* of the `Source`, e.g. the `Subscriber` of a [[SubscriberSource]].
|
2014-10-02 13:34:27 +02:00
|
|
|
*/
|
2015-02-26 22:42:34 +01:00
|
|
|
def runWith[Mat2](source: Source[In, Mat2])(implicit materializer: FlowMaterializer): Mat2 =
|
2015-01-28 14:19:50 +01:00
|
|
|
source.to(this).run()
|
|
|
|
|
|
|
|
|
|
def mapMaterialized[Mat2](f: Mat ⇒ Mat2): Sink[In, Mat2] =
|
|
|
|
|
new Sink(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any]))
|
2014-10-10 10:39:29 +02:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
def withAttributes(attr: OperationAttributes): Sink[In, Mat] =
|
|
|
|
|
new Sink(module.withAttributes(attr).wrap())
|
2014-09-03 21:54:18 +02:00
|
|
|
}
|
2014-10-08 15:15:46 +02:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
object Sink extends SinkApply {
|
|
|
|
|
|
|
|
|
|
import OperationAttributes.{ none, name ⇒ named }
|
|
|
|
|
|
|
|
|
|
private def shape[T](name: String): SinkShape[T] = SinkShape(new Inlet(name + ".in"))
|
|
|
|
|
|
2014-10-08 15:15:46 +02:00
|
|
|
/**
|
2015-01-28 14:19:50 +01:00
|
|
|
* A graph with the shape of a source logically is a source, this method makes
|
|
|
|
|
* it so also in type.
|
2014-10-08 15:15:46 +02:00
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def wrap[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] = new Sink(g.module)
|
2014-10-10 10:39:29 +02:00
|
|
|
|
|
|
|
|
/**
|
2015-01-28 14:19:50 +01:00
|
|
|
* Helper to create [[Sink]] from `Subscriber`.
|
2014-10-10 10:39:29 +02:00
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def apply[T](subscriber: Subscriber[T]): Sink[T, Unit] = new Sink(new SubscriberSink(subscriber, none, shape("SubscriberSink")))
|
2014-10-10 10:39:29 +02:00
|
|
|
|
|
|
|
|
/**
|
2015-01-28 14:19:50 +01:00
|
|
|
* Helper to create [[Sink]] from `Subscriber`.
|
2014-10-10 10:39:29 +02:00
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def apply[T](subscriber: Subscriber[T], name: String): Sink[T, Unit] = new Sink(new SubscriberSink(subscriber, named(name), shape(name)))
|
2014-10-10 10:39:29 +02:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
/**
|
|
|
|
|
* Creates a `Sink` that is materialized to an [[akka.actor.ActorRef]] which points to an Actor
|
|
|
|
|
* created according to the passed in [[akka.actor.Props]]. Actor created by the `props` should
|
|
|
|
|
* be [[akka.stream.actor.ActorSubscriber]].
|
|
|
|
|
*/
|
|
|
|
|
def apply[T](props: Props): Sink[T, ActorRef] = new Sink(new PropsSink(props, none, shape("PropsSink")))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
2014-10-27 09:48:54 +02:00
|
|
|
/**
|
|
|
|
|
* Creates a `Sink` that is materialized to an [[akka.actor.ActorRef]] which points to an Actor
|
|
|
|
|
* created according to the passed in [[akka.actor.Props]]. Actor created by the `props` should
|
|
|
|
|
* be [[akka.stream.actor.ActorSubscriber]].
|
|
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def apply[T](props: Props, name: String): Sink[T, ActorRef] = new Sink(new PropsSink(props, named(name), shape(name)))
|
2014-10-27 09:48:54 +02:00
|
|
|
|
2014-10-17 14:05:50 +02:00
|
|
|
/**
|
|
|
|
|
* A `Sink` that immediately cancels its upstream after materialization.
|
|
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def cancelled[T](): Sink[T, Unit] = new Sink[Any, Unit](new CancelSink(none, shape("CancelledSink")))
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that immediately cancels its upstream after materialization.
|
|
|
|
|
*/
|
|
|
|
|
def cancelled[T](name: String): Sink[T, Unit] = new Sink[Any, Unit](new CancelSink(named(name), shape(name)))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a `Future` of the first value received.
|
|
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def head[T](): Sink[T, Future[T]] = new Sink(new HeadSink[T](none, shape("HeadSink")))
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a `Future` of the first value received.
|
|
|
|
|
*/
|
|
|
|
|
def head[T](name: String): Sink[T, Future[T]] = new Sink(new HeadSink[T](named(name), shape(name)))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a [[org.reactivestreams.Publisher]].
|
|
|
|
|
* that can handle one [[org.reactivestreams.Subscriber]].
|
|
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def publisher[T](): Sink[T, Publisher[T]] = new Sink(new PublisherSink[T](none, shape("PublisherSink")))
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a [[org.reactivestreams.Publisher]].
|
|
|
|
|
* that can handle one [[org.reactivestreams.Subscriber]].
|
|
|
|
|
*/
|
|
|
|
|
def publisher[T](name: String): Sink[T, Publisher[T]] = new Sink(new PublisherSink[T](named(name), shape(name)))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a [[org.reactivestreams.Publisher]]
|
|
|
|
|
* that can handle more than one [[org.reactivestreams.Subscriber]].
|
|
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def fanoutPublisher[T](initialBufferSize: Int, maximumBufferSize: Int): Sink[T, Publisher[T]] =
|
|
|
|
|
new Sink(new FanoutPublisherSink[T](initialBufferSize, maximumBufferSize, none, shape("FanoutPublisherSink")))
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a [[org.reactivestreams.Publisher]]
|
|
|
|
|
* that can handle more than one [[org.reactivestreams.Subscriber]].
|
|
|
|
|
*/
|
|
|
|
|
def fanoutPublisher[T](initialBufferSize: Int, maximumBufferSize: Int, name: String): Sink[T, Publisher[T]] =
|
|
|
|
|
new Sink(new FanoutPublisherSink[T](initialBufferSize, maximumBufferSize, named(name), shape(name)))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that will consume the stream and discard the elements.
|
|
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def ignore(): Sink[Any, Unit] = new Sink(new BlackholeSink(none, shape("BlackholeSink")))
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that will consume the stream and discard the elements.
|
|
|
|
|
*/
|
|
|
|
|
def ignore(name: String): Sink[Any, Unit] = new Sink(new BlackholeSink(named(name), shape(name)))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that will invoke the given procedure for each received element. The sink is materialized
|
|
|
|
|
* into a [[scala.concurrent.Future]] will be completed with `Success` when reaching the
|
2015-01-30 10:30:56 +01:00
|
|
|
* normal end of the stream, or completed with `Failure` if there is a failure signaled in
|
2014-10-17 14:05:50 +02:00
|
|
|
* the stream..
|
|
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def foreach[T](f: T ⇒ Unit): Sink[T, Future[Unit]] = {
|
|
|
|
|
|
|
|
|
|
def newForeachStage(): (PushStage[T, Unit], Future[Unit]) = {
|
|
|
|
|
val promise = Promise[Unit]()
|
|
|
|
|
|
|
|
|
|
val stage = new PushStage[T, Unit] {
|
|
|
|
|
override def onPush(elem: T, ctx: Context[Unit]): Directive = {
|
|
|
|
|
f(elem)
|
|
|
|
|
ctx.pull()
|
|
|
|
|
}
|
|
|
|
|
override def onUpstreamFailure(cause: Throwable, ctx: Context[Unit]): TerminationDirective = {
|
|
|
|
|
promise.failure(cause)
|
|
|
|
|
ctx.fail(cause)
|
|
|
|
|
}
|
|
|
|
|
override def onUpstreamFinish(ctx: Context[Unit]): TerminationDirective = {
|
|
|
|
|
promise.success(())
|
|
|
|
|
ctx.finish()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
(stage, promise.future)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
Flow[T].transformMaterializing(newForeachStage).to(Sink.ignore).withAttributes(name("foreach"))
|
|
|
|
|
|
|
|
|
|
}
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that will invoke the given function for every received element, giving it its previous
|
|
|
|
|
* output (or the given `zero` value) and the element as input.
|
|
|
|
|
* The returned [[scala.concurrent.Future]] will be completed with value of the final
|
|
|
|
|
* function evaluation when the input stream ends, or completed with `Failure`
|
2015-01-30 10:30:56 +01:00
|
|
|
* if there is a failure signaled in the stream.
|
2014-10-17 14:05:50 +02:00
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def fold[U, T](zero: U)(f: (U, T) ⇒ U): Sink[T, Future[U]] = {
|
|
|
|
|
|
|
|
|
|
def newFoldStage(): (PushStage[T, U], Future[U]) = {
|
|
|
|
|
val promise = Promise[U]()
|
|
|
|
|
|
|
|
|
|
val stage = new PushStage[T, U] {
|
|
|
|
|
private var aggregator = zero
|
|
|
|
|
|
|
|
|
|
override def onPush(elem: T, ctx: Context[U]): Directive = {
|
|
|
|
|
aggregator = f(aggregator, elem)
|
|
|
|
|
ctx.pull()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFailure(cause: Throwable, ctx: Context[U]): TerminationDirective = {
|
|
|
|
|
promise.failure(cause)
|
|
|
|
|
ctx.fail(cause)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFinish(ctx: Context[U]): TerminationDirective = {
|
|
|
|
|
promise.success(aggregator)
|
|
|
|
|
ctx.finish()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
(stage, promise.future)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
Flow[T].transformMaterializing(newFoldStage).to(Sink.ignore).withAttributes(name("fold"))
|
|
|
|
|
|
|
|
|
|
}
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
2015-01-30 10:30:56 +01:00
|
|
|
* A `Sink` that when the flow is completed, either through a failure or normal
|
2014-10-17 14:05:50 +02:00
|
|
|
* completion, apply the provided function with [[scala.util.Success]]
|
|
|
|
|
* or [[scala.util.Failure]].
|
|
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def onComplete[T](callback: Try[Unit] ⇒ Unit): Sink[T, Unit] = {
|
|
|
|
|
|
|
|
|
|
def newOnCompleteStage(): PushStage[T, Unit] = {
|
|
|
|
|
new PushStage[T, Unit] {
|
|
|
|
|
override def onPush(elem: T, ctx: Context[Unit]): Directive = ctx.pull()
|
|
|
|
|
override def onUpstreamFailure(cause: Throwable, ctx: Context[Unit]): TerminationDirective = {
|
|
|
|
|
callback(Failure(cause))
|
|
|
|
|
ctx.fail(cause)
|
|
|
|
|
}
|
|
|
|
|
override def onUpstreamFinish(ctx: Context[Unit]): TerminationDirective = {
|
|
|
|
|
callback(Success[Unit](()))
|
|
|
|
|
ctx.finish()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
Flow[T].transform(newOnCompleteStage).to(Sink.ignore).withAttributes(name("onComplete"))
|
|
|
|
|
}
|
2014-10-17 14:05:50 +02:00
|
|
|
}
|