#19440 replace Scala Future usage with CompletionStage in javadsl
This entails:
* adding akka.pattern.PatternCS.* to enable ask etc. with
CompletionStage
* changing RequestContext to offer an ExecutionContextExecutor for the
CompletionStage.*Async combinators
* splitting up akka.stream.Queue for JavaDSL consistency
This commit is contained in:
parent
396f4370e9
commit
4c72495581
118 changed files with 1646 additions and 1379 deletions
|
|
@ -4,7 +4,6 @@
|
|||
package akka.stream.javadsl
|
||||
|
||||
import java.util.Optional
|
||||
|
||||
import akka.{ Done, NotUsed }
|
||||
import akka.actor.{ ActorRef, Props }
|
||||
import akka.dispatch.ExecutionContexts
|
||||
|
|
@ -12,32 +11,34 @@ import akka.japi.function
|
|||
import akka.stream.impl.StreamLayout
|
||||
import akka.stream.{ javadsl, scaladsl, _ }
|
||||
import org.reactivestreams.{ Publisher, Subscriber }
|
||||
|
||||
import scala.compat.java8.OptionConverters._
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
import scala.concurrent.ExecutionContext
|
||||
import scala.util.Try
|
||||
import java.util.concurrent.CompletionStage
|
||||
import scala.compat.java8.FutureConverters.FutureOps
|
||||
import akka.stream.impl.SinkQueueAdapter
|
||||
|
||||
/** Java API */
|
||||
object Sink {
|
||||
/**
|
||||
* 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
|
||||
* The returned [[java.util.concurrent.CompletionStage]] will be completed with value of the final
|
||||
* function evaluation when the input stream ends, or completed with `Failure`
|
||||
* if there is a failure is signaled in the stream.
|
||||
*/
|
||||
def fold[U, In](zero: U, f: function.Function2[U, In, U]): javadsl.Sink[In, Future[U]] =
|
||||
new Sink(scaladsl.Sink.fold[U, In](zero)(f.apply))
|
||||
def fold[U, In](zero: U, f: function.Function2[U, In, U]): javadsl.Sink[In, CompletionStage[U]] =
|
||||
new Sink(scaladsl.Sink.fold[U, In](zero)(f.apply).toCompletionStage())
|
||||
|
||||
/**
|
||||
* A `Sink` that will invoke the given function for every received element, giving it its previous
|
||||
* output (from the second element) and the element as input.
|
||||
* The returned [[scala.concurrent.Future]] will be completed with value of the final
|
||||
* The returned [[java.util.concurrent.CompletionStage]] will be completed with value of the final
|
||||
* function evaluation when the input stream ends, or completed with `Failure`
|
||||
* if there is a failure signaled in the stream.
|
||||
*/
|
||||
def reduce[In](f: function.Function2[In, In, In]): Sink[In, Future[In]] =
|
||||
new Sink(scaladsl.Sink.reduce[In](f.apply))
|
||||
def reduce[In](f: function.Function2[In, In, In]): Sink[In, CompletionStage[In]] =
|
||||
new Sink(scaladsl.Sink.reduce[In](f.apply).toCompletionStage())
|
||||
|
||||
/**
|
||||
* Helper to create [[Sink]] from `Subscriber`.
|
||||
|
|
@ -54,8 +55,8 @@ object Sink {
|
|||
/**
|
||||
* A `Sink` that will consume the stream and discard the elements.
|
||||
*/
|
||||
def ignore[T](): Sink[T, Future[Done]] =
|
||||
new Sink(scaladsl.Sink.ignore)
|
||||
def ignore[T](): Sink[T, CompletionStage[Done]] =
|
||||
new Sink(scaladsl.Sink.ignore.toCompletionStage())
|
||||
|
||||
/**
|
||||
* A `Sink` that materializes into a [[org.reactivestreams.Publisher]].
|
||||
|
|
@ -73,26 +74,26 @@ object Sink {
|
|||
|
||||
/**
|
||||
* 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
|
||||
* into a [[java.util.concurrent.CompletionStage]] will be completed with `Success` when reaching the
|
||||
* normal end of the stream, or completed with `Failure` if there is a failure is signaled in
|
||||
* the stream..
|
||||
*/
|
||||
def foreach[T](f: function.Procedure[T]): Sink[T, Future[Done]] =
|
||||
new Sink(scaladsl.Sink.foreach(f.apply))
|
||||
def foreach[T](f: function.Procedure[T]): Sink[T, CompletionStage[Done]] =
|
||||
new Sink(scaladsl.Sink.foreach(f.apply).toCompletionStage())
|
||||
|
||||
/**
|
||||
* A `Sink` that will invoke the given procedure for each received element in parallel. The sink is materialized
|
||||
* into a [[scala.concurrent.Future]].
|
||||
* into a [[java.util.concurrent.CompletionStage]].
|
||||
*
|
||||
* If `f` throws an exception and the supervision decision is
|
||||
* [[akka.stream.Supervision.Stop]] the `Future` will be completed with failure.
|
||||
* [[akka.stream.Supervision.Stop]] the `CompletionStage` 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[Done]] =
|
||||
new Sink(scaladsl.Sink.foreachParallel(parallel)(f.apply)(ec))
|
||||
def foreachParallel[T](parallel: Int)(f: function.Procedure[T])(ec: ExecutionContext): Sink[T, CompletionStage[Done]] =
|
||||
new Sink(scaladsl.Sink.foreachParallel(parallel)(f.apply)(ec).toCompletionStage())
|
||||
|
||||
/**
|
||||
* A `Sink` that when the flow is completed, either through a failure or normal
|
||||
|
|
@ -103,60 +104,60 @@ object Sink {
|
|||
new Sink(scaladsl.Sink.onComplete[In](x ⇒ callback.apply(x)))
|
||||
|
||||
/**
|
||||
* A `Sink` that materializes into a `Future` of the first 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.
|
||||
* A `Sink` that materializes into a `CompletionStage` of the first value received.
|
||||
* If the stream completes before signaling at least a single element, the CompletionStage will be failed with a [[NoSuchElementException]].
|
||||
* If the stream signals an error errors before signaling at least a single element, the CompletionStage will be failed with the streams exception.
|
||||
*
|
||||
* See also [[headOption]].
|
||||
*/
|
||||
def head[In](): Sink[In, Future[In]] =
|
||||
new Sink(scaladsl.Sink.head[In])
|
||||
def head[In](): Sink[In, CompletionStage[In]] =
|
||||
new Sink(scaladsl.Sink.head[In].toCompletionStage())
|
||||
|
||||
/**
|
||||
* 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 an empty [[java.util.Optional]].
|
||||
* If the stream signals an error errors before signaling at least a single element, the Future will be failed with the streams exception.
|
||||
* A `Sink` that materializes into a `CompletionStage` of the optional first value received.
|
||||
* If the stream completes before signaling at least a single element, the value of the CompletionStage will be an empty [[java.util.Optional]].
|
||||
* If the stream signals an error errors before signaling at least a single element, the CompletionStage will be failed with the streams exception.
|
||||
*
|
||||
* See also [[head]].
|
||||
*/
|
||||
def headOption[In](): Sink[In, Future[Optional[In]]] =
|
||||
def headOption[In](): Sink[In, CompletionStage[Optional[In]]] =
|
||||
new Sink(scaladsl.Sink.headOption[In].mapMaterializedValue(
|
||||
_.map(_.asJava)(ExecutionContexts.sameThreadExecutionContext)))
|
||||
_.map(_.asJava)(ExecutionContexts.sameThreadExecutionContext).toJava))
|
||||
|
||||
/**
|
||||
* 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.
|
||||
* A `Sink` that materializes into a `CompletionStage` of the last value received.
|
||||
* If the stream completes before signaling at least a single element, the CompletionStage will be failed with a [[NoSuchElementException]].
|
||||
* If the stream signals an error errors before signaling at least a single element, the CompletionStage will be failed with the streams exception.
|
||||
*
|
||||
* See also [[lastOption]].
|
||||
*/
|
||||
def last[In](): Sink[In, Future[In]] =
|
||||
new Sink(scaladsl.Sink.last[In])
|
||||
def last[In](): Sink[In, CompletionStage[In]] =
|
||||
new Sink(scaladsl.Sink.last[In].toCompletionStage())
|
||||
|
||||
/**
|
||||
* 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 an empty [[java.util.Optional]].
|
||||
* If the stream signals an error errors before signaling at least a single element, the Future will be failed with the streams exception.
|
||||
* A `Sink` that materializes into a `CompletionStage` of the optional last value received.
|
||||
* If the stream completes before signaling at least a single element, the value of the CompletionStage will be an empty [[java.util.Optional]].
|
||||
* If the stream signals an error errors before signaling at least a single element, the CompletionStage will be failed with the streams exception.
|
||||
*
|
||||
* See also [[head]].
|
||||
*/
|
||||
def lastOption[In](): Sink[In, Future[Optional[In]]] =
|
||||
def lastOption[In](): Sink[In, CompletionStage[Optional[In]]] =
|
||||
new Sink(scaladsl.Sink.lastOption[In].mapMaterializedValue(
|
||||
_.map(_.asJava)(ExecutionContexts.sameThreadExecutionContext)))
|
||||
_.map(_.asJava)(ExecutionContexts.sameThreadExecutionContext).toJava))
|
||||
|
||||
/**
|
||||
* A `Sink` that keeps on collecting incoming elements until upstream terminates.
|
||||
* As upstream may be unbounded, `Flow[T].take` or the stricter `Flow[T].limit` (and their variants)
|
||||
* may be used to ensure boundedness.
|
||||
* Materializes into a `Future` of `Seq[T]` containing all the collected elements.
|
||||
* Materializes into a `CompletionStage` of `Seq[T]` containing all the collected elements.
|
||||
* `List` is limited to `Integer.MAX_VALUE` elements, this Sink will cancel the stream
|
||||
* after having received that many elements.
|
||||
*
|
||||
* See also [[Flow.limit]], [[Flow.limitWeighted]], [[Flow.take]], [[Flow.takeWithin]], [[Flow.takeWhile]]
|
||||
*/
|
||||
def seq[In]: Sink[In, Future[java.util.List[In]]] = {
|
||||
def seq[In]: Sink[In, CompletionStage[java.util.List[In]]] = {
|
||||
import scala.collection.JavaConverters._
|
||||
new Sink(scaladsl.Sink.seq[In].mapMaterializedValue(fut ⇒ fut.map(sq ⇒ sq.asJava)(ExecutionContexts.sameThreadExecutionContext)))
|
||||
new Sink(scaladsl.Sink.seq[In].mapMaterializedValue(fut ⇒ fut.map(sq ⇒ sq.asJava)(ExecutionContexts.sameThreadExecutionContext).toJava))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -224,10 +225,10 @@ object Sink {
|
|||
|
||||
/**
|
||||
* 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.
|
||||
* [[akka.stream.SinkQueue.pull]] method is pulling element from the stream and returns ``CompletionStage[Option[T]]``.
|
||||
* `CompletionStage` completes when element is available.
|
||||
*
|
||||
* Before calling pull method second time you need to wait until previous Future completes.
|
||||
* Before calling pull method second time you need to wait until previous CompletionStage completes.
|
||||
* Pull returns Failed future with ''IllegalStateException'' if previous future has not yet completed.
|
||||
*
|
||||
* `Sink` will request at most number of elements equal to size of `inputBuffer` from
|
||||
|
|
@ -240,7 +241,7 @@ object Sink {
|
|||
* @see [[akka.stream.SinkQueue]]
|
||||
*/
|
||||
def queue[T](): Sink[T, SinkQueue[T]] =
|
||||
new Sink(scaladsl.Sink.queue())
|
||||
new Sink(scaladsl.Sink.queue[T]().mapMaterializedValue(new SinkQueueAdapter(_)))
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue