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
|
|
|
|
2014-11-09 21:09:50 +01:00
|
|
|
import scala.language.higherKinds
|
|
|
|
|
|
2014-10-27 09:48:54 +02:00
|
|
|
import akka.actor.Props
|
2014-11-17 22:50:15 +01:00
|
|
|
import akka.stream.impl.{ EmptyPublisher, ErrorPublisher, SynchronousIterablePublisher }
|
2014-10-03 17:33:14 +02:00
|
|
|
import org.reactivestreams.Publisher
|
2014-09-03 21:54:18 +02:00
|
|
|
import scala.collection.immutable
|
|
|
|
|
import scala.concurrent.duration.FiniteDuration
|
2014-11-17 22:50:15 +01:00
|
|
|
import scala.concurrent.{ ExecutionContext, Future }
|
2014-10-27 14:35:41 +01:00
|
|
|
import akka.stream.FlowMaterializer
|
2014-09-03 21:54:18 +02:00
|
|
|
|
2014-10-02 17:32:08 +02:00
|
|
|
/**
|
|
|
|
|
* A `Source` is a set of stream processing steps that has one open output and an attached input.
|
|
|
|
|
* Can be used as a `Publisher`
|
|
|
|
|
*/
|
|
|
|
|
trait Source[+Out] extends FlowOps[Out] {
|
2014-10-30 14:58:44 +01:00
|
|
|
type MaterializedType
|
2014-10-02 17:32:08 +02:00
|
|
|
override type Repr[+O] <: Source[O]
|
2014-09-03 21:54:18 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-11-06 14:03:01 +01:00
|
|
|
* Transform this [[akka.stream.scaladsl.Source]] by appending the given processing stages.
|
2014-09-03 21:54:18 +02:00
|
|
|
*/
|
2014-10-31 10:43:42 +02:00
|
|
|
def via[T](flow: Flow[Out, T]): Source[T]
|
2014-10-02 17:32:08 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-11-06 14:03:01 +01:00
|
|
|
* Connect this [[akka.stream.scaladsl.Source]] to a [[akka.stream.scaladsl.Sink]],
|
|
|
|
|
* concatenating the processing steps of both.
|
2014-10-02 17:32:08 +02:00
|
|
|
*/
|
2014-10-31 10:43:42 +02:00
|
|
|
def to(sink: Sink[Out]): RunnableFlow
|
2014-10-02 17:32:08 +02:00
|
|
|
|
2014-10-02 13:34:27 +02:00
|
|
|
/**
|
2014-10-17 14:05:50 +02:00
|
|
|
* Connect this `Source` to a `Sink` and run it. The returned value is the materialized value
|
2014-11-06 14:03:01 +01:00
|
|
|
* of the `Sink`, e.g. the `Publisher` of a [[akka.stream.scaladsl.Sink#publisher]].
|
2014-10-02 13:34:27 +02:00
|
|
|
*/
|
2014-11-09 21:09:50 +01:00
|
|
|
def runWith(sink: Sink[Out])(implicit materializer: FlowMaterializer): sink.MaterializedType = to(sink).run().get(sink)
|
2014-09-03 21:54:18 +02:00
|
|
|
|
2014-10-06 14:46:52 +02:00
|
|
|
/**
|
|
|
|
|
* Shortcut for running this `Source` with a fold function.
|
|
|
|
|
* The given function is invoked 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`
|
|
|
|
|
* if there is an error is signaled in the stream.
|
|
|
|
|
*/
|
2014-11-09 21:09:50 +01:00
|
|
|
def fold[U](zero: U)(f: (U, Out) ⇒ U)(implicit materializer: FlowMaterializer): Future[U] = runWith(FoldSink(zero)(f)) // FIXME why is fold always an end step?
|
2014-10-06 14:46:52 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Shortcut for running this `Source` with a foreach procedure. The given procedure is invoked
|
|
|
|
|
* for each received element.
|
|
|
|
|
* The returned [[scala.concurrent.Future]] will be completed with `Success` when reaching the
|
|
|
|
|
* normal end of the stream, or completed with `Failure` if there is an error is signaled in
|
|
|
|
|
* the stream.
|
|
|
|
|
*/
|
2014-11-09 21:09:50 +01:00
|
|
|
def foreach(f: Out ⇒ Unit)(implicit materializer: FlowMaterializer): Future[Unit] = runWith(ForeachSink(f))
|
2014-10-06 14:46:52 +02:00
|
|
|
|
2014-10-07 13:55:56 +02:00
|
|
|
/**
|
|
|
|
|
* Concatenates a second source so that the first element
|
|
|
|
|
* emitted by that source is emitted after the last element of this
|
|
|
|
|
* source.
|
|
|
|
|
*/
|
|
|
|
|
def concat[Out2 >: Out](second: Source[Out2]): Source[Out2] = Source.concat(this, second)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Concatenates a second source so that the first element
|
|
|
|
|
* emitted by that source is emitted after the last element of this
|
|
|
|
|
* source.
|
|
|
|
|
*
|
|
|
|
|
* This is a shorthand for [[concat]]
|
|
|
|
|
*/
|
|
|
|
|
def ++[Out2 >: Out](second: Source[Out2]): Source[Out2] = concat(second)
|
|
|
|
|
|
2014-11-28 10:41:57 +01:00
|
|
|
/**
|
|
|
|
|
* Add a key that will have a value available after materialization.
|
|
|
|
|
* The key can only use other keys if they have been added to the source
|
|
|
|
|
* before this key. This also includes the keyed source if applicable.
|
|
|
|
|
*/
|
|
|
|
|
def withKey(key: Key): Source[Out]
|
2014-12-01 20:07:55 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Applies given [[OperationAttributes]] to a given section.
|
|
|
|
|
*/
|
|
|
|
|
def section[T](attributes: OperationAttributes)(section: Source[Out] ⇒ Source[T]): Source[T] =
|
|
|
|
|
section(this.withAttributes(attributes)).withAttributes(OperationAttributes.none)
|
|
|
|
|
|
2014-10-02 17:32:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object Source {
|
2014-09-03 21:54:18 +02:00
|
|
|
/**
|
2014-10-02 17:32:08 +02:00
|
|
|
* Helper to create [[Source]] from `Publisher`.
|
2014-09-03 21:54:18 +02:00
|
|
|
*
|
|
|
|
|
* Construct a transformation starting with given publisher. The transformation steps
|
|
|
|
|
* are executed by a series of [[org.reactivestreams.Processor]] instances
|
|
|
|
|
* that mediate the flow of elements downstream and the propagation of
|
|
|
|
|
* back-pressure upstream.
|
|
|
|
|
*/
|
2014-10-17 14:05:50 +02:00
|
|
|
def apply[T](publisher: Publisher[T]): Source[T] = PublisherSource(publisher)
|
2014-09-03 21:54:18 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-10-02 17:32:08 +02:00
|
|
|
* Helper to create [[Source]] from `Iterator`.
|
2014-11-09 21:09:50 +01:00
|
|
|
* Example usage: `Source(() => Iterator.from(0))`
|
2014-09-03 21:54:18 +02:00
|
|
|
*
|
2014-11-09 21:09:50 +01:00
|
|
|
* Start a new `Source` from the given function that produces anIterator.
|
|
|
|
|
* The produced stream of elements will continue until the iterator runs empty
|
|
|
|
|
* or fails during evaluation of the `next()` method.
|
|
|
|
|
* Elements are pulled out of the iterator in accordance with the demand coming
|
|
|
|
|
* from the downstream transformation steps.
|
2014-09-03 21:54:18 +02:00
|
|
|
*/
|
2014-11-09 21:09:50 +01:00
|
|
|
def apply[T](f: () ⇒ Iterator[T]): Source[T] = apply(new FuncIterable(f))
|
2014-09-03 21:54:18 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-10-02 17:32:08 +02:00
|
|
|
* Helper to create [[Source]] from `Iterable`.
|
|
|
|
|
* Example usage: `Source(Seq(1,2,3))`
|
2014-09-03 21:54:18 +02:00
|
|
|
*
|
2014-10-02 17:32:08 +02:00
|
|
|
* Starts a new `Source` from the given `Iterable`. This is like starting from an
|
2014-09-03 21:54:18 +02:00
|
|
|
* Iterator, but every Subscriber directly attached to the Publisher of this
|
|
|
|
|
* stream will see an individual flow of elements (always starting from the
|
|
|
|
|
* beginning) regardless of when they subscribed.
|
|
|
|
|
*/
|
2014-11-18 22:06:36 +01:00
|
|
|
def apply[T](iterable: immutable.Iterable[T]): Source[T] = IterableSource(iterable)
|
2014-09-03 21:54:18 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-10-02 17:32:08 +02:00
|
|
|
* Start a new `Source` from the given `Future`. The stream will consist of
|
2014-09-03 21:54:18 +02:00
|
|
|
* one element when the `Future` is completed with a successful value, which
|
|
|
|
|
* may happen before or after materializing the `Flow`.
|
|
|
|
|
* The stream terminates with an error if the `Future` is completed with a failure.
|
|
|
|
|
*/
|
2014-10-17 14:05:50 +02:00
|
|
|
def apply[T](future: Future[T]): Source[T] = FutureSource(future)
|
2014-09-03 21:54:18 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Elements are produced from the tick closure periodically with the specified interval.
|
|
|
|
|
* The tick element will be delivered to downstream consumers that has requested any elements.
|
|
|
|
|
* If a consumer has not requested any elements at the point in time when the tick
|
|
|
|
|
* element is produced it will not receive that tick element later. It will
|
|
|
|
|
* receive new tick elements as soon as it has requested more elements.
|
|
|
|
|
*/
|
2014-11-26 16:33:58 +01:00
|
|
|
def apply[T](initialDelay: FiniteDuration, interval: FiniteDuration, tick: () ⇒ T): TickSource[T] =
|
2014-10-17 14:05:50 +02:00
|
|
|
TickSource(initialDelay, interval, tick)
|
2014-10-02 13:34:27 +02:00
|
|
|
|
2014-10-10 10:39:29 +02:00
|
|
|
/**
|
|
|
|
|
* Creates a `Source` by using an empty [[FlowGraphBuilder]] on a block that expects a [[FlowGraphBuilder]] and
|
|
|
|
|
* returns the `UndefinedSink`.
|
|
|
|
|
*/
|
|
|
|
|
def apply[T]()(block: FlowGraphBuilder ⇒ UndefinedSink[T]): Source[T] =
|
|
|
|
|
createSourceFromBuilder(new FlowGraphBuilder(), block)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Creates a `Source` by using a [[FlowGraphBuilder]] from this [[PartialFlowGraph]] on a block that expects
|
|
|
|
|
* a [[FlowGraphBuilder]] and returns the `UndefinedSink`.
|
|
|
|
|
*/
|
|
|
|
|
def apply[T](graph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ UndefinedSink[T]): Source[T] =
|
2014-11-28 10:41:57 +01:00
|
|
|
createSourceFromBuilder(new FlowGraphBuilder(graph), block)
|
2014-10-10 10:39:29 +02:00
|
|
|
|
|
|
|
|
private def createSourceFromBuilder[T](builder: FlowGraphBuilder, block: FlowGraphBuilder ⇒ UndefinedSink[T]): Source[T] = {
|
|
|
|
|
val out = block(builder)
|
|
|
|
|
builder.partialBuild().toSource(out)
|
|
|
|
|
}
|
2014-10-17 14:05:50 +02:00
|
|
|
|
2014-10-27 09:48:54 +02:00
|
|
|
/**
|
|
|
|
|
* Creates a `Source` 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.ActorPublisher]].
|
|
|
|
|
*/
|
|
|
|
|
def apply[T](props: Props): PropsSource[T] = PropsSource(props)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Create a `Source` with one element.
|
|
|
|
|
* Every connected `Sink` of this stream will see an individual stream consisting of one element.
|
|
|
|
|
*/
|
2014-11-17 22:50:15 +01:00
|
|
|
def singleton[T](element: T): Source[T] = apply(SynchronousIterablePublisher(List(element), "singleton")) // FIXME optimize
|
2014-10-27 09:48:54 +02:00
|
|
|
|
|
|
|
|
/**
|
2014-10-30 17:04:36 +01:00
|
|
|
* A `Source` with no elements, i.e. an empty stream that is completed immediately for every connected `Sink`.
|
2014-10-27 09:48:54 +02:00
|
|
|
*/
|
2014-10-30 17:04:36 +01:00
|
|
|
def empty[T](): Source[T] = _empty
|
|
|
|
|
private[this] val _empty: Source[Nothing] = apply(EmptyPublisher)
|
2014-10-27 09:48:54 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Create a `Source` that immediately ends the stream with the `cause` error to every connected `Sink`.
|
|
|
|
|
*/
|
2014-11-17 22:50:15 +01:00
|
|
|
def failed[T](cause: Throwable): Source[T] = apply(ErrorPublisher(cause, "failed"))
|
2014-10-27 09:48:54 +02:00
|
|
|
|
2014-10-07 13:55:56 +02:00
|
|
|
/**
|
|
|
|
|
* Concatenates two sources so that the first element
|
|
|
|
|
* emitted by the second source is emitted after the last element of the first
|
|
|
|
|
* source.
|
|
|
|
|
*/
|
2014-11-26 16:33:58 +01:00
|
|
|
def concat[T](source1: Source[T], source2: Source[T]): Source[T] = {
|
|
|
|
|
val output = UndefinedSink[T]
|
|
|
|
|
val concat = Concat[T]
|
|
|
|
|
Source() { b ⇒
|
|
|
|
|
b.addEdge(source1, Pipe.empty[T], concat.first)
|
|
|
|
|
.addEdge(source2, Pipe.empty[T], concat.second)
|
|
|
|
|
.addEdge(concat.out, Pipe.empty[T], output)
|
|
|
|
|
output
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Creates a `Source` that is materialized as a [[org.reactivestreams.Subscriber]]
|
|
|
|
|
*/
|
|
|
|
|
def subscriber[T]: SubscriberSource[T] = SubscriberSource[T]
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Source` that will create an object during materialization that the user will need
|
|
|
|
|
* to retrieve in order to access aspects of this source (could be a Subscriber, a
|
|
|
|
|
* Future/Promise, etc.).
|
|
|
|
|
*/
|
2014-10-30 14:58:44 +01:00
|
|
|
trait KeyedSource[+Out] extends Source[Out]
|