2014-10-03 17:33:14 +02:00
|
|
|
/**
|
2015-04-16 02:24:01 +02:00
|
|
|
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
2014-10-03 17:33:14 +02:00
|
|
|
*/
|
|
|
|
|
package akka.stream.javadsl
|
|
|
|
|
|
2015-04-16 02:24:01 +02:00
|
|
|
import akka.actor.{ ActorRef, Props }
|
2015-04-23 20:59:55 +02:00
|
|
|
import akka.japi.function
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.impl.StreamLayout
|
2015-04-16 02:24:01 +02:00
|
|
|
import akka.stream.{ javadsl, scaladsl, _ }
|
|
|
|
|
import org.reactivestreams.{ Publisher, Subscriber }
|
|
|
|
|
|
2015-08-19 23:04:20 -04:00
|
|
|
import scala.concurrent.duration.FiniteDuration
|
2015-06-09 00:05:56 -04:00
|
|
|
import scala.concurrent.{ ExecutionContext, Future }
|
2015-03-05 12:21:17 +01:00
|
|
|
import scala.util.Try
|
2014-10-03 17:33:14 +02:00
|
|
|
|
2014-10-20 14:09:24 +02:00
|
|
|
/** Java API */
|
2014-10-03 17:33:14 +02:00
|
|
|
object Sink {
|
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 is signaled in the stream.
|
2014-10-17 14:05:50 +02:00
|
|
|
*/
|
2015-04-23 20:59:55 +02:00
|
|
|
def fold[U, In](zero: U, f: function.Function2[U, In, U]): javadsl.Sink[In, Future[U]] =
|
2015-01-28 14:19:50 +01:00
|
|
|
new Sink(scaladsl.Sink.fold[U, In](zero)(f.apply))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-10-20 14:09:24 +02:00
|
|
|
* Helper to create [[Sink]] from `Subscriber`.
|
2014-10-17 14:05:50 +02:00
|
|
|
*/
|
2015-01-28 14:19:50 +01:00
|
|
|
def create[In](subs: Subscriber[In]): Sink[In, Unit] =
|
|
|
|
|
new Sink(scaladsl.Sink(subs))
|
2014-10-20 14:09:24 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that immediately cancels its upstream after materialization.
|
|
|
|
|
*/
|
2015-03-05 12:21:17 +01:00
|
|
|
def cancelled[T](): Sink[T, Unit] =
|
2014-10-27 14:35:41 +01:00
|
|
|
new Sink(scaladsl.Sink.cancelled)
|
2014-10-20 14:09:24 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that will consume the stream and discard the elements.
|
2014-10-17 14:05:50 +02:00
|
|
|
*/
|
2015-06-05 18:26:32 +02:00
|
|
|
def ignore[T](): Sink[T, Future[Unit]] =
|
2014-10-27 14:35:41 +01:00
|
|
|
new Sink(scaladsl.Sink.ignore)
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a [[org.reactivestreams.Publisher]].
|
2015-10-30 16:00:44 +01:00
|
|
|
* that can handle `maxNumberOfSubscribers` [[org.reactivestreams.Subscriber]]s.
|
|
|
|
|
*
|
|
|
|
|
* If `maxNumberOfSubscribers` is greater than 1, the size of the `inputBuffer` configured for this stage
|
|
|
|
|
* becomes the maximum number of elements that the fastest [[org.reactivestreams.Subscriber]] can be ahead
|
|
|
|
|
* of the slowest one before slowing the processing down due to back pressure.
|
2014-10-17 14:05:50 +02:00
|
|
|
*/
|
2015-10-30 16:00:44 +01:00
|
|
|
def publisher[In](maxNumberOfSubscribers: Int): Sink[In, Publisher[In]] =
|
|
|
|
|
new Sink(scaladsl.Sink.publisher(maxNumberOfSubscribers))
|
2014-10-20 14:09:24 +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 is signaled in
|
2014-10-20 14:09:24 +02:00
|
|
|
* the stream..
|
|
|
|
|
*/
|
2015-04-23 20:59:55 +02:00
|
|
|
def foreach[T](f: function.Procedure[T]): Sink[T, Future[Unit]] =
|
2015-01-28 14:19:50 +01:00
|
|
|
new Sink(scaladsl.Sink.foreach(f.apply))
|
2015-06-06 14:36:49 +02:00
|
|
|
|
2015-06-09 00:05:56 -04:00
|
|
|
/**
|
|
|
|
|
* A `Sink` that will invoke the given procedure for each received element in parallel. The sink is materialized
|
|
|
|
|
* into a [[scala.concurrent.Future]].
|
|
|
|
|
*
|
|
|
|
|
* If `f` throws an exception and the supervision decision is
|
|
|
|
|
* [[akka.stream.Supervision.Stop]] the `Future` will be completed with failure.
|
|
|
|
|
*
|
|
|
|
|
* If `f` throws an exception and the supervision decision is
|
|
|
|
|
* [[akka.stream.Supervision.Resume]] or [[akka.stream.Supervision.Restart]] the
|
|
|
|
|
* element is dropped and the stream continues.
|
|
|
|
|
*/
|
|
|
|
|
def foreachParallel[T](parallel: Int)(f: function.Procedure[T])(ec: ExecutionContext): Sink[T, Future[Unit]] =
|
|
|
|
|
new Sink(scaladsl.Sink.foreachParallel(parallel)(f.apply)(ec))
|
2014-10-20 14:09:24 +02:00
|
|
|
|
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-04-23 20:59:55 +02:00
|
|
|
def onComplete[In](callback: function.Procedure[Try[Unit]]): Sink[In, Unit] =
|
2015-03-05 12:21:17 +01:00
|
|
|
new Sink(scaladsl.Sink.onComplete[In](x ⇒ callback.apply(x)))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a `Future` of the first value received.
|
|
|
|
|
*/
|
2015-03-05 12:21:17 +01:00
|
|
|
def head[In](): Sink[In, Future[In]] =
|
2015-01-28 14:19:50 +01:00
|
|
|
new Sink(scaladsl.Sink.head[In])
|
2014-10-20 14:09:24 +02:00
|
|
|
|
2015-03-30 14:42:30 +02:00
|
|
|
/**
|
|
|
|
|
* Sends the elements of the stream to the given `ActorRef`.
|
2015-09-28 22:23:59 -07:00
|
|
|
* If the target actor terminates the stream will be canceled.
|
2015-03-30 14:42:30 +02:00
|
|
|
* When the stream is completed successfully the given `onCompleteMessage`
|
|
|
|
|
* will be sent to the destination actor.
|
|
|
|
|
* When the stream is completed with failure a [[akka.actor.Status.Failure]]
|
|
|
|
|
* message will be sent to the destination actor.
|
|
|
|
|
*
|
|
|
|
|
* It will request at most `maxInputBufferSize` number of elements from
|
|
|
|
|
* upstream, but there is no back-pressure signal from the destination actor,
|
|
|
|
|
* i.e. if the actor is not consuming the messages fast enough the mailbox
|
|
|
|
|
* of the actor will grow. For potentially slow consumer actors it is recommended
|
|
|
|
|
* to use a bounded mailbox with zero `mailbox-push-timeout-time` or use a rate
|
2015-03-31 15:13:57 +02:00
|
|
|
* limiting stage in front of this `Sink`.
|
2015-03-30 14:42:30 +02:00
|
|
|
*
|
|
|
|
|
*/
|
|
|
|
|
def actorRef[In](ref: ActorRef, onCompleteMessage: Any): Sink[In, Unit] =
|
|
|
|
|
new Sink(scaladsl.Sink.actorRef[In](ref, onCompleteMessage))
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* 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 actorSubscriber[T](props: Props): Sink[T, ActorRef] =
|
|
|
|
|
new Sink(scaladsl.Sink.actorSubscriber(props))
|
|
|
|
|
|
2015-03-04 15:22:33 +01:00
|
|
|
/**
|
|
|
|
|
* A graph with the shape of a sink logically is a sink, this method makes
|
|
|
|
|
* it so also in type.
|
|
|
|
|
*/
|
2015-10-21 22:45:39 +02:00
|
|
|
def fromGraph[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] =
|
2015-06-06 17:17:23 +02:00
|
|
|
g match {
|
|
|
|
|
case s: Sink[T, M] ⇒ s
|
2015-10-21 22:45:39 +02:00
|
|
|
case other ⇒ new Sink(scaladsl.Sink.fromGraph(other))
|
2015-06-06 17:17:23 +02:00
|
|
|
}
|
2015-06-29 23:47:31 -04:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Combine several sinks with fan-out strategy like `Broadcast` or `Balance` and returns `Sink`.
|
|
|
|
|
*/
|
|
|
|
|
def combine[T, U](output1: Sink[U, _], output2: Sink[U, _], rest: java.util.List[Sink[U, _]], strategy: function.Function[java.lang.Integer, Graph[UniformFanOutShape[T, U], Unit]]): Sink[T, Unit] = {
|
|
|
|
|
import scala.collection.JavaConverters._
|
|
|
|
|
val seq = if (rest != null) rest.asScala.map(_.asScala) else Seq()
|
|
|
|
|
new Sink(scaladsl.Sink.combine(output1.asScala, output2.asScala, seq: _*)(num ⇒ strategy.apply(num)))
|
|
|
|
|
}
|
|
|
|
|
|
2015-08-19 23:04:20 -04:00
|
|
|
/**
|
|
|
|
|
* Creates a `Sink` that is materialized as an [[akka.stream.SinkQueue]].
|
|
|
|
|
* [[akka.stream.SinkQueue.pull]] method is pulling element from the stream and returns ``Future[Option[T]]``.
|
|
|
|
|
* `Future` completes when element is available.
|
|
|
|
|
*
|
|
|
|
|
* `Sink` will request at most `bufferSize` number of elements from
|
|
|
|
|
* upstream and then stop back pressure.
|
|
|
|
|
*
|
|
|
|
|
* @param bufferSize The size of the buffer in element count
|
|
|
|
|
* @param timeout Timeout for ``SinkQueue.pull():Future[Option[T] ]``
|
|
|
|
|
*/
|
|
|
|
|
def queue[T](bufferSize: Int, timeout: FiniteDuration): Sink[T, SinkQueue[T]] =
|
|
|
|
|
new Sink(scaladsl.Sink.queue(bufferSize, timeout))
|
|
|
|
|
|
2014-10-03 17:33:14 +02:00
|
|
|
}
|
2014-10-17 14:05:50 +02:00
|
|
|
|
2014-10-03 17:33:14 +02:00
|
|
|
/**
|
2014-10-20 14:09:24 +02:00
|
|
|
* Java API
|
|
|
|
|
*
|
2014-10-03 17:33:14 +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`
|
|
|
|
|
*/
|
2015-11-01 21:03:28 +01:00
|
|
|
final class Sink[-In, +Mat](delegate: scaladsl.Sink[In, Mat]) extends Graph[SinkShape[In], Mat] {
|
2014-10-03 17:33:14 +02:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
override def shape: SinkShape[In] = delegate.shape
|
|
|
|
|
private[stream] def module: StreamLayout.Module = delegate.module
|
2014-10-03 17:33:14 +02:00
|
|
|
|
2015-03-06 12:22:14 +01:00
|
|
|
/** Converts this Sink to its Scala DSL counterpart */
|
2015-01-28 14:19:50 +01:00
|
|
|
def asScala: scaladsl.Sink[In, Mat] = delegate
|
2014-10-03 17:33:14 +02:00
|
|
|
|
2014-10-20 14:09:24 +02:00
|
|
|
/**
|
2014-10-20 14:09:24 +02:00
|
|
|
* Connect this `Sink` to a `Source` and run it.
|
2014-10-20 14:09:24 +02:00
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def runWith[M](source: Graph[SourceShape[In], M], materializer: Materializer): M =
|
2015-04-24 12:14:04 +02:00
|
|
|
asScala.runWith(source)(materializer)
|
2014-10-03 17:33:14 +02:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
/**
|
|
|
|
|
* Transform only the materialized value of this Sink, leaving all other properties as they were.
|
|
|
|
|
*/
|
2015-05-05 10:29:41 +02:00
|
|
|
def mapMaterializedValue[Mat2](f: function.Function[Mat, Mat2]): Sink[In, Mat2] =
|
|
|
|
|
new Sink(delegate.mapMaterializedValue(f.apply _))
|
2015-03-05 12:21:17 +01:00
|
|
|
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): javadsl.Sink[In, Mat] =
|
2015-04-10 16:49:49 +02:00
|
|
|
new Sink(delegate.withAttributes(attr))
|
2015-03-05 12:21:17 +01:00
|
|
|
|
2015-04-14 08:59:37 +02:00
|
|
|
override def named(name: String): javadsl.Sink[In, Mat] =
|
2015-03-05 12:21:17 +01:00
|
|
|
new Sink(delegate.named(name))
|
2014-10-17 14:05:50 +02:00
|
|
|
}
|