+str - Adds Sink.last and Sink.lastOption to mirror Sink.head and Sink.headOption
* Renames BlackholeSubscriber to SinkholeSunbscriber * Makes SinkholeSubscriber request Long.MaxValue * SinkholeSink seems like the best name ever
This commit is contained in:
parent
cb8d3c4609
commit
94fe1fb26d
13 changed files with 268 additions and 191 deletions
|
|
@ -3,18 +3,17 @@
|
|||
*/
|
||||
package akka.stream.impl
|
||||
|
||||
import akka.actor.{ Deploy, ActorRef, Props }
|
||||
import akka.dispatch.ExecutionContexts
|
||||
import akka.actor.{ ActorRef, Props }
|
||||
import akka.stream.actor.ActorPublisherMessage.Request
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream._
|
||||
import akka.stream.stage.{ InHandler, GraphStageLogic, SinkStage }
|
||||
import akka.util.Timeout
|
||||
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
|
||||
import org.reactivestreams.{ Publisher, Subscriber }
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.concurrent.duration.{ FiniteDuration, _ }
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import scala.language.postfixOps
|
||||
import scala.util.Try
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -87,75 +86,22 @@ private[akka] final class FanoutPublisherSink[In](
|
|||
new FanoutPublisherSink[In](attr, amendShape(attr))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object HeadSink {
|
||||
final class HeadOptionSinkSubscriber[In] extends Subscriber[In] {
|
||||
private[this] var subscription: Subscription = null
|
||||
private[this] val promise: Promise[Option[In]] = Promise[Option[In]]()
|
||||
def future: Future[Option[In]] = promise.future
|
||||
override def onSubscribe(s: Subscription): Unit = {
|
||||
ReactiveStreamsCompliance.requireNonNullSubscription(s)
|
||||
if (subscription ne null) s.cancel()
|
||||
else {
|
||||
subscription = s
|
||||
s.request(1)
|
||||
}
|
||||
}
|
||||
|
||||
override def onNext(elem: In): Unit = {
|
||||
ReactiveStreamsCompliance.requireNonNullElement(elem)
|
||||
promise.trySuccess(Some(elem))
|
||||
subscription.cancel()
|
||||
subscription = null
|
||||
}
|
||||
|
||||
override def onError(t: Throwable): Unit = {
|
||||
ReactiveStreamsCompliance.requireNonNullException(t)
|
||||
promise.tryFailure(t)
|
||||
}
|
||||
|
||||
override def onComplete(): Unit =
|
||||
promise.trySuccess(None)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Holds a [[scala.concurrent.Future]] that will be fulfilled with the first
|
||||
* element that is signaled to this stream (wrapped in a [[Some]]),
|
||||
* which can be either an element (after which the upstream subscription is canceled),
|
||||
* an error condition (putting the Future into the corresponding failed state) or
|
||||
* the end-of-stream (yielding [[None]]).
|
||||
*/
|
||||
private[akka] final class HeadOptionSink[In](val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, Future[Option[In]]](shape) {
|
||||
override def create(context: MaterializationContext) = {
|
||||
val sub = new HeadSink.HeadOptionSinkSubscriber[In]
|
||||
(sub, sub.future)
|
||||
}
|
||||
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Future[Option[In]]] = new HeadOptionSink[In](attributes, shape)
|
||||
override def withAttributes(attr: Attributes): Module = new HeadOptionSink[In](attr, amendShape(attr))
|
||||
override def toString: String = "HeadOptionSink"
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Attaches a subscriber to this stream which will just discard all received
|
||||
* elements.
|
||||
*/
|
||||
private[akka] final class BlackholeSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, Future[Unit]](shape) {
|
||||
private[akka] final class SinkholeSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, Future[Unit]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val effectiveSettings = ActorMaterializer.downcast(context.materializer).effectiveSettings(context.effectiveAttributes)
|
||||
val p = Promise[Unit]()
|
||||
(new BlackholeSubscriber[Any](effectiveSettings.maxInputBufferSize, p), p.future)
|
||||
(new SinkholeSubscriber[Any](p), p.future)
|
||||
}
|
||||
|
||||
override protected def newInstance(shape: SinkShape[Any]): SinkModule[Any, Future[Unit]] = new BlackholeSink(attributes, shape)
|
||||
override def withAttributes(attr: Attributes): Module = new BlackholeSink(attr, amendShape(attr))
|
||||
override def toString: String = "BlackholeSink"
|
||||
override protected def newInstance(shape: SinkShape[Any]): SinkModule[Any, Future[Unit]] = new SinkholeSink(attributes, shape)
|
||||
override def withAttributes(attr: Attributes): Module = new SinkholeSink(attr, amendShape(attr))
|
||||
override def toString: String = "SinkholeSink"
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -246,3 +192,58 @@ private[akka] final class AcknowledgeSink[In](bufferSize: Int, val attributes: A
|
|||
new AcknowledgeSink[In](bufferSize, attr, amendShape(attr), timeout)
|
||||
override def toString: String = "AcknowledgeSink"
|
||||
}
|
||||
|
||||
private[akka] final class LastOptionStage[T] extends SinkStage[T, Future[Option[T]]]("lastOption") {
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
||||
val p: Promise[Option[T]] = Promise()
|
||||
(new GraphStageLogic(shape) {
|
||||
override def preStart(): Unit = pull(in)
|
||||
setHandler(in, new InHandler {
|
||||
private[this] var prev: T = null.asInstanceOf[T]
|
||||
|
||||
override def onPush(): Unit = {
|
||||
prev = grab(in)
|
||||
pull(in)
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(): Unit = {
|
||||
val head = prev
|
||||
prev = null.asInstanceOf[T]
|
||||
p.trySuccess(Option(head))
|
||||
completeStage()
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(ex: Throwable): Unit = {
|
||||
prev = null.asInstanceOf[T]
|
||||
p.tryFailure(ex)
|
||||
failStage(ex)
|
||||
}
|
||||
})
|
||||
}, p.future)
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] final class HeadOptionStage[T] extends SinkStage[T, Future[Option[T]]]("headOption") {
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
||||
val p: Promise[Option[T]] = Promise()
|
||||
(new GraphStageLogic(shape) {
|
||||
override def preStart(): Unit = pull(in)
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
p.trySuccess(Option(grab(in)))
|
||||
completeStage()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(): Unit = {
|
||||
p.trySuccess(None)
|
||||
completeStage()
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(ex: Throwable): Unit = {
|
||||
p.tryFailure(ex)
|
||||
failStage(ex)
|
||||
}
|
||||
})
|
||||
}, p.future)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue