2014-09-03 21:54:18 +02:00
|
|
|
/**
|
2015-04-16 02:24:01 +02:00
|
|
|
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
|
2014-09-03 21:54:18 +02:00
|
|
|
*/
|
2014-10-27 14:35:41 +01:00
|
|
|
package akka.stream.scaladsl
|
2014-09-03 21:54:18 +02:00
|
|
|
|
2015-11-17 13:17:30 +01:00
|
|
|
import java.io.{ InputStream, OutputStream, File }
|
2015-11-10 15:15:59 +01:00
|
|
|
import akka.dispatch.ExecutionContexts
|
2015-10-24 00:07:51 -04:00
|
|
|
import akka.actor.{ Status, ActorRef, Props }
|
2015-10-09 15:11:01 -04:00
|
|
|
import akka.stream.actor.ActorSubscriber
|
2015-08-19 23:04:20 -04:00
|
|
|
import akka.stream.impl.Stages.DefaultAttributes
|
2015-04-16 02:24:01 +02:00
|
|
|
import akka.stream.impl.StreamLayout.Module
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.impl._
|
2015-11-17 13:17:30 +01:00
|
|
|
import akka.stream.impl.io.{ InputStreamSinkStage, OutputStreamSink, FileSink }
|
2015-08-19 23:04:20 -04:00
|
|
|
import akka.stream.stage.{ Context, PushStage, SyncDirective, TerminationDirective }
|
|
|
|
|
import akka.stream.{ javadsl, _ }
|
2015-11-14 22:42:22 +01:00
|
|
|
import akka.util.ByteString
|
2015-01-28 14:19:50 +01:00
|
|
|
import org.reactivestreams.{ Publisher, Subscriber }
|
2015-04-16 02:24:01 +02:00
|
|
|
|
2015-06-29 23:47:31 -04:00
|
|
|
import scala.annotation.tailrec
|
2015-08-19 23:04:20 -04:00
|
|
|
import scala.concurrent.duration.{ FiniteDuration, _ }
|
|
|
|
|
import scala.concurrent.{ ExecutionContext, Future }
|
2015-04-16 02:24:01 +02:00
|
|
|
import scala.util.{ Failure, Success, Try }
|
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
|
2015-03-06 10:23:26 +01:00
|
|
|
* of the `Source`, e.g. the `Subscriber` of a [[Source#subscriber]].
|
2014-10-02 13:34:27 +02:00
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def runWith[Mat2](source: Graph[SourceShape[In], Mat2])(implicit materializer: Materializer): Mat2 =
|
2015-10-21 22:45:39 +02:00
|
|
|
Source.fromGraph(source).to(this).run()
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-05-05 10:29:41 +02:00
|
|
|
def mapMaterializedValue[Mat2](f: Mat ⇒ Mat2): Sink[In, Mat2] =
|
2015-01-28 14:19:50 +01:00
|
|
|
new Sink(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any]))
|
2014-10-10 10:39:29 +02:00
|
|
|
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Sink[In, Mat] =
|
2015-07-06 22:00:21 +02:00
|
|
|
new Sink(module.withAttributes(attr).nest())
|
2015-03-05 12:21:17 +01:00
|
|
|
|
2015-06-23 17:32:55 +02:00
|
|
|
override def named(name: String): Sink[In, Mat] = withAttributes(Attributes.name(name))
|
2015-03-06 12:22:14 +01:00
|
|
|
|
|
|
|
|
/** Converts this Scala DSL element to it's Java DSL counterpart. */
|
|
|
|
|
def asJava: javadsl.Sink[In, Mat] = new javadsl.Sink(this)
|
2014-09-03 21:54:18 +02:00
|
|
|
}
|
2014-10-08 15:15:46 +02:00
|
|
|
|
2015-10-21 22:45:39 +02:00
|
|
|
object Sink {
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-04-16 02:24:01 +02:00
|
|
|
/** INTERNAL API */
|
2015-06-13 16:28:38 -04:00
|
|
|
private[stream] def shape[T](name: String): SinkShape[T] = SinkShape(Inlet(name + ".in"))
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2014-10-08 15:15:46 +02:00
|
|
|
/**
|
2015-03-04 15:22:33 +01:00
|
|
|
* A graph with the shape of a sink logically is a sink, this method makes
|
2015-01-28 14:19:50 +01:00
|
|
|
* it so also in type.
|
2014-10-08 15:15:46 +02:00
|
|
|
*/
|
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 {
|
2015-10-21 22:45:39 +02:00
|
|
|
case s: Sink[T, M] ⇒ s
|
|
|
|
|
case s: javadsl.Sink[T, M] ⇒ s.asScala
|
|
|
|
|
case other ⇒ new Sink(other.module)
|
2015-06-06 17:17:23 +02:00
|
|
|
}
|
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-03-05 12:21:17 +01:00
|
|
|
def apply[T](subscriber: Subscriber[T]): Sink[T, Unit] =
|
2015-04-20 21:04:03 +02:00
|
|
|
new Sink(new SubscriberSink(subscriber, DefaultAttributes.subscriberSink, shape("SubscriberSink")))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that immediately cancels its upstream after materialization.
|
|
|
|
|
*/
|
2015-04-20 21:04:03 +02:00
|
|
|
def cancelled[T]: Sink[T, Unit] =
|
|
|
|
|
new Sink[Any, Unit](new CancelSink(DefaultAttributes.cancelledSink, shape("CancelledSink")))
|
2014-10-17 14:05:50 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a `Future` of the first value received.
|
2015-11-10 15:15:59 +01:00
|
|
|
* If the stream completes before signaling at least a single element, the Future will be failed with a [[NoSuchElementException]].
|
|
|
|
|
* If the stream signals an error errors before signaling at least a single element, the Future will be failed with the streams exception.
|
|
|
|
|
*
|
|
|
|
|
* See also [[headOption]].
|
|
|
|
|
*/
|
2015-11-18 00:09:04 +01:00
|
|
|
def head[T]: Sink[T, Future[T]] =
|
|
|
|
|
Sink.fromGraph(new HeadOptionStage[T]).withAttributes(DefaultAttributes.headSink)
|
|
|
|
|
.mapMaterializedValue(e ⇒ e.map(_.getOrElse(throw new NoSuchElementException("head of empty stream")))(ExecutionContexts.sameThreadExecutionContext))
|
2015-11-10 15:15:59 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a `Future` of the optional first value received.
|
|
|
|
|
* If the stream completes before signaling at least a single element, the value of the Future will be [[None]].
|
|
|
|
|
* If the stream signals an error errors before signaling at least a single element, the Future will be failed with the streams exception.
|
|
|
|
|
*
|
|
|
|
|
* See also [[head]].
|
2014-10-17 14:05:50 +02:00
|
|
|
*/
|
2015-11-18 00:09:04 +01:00
|
|
|
def headOption[T]: Sink[T, Future[Option[T]]] =
|
|
|
|
|
Sink.fromGraph(new HeadOptionStage[T]).withAttributes(DefaultAttributes.headOptionSink)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a `Future` of the last value received.
|
|
|
|
|
* If the stream completes before signaling at least a single element, the Future will be failed with a [[NoSuchElementException]].
|
|
|
|
|
* If the stream signals an error errors before signaling at least a single element, the Future will be failed with the streams exception.
|
|
|
|
|
*
|
|
|
|
|
* See also [[lastOption]].
|
|
|
|
|
*/
|
|
|
|
|
def last[T]: Sink[T, Future[T]] = Sink.fromGraph(new LastOptionStage[T]).withAttributes(DefaultAttributes.lastSink)
|
|
|
|
|
.mapMaterializedValue(e ⇒ e.map(_.getOrElse(throw new NoSuchElementException("last of empty stream")))(ExecutionContexts.sameThreadExecutionContext))
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that materializes into a `Future` of the optional last value received.
|
|
|
|
|
* If the stream completes before signaling at least a single element, the value of the Future will be [[None]].
|
|
|
|
|
* If the stream signals an error errors before signaling at least a single element, the Future will be failed with the streams exception.
|
|
|
|
|
*
|
|
|
|
|
* See also [[last]].
|
|
|
|
|
*/
|
|
|
|
|
def lastOption[T]: Sink[T, Future[Option[T]]] = Sink.fromGraph(new LastOptionStage[T]).withAttributes(DefaultAttributes.lastOptionSink)
|
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
|
|
|
*
|
2015-11-03 12:53:24 +01:00
|
|
|
* If `fanout` is `true`, the materialized `Publisher` will support multiple `Subscriber`s and
|
|
|
|
|
* 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.
|
|
|
|
|
*
|
|
|
|
|
* If `fanout` is `false` then the materialized `Publisher` will only support a single `Subscriber` and
|
|
|
|
|
* reject any additional `Subscriber`s.
|
2014-10-17 14:05:50 +02:00
|
|
|
*/
|
2015-11-03 12:53:24 +01:00
|
|
|
def publisher[T](fanout: Boolean): Sink[T, Publisher[T]] =
|
2015-10-30 16:00:44 +01:00
|
|
|
new Sink(
|
2015-11-03 12:53:24 +01:00
|
|
|
if (fanout) new FanoutPublisherSink[T](DefaultAttributes.fanoutPublisherSink, shape("FanoutPublisherSink"))
|
|
|
|
|
else new PublisherSink[T](DefaultAttributes.publisherSink, shape("PublisherSink")))
|
2015-01-28 14:19:50 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A `Sink` that will consume the stream and discard the elements.
|
|
|
|
|
*/
|
2015-06-05 18:26:32 +02:00
|
|
|
def ignore: Sink[Any, Future[Unit]] =
|
2015-11-18 00:09:04 +01:00
|
|
|
new Sink(new SinkholeSink(DefaultAttributes.ignoreSink, shape("SinkholeSink")))
|
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-06-14 03:12:30 -04:00
|
|
|
def foreach[T](f: T ⇒ Unit): Sink[T, Future[Unit]] =
|
|
|
|
|
Flow[T].map(f).toMat(Sink.ignore)(Keep.right).named("foreachSink")
|
2014-10-17 14:05:50 +02:00
|
|
|
|
2015-06-29 23:47:31 -04:00
|
|
|
/**
|
|
|
|
|
* Combine several sinks with fun-out strategy like `Broadcast` or `Balance` and returns `Sink`.
|
|
|
|
|
*/
|
|
|
|
|
def combine[T, U](first: Sink[U, _], second: Sink[U, _], rest: Sink[U, _]*)(strategy: Int ⇒ Graph[UniformFanOutShape[T, U], Unit]): Sink[T, Unit] =
|
|
|
|
|
|
2015-11-30 15:45:37 +01:00
|
|
|
Sink.fromGraph(GraphDSL.create() { implicit b ⇒
|
|
|
|
|
import GraphDSL.Implicits._
|
2015-06-29 23:47:31 -04:00
|
|
|
val d = b.add(strategy(rest.size + 2))
|
|
|
|
|
d.out(0) ~> first
|
|
|
|
|
d.out(1) ~> second
|
|
|
|
|
|
|
|
|
|
@tailrec def combineRest(idx: Int, i: Iterator[Sink[U, _]]): SinkShape[T] =
|
|
|
|
|
if (i.hasNext) {
|
|
|
|
|
d.out(idx) ~> i.next()
|
|
|
|
|
combineRest(idx + 1, i)
|
|
|
|
|
} else new SinkShape(d.in)
|
|
|
|
|
|
|
|
|
|
combineRest(2, rest.iterator)
|
|
|
|
|
})
|
|
|
|
|
|
2015-06-09 00:05:56 -04:00
|
|
|
/**
|
|
|
|
|
* A `Sink` that will invoke the given function to each of the elements
|
|
|
|
|
* as they pass in. 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.
|
|
|
|
|
*
|
|
|
|
|
* @see [[#mapAsyncUnordered]]
|
|
|
|
|
*/
|
|
|
|
|
def foreachParallel[T](parallelism: Int)(f: T ⇒ Unit)(implicit ec: ExecutionContext): Sink[T, Future[Unit]] =
|
2015-06-14 03:12:30 -04:00
|
|
|
Flow[T].mapAsyncUnordered(parallelism)(t ⇒ Future(f(t))).toMat(Sink.ignore)(Keep.right)
|
2015-06-09 00:05:56 -04:00
|
|
|
|
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-06-14 03:12:30 -04:00
|
|
|
def fold[U, T](zero: U)(f: (U, T) ⇒ U): Sink[T, Future[U]] =
|
|
|
|
|
Flow[T].fold(zero)(f).toMat(Sink.head)(Keep.right).named("foldSink")
|
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] {
|
2015-04-09 22:28:16 +02:00
|
|
|
override def onPush(elem: T, ctx: Context[Unit]): SyncDirective = ctx.pull()
|
2015-06-09 00:05:56 -04:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
override def onUpstreamFailure(cause: Throwable, ctx: Context[Unit]): TerminationDirective = {
|
|
|
|
|
callback(Failure(cause))
|
|
|
|
|
ctx.fail(cause)
|
|
|
|
|
}
|
2015-06-09 00:05:56 -04:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
override def onUpstreamFinish(ctx: Context[Unit]): TerminationDirective = {
|
|
|
|
|
callback(Success[Unit](()))
|
|
|
|
|
ctx.finish()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-04-20 21:04:03 +02:00
|
|
|
Flow[T].transform(newOnCompleteStage).to(Sink.ignore).named("onCompleteSink")
|
2015-01-28 14:19:50 +01: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[T](ref: ActorRef, onCompleteMessage: Any): Sink[T, Unit] =
|
2015-04-20 21:04:03 +02:00
|
|
|
new Sink(new ActorRefSink(ref, onCompleteMessage, DefaultAttributes.actorRefSink, shape("ActorRefSink")))
|
2015-03-30 14:42:30 +02:00
|
|
|
|
2015-10-24 00:07:51 -04:00
|
|
|
/**
|
|
|
|
|
* Sends the elements of the stream to the given `ActorRef` that sends back back-pressure signal.
|
|
|
|
|
* First element is always `onInitMessage`, then stream is waiting for acknowledgement message
|
|
|
|
|
* `ackMessage` from the given actor which means that it is ready to process
|
|
|
|
|
* elements. It also requires `ackMessage` message after each stream element
|
|
|
|
|
* to make backpressure work.
|
|
|
|
|
*
|
|
|
|
|
* If the target actor terminates the stream will be canceled.
|
|
|
|
|
* When the stream is completed successfully the given `onCompleteMessage`
|
|
|
|
|
* will be sent to the destination actor.
|
|
|
|
|
* When the stream is completed with failure - result of `onFailureMessage(throwable)`
|
|
|
|
|
* function will be sent to the destination actor.
|
|
|
|
|
*/
|
|
|
|
|
def actorRefWithAck[T](ref: ActorRef, onInitMessage: Any, ackMessage: Any, onCompleteMessage: Any,
|
|
|
|
|
onFailureMessage: (Throwable) ⇒ Any = Status.Failure): Sink[T, Unit] =
|
|
|
|
|
Sink.fromGraph(new ActorRefBackpressureSinkStage(ref, onInitMessage, ackMessage, onCompleteMessage, onFailureMessage))
|
|
|
|
|
|
2015-03-30 14:42:30 +02:00
|
|
|
/**
|
|
|
|
|
* Creates a `Sink` that is materialized to an [[akka.actor.ActorRef]] which points to an Actor
|
2015-10-09 15:11:01 -04:00
|
|
|
* created according to the passed in [[akka.actor.Props]]. Actor created by the `props` must
|
2015-03-30 14:42:30 +02:00
|
|
|
* be [[akka.stream.actor.ActorSubscriber]].
|
|
|
|
|
*/
|
2015-10-09 15:11:01 -04:00
|
|
|
def actorSubscriber[T](props: Props): Sink[T, ActorRef] = {
|
|
|
|
|
require(classOf[ActorSubscriber].isAssignableFrom(props.actorClass()), "Actor must be ActorSubscriber")
|
2015-04-20 21:04:03 +02:00
|
|
|
new Sink(new ActorSubscriberSink(props, DefaultAttributes.actorSubscriberSink, shape("ActorSubscriberSink")))
|
2015-10-09 15:11:01 -04:00
|
|
|
}
|
2015-03-30 14:42:30 +02:00
|
|
|
|
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
|
2015-11-14 22:42:22 +01:00
|
|
|
* @param timeout Timeout for ``SinkQueue.pull():Future[Option[T]]``
|
2015-08-19 23:04:20 -04:00
|
|
|
*/
|
|
|
|
|
def queue[T](bufferSize: Int, timeout: FiniteDuration = 5.seconds): Sink[T, SinkQueue[T]] = {
|
|
|
|
|
require(bufferSize >= 0, "bufferSize must be greater than or equal to 0")
|
|
|
|
|
new Sink(new AcknowledgeSink(bufferSize, DefaultAttributes.acknowledgeSink, shape("AcknowledgeSink"), timeout))
|
|
|
|
|
}
|
2015-11-14 22:42:22 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Creates a Sink which writes incoming [[ByteString]] elements to the given file and either overwrites
|
|
|
|
|
* or appends to it.
|
|
|
|
|
*
|
|
|
|
|
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
|
|
|
|
|
*
|
|
|
|
|
* This source is backed by an Actor which will use the dedicated `akka.stream.blocking-io-dispatcher`,
|
|
|
|
|
* unless configured otherwise by using [[ActorAttributes]].
|
|
|
|
|
*/
|
|
|
|
|
def file(f: File, append: Boolean = false): Sink[ByteString, Future[Long]] =
|
|
|
|
|
new Sink(new FileSink(f, append, DefaultAttributes.fileSink, shape("FileSink")))
|
2015-11-17 13:17:30 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Creates a Sink which writes incoming [[ByteString]]s to an [[OutputStream]] created by the given function.
|
|
|
|
|
*
|
|
|
|
|
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
|
|
|
|
|
*
|
|
|
|
|
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
|
|
|
|
|
* set it for a given Source by using [[ActorAttributes]].
|
|
|
|
|
*/
|
|
|
|
|
def outputStream(out: () ⇒ OutputStream): Sink[ByteString, Future[Long]] =
|
|
|
|
|
new Sink(new OutputStreamSink(out, DefaultAttributes.outputStreamSink, shape("OutputStreamSink")))
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Creates a Sink which when materialized will return an [[InputStream]] which it is possible
|
|
|
|
|
* to read the values produced by the stream this Sink is attached to.
|
|
|
|
|
*
|
|
|
|
|
* This Sink is intended for inter-operation with legacy APIs since it is inherently blocking.
|
|
|
|
|
*
|
|
|
|
|
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
|
|
|
|
|
* set it for a given Source by using [[ActorAttributes]].
|
|
|
|
|
*
|
|
|
|
|
* @param readTimeout the max time the read operation on the materialized InputStream should block
|
|
|
|
|
*/
|
|
|
|
|
def inputStream(readTimeout: FiniteDuration = 5.seconds): Sink[ByteString, InputStream] =
|
|
|
|
|
Sink.fromGraph(new InputStreamSinkStage(readTimeout)).withAttributes(DefaultAttributes.inputStreamSink)
|
2014-10-17 14:05:50 +02:00
|
|
|
}
|