#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
|
|
@ -14,6 +14,5 @@ case object Done extends Done {
|
|||
/**
|
||||
* Java API: the singleton instance
|
||||
*/
|
||||
def getInstance() = this
|
||||
def getInstance(): Done = this
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -11,6 +11,8 @@ import java.lang.{ Iterable ⇒ JIterable }
|
|||
import java.util.{ LinkedList ⇒ JLinkedList }
|
||||
import java.util.concurrent.{ Executor, ExecutorService, ExecutionException, Callable, TimeoutException }
|
||||
import scala.util.{ Try, Success, Failure }
|
||||
import java.util.concurrent.CompletionStage
|
||||
import java.util.concurrent.CompletableFuture
|
||||
|
||||
/**
|
||||
* ExecutionContexts is the Java API for ExecutionContexts
|
||||
|
|
@ -111,6 +113,15 @@ object Futures {
|
|||
*/
|
||||
def successful[T](result: T): Future[T] = Future.successful(result)
|
||||
|
||||
/**
|
||||
* Creates an already completed CompletionStage with the specified exception
|
||||
*/
|
||||
def failedCompletionStage[T](ex: Throwable): CompletionStage[T] = {
|
||||
val f = CompletableFuture.completedFuture[T](null.asInstanceOf[T])
|
||||
f.obtrudeException(ex)
|
||||
f
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a Future that will hold the optional result of the first Future with a result that matches the predicate
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -8,6 +8,10 @@ import scala.concurrent.{ ExecutionContext, Promise, Future }
|
|||
import akka.actor._
|
||||
import scala.util.control.NonFatal
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import java.util.concurrent.CompletionStage
|
||||
import java.util.concurrent.CompletableFuture
|
||||
import akka.dispatch.Futures
|
||||
import java.util.function.BiConsumer
|
||||
|
||||
trait FutureTimeoutSupport {
|
||||
/**
|
||||
|
|
@ -22,4 +26,29 @@ trait FutureTimeoutSupport {
|
|||
using.scheduleOnce(duration) { p completeWith { try value catch { case NonFatal(t) ⇒ Future.failed(t) } } }
|
||||
p.future
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a [[scala.concurrent.Future]] that will be completed with the success or failure of the provided value
|
||||
* after the specified duration.
|
||||
*/
|
||||
def afterCompletionStage[T](duration: FiniteDuration, using: Scheduler)(value: ⇒ CompletionStage[T])(implicit ec: ExecutionContext): CompletionStage[T] =
|
||||
if (duration.isFinite() && duration.length < 1) {
|
||||
try value catch { case NonFatal(t) ⇒ Futures.failedCompletionStage(t) }
|
||||
} else {
|
||||
val p = new CompletableFuture[T]
|
||||
using.scheduleOnce(duration) {
|
||||
try {
|
||||
val future = value
|
||||
future.whenComplete(new BiConsumer[T, Throwable] {
|
||||
override def accept(t: T, ex: Throwable): Unit = {
|
||||
if (t != null) p.complete(t)
|
||||
if (ex != null) p.completeExceptionally(ex)
|
||||
}
|
||||
})
|
||||
} catch {
|
||||
case NonFatal(ex) ⇒ p.completeExceptionally(ex)
|
||||
}
|
||||
}
|
||||
p
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,6 +7,8 @@ import akka.actor.{ ActorSelection, Scheduler }
|
|||
import java.util.concurrent.{ Callable, TimeUnit }
|
||||
import scala.concurrent.ExecutionContext
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import java.util.concurrent.CompletionStage
|
||||
import scala.compat.java8.FutureConverters._
|
||||
|
||||
object Patterns {
|
||||
import akka.japi
|
||||
|
|
@ -250,9 +252,262 @@ object Patterns {
|
|||
scalaAfter(duration, scheduler)(value.call())(context)
|
||||
|
||||
/**
|
||||
* Returns a [[scala.concurrent.Future]] that will be completed with the success or failure of the provided value
|
||||
* Returns a [[scala.concurrent.Future]] that will be completed with the success or failure of the provided Callable
|
||||
* after the specified duration.
|
||||
*/
|
||||
def after[T](duration: FiniteDuration, scheduler: Scheduler, context: ExecutionContext, value: Future[T]): Future[T] =
|
||||
scalaAfter(duration, scheduler)(value)(context)
|
||||
}
|
||||
|
||||
object PatternsCS {
|
||||
import akka.japi
|
||||
import akka.actor.{ ActorRef, ActorSystem }
|
||||
import akka.pattern.{ ask ⇒ scalaAsk, pipe ⇒ scalaPipe, gracefulStop ⇒ scalaGracefulStop, after ⇒ scalaAfter }
|
||||
import akka.util.Timeout
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
|
||||
/**
|
||||
* <i>Java API for `akka.pattern.ask`:</i>
|
||||
* Sends a message asynchronously and returns a [[java.util.concurrent.CompletionStage]]
|
||||
* holding the eventual reply message; this means that the target actor
|
||||
* needs to send the result to the `sender` reference provided. The CompletionStage
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
*
|
||||
* <b>Warning:</b>
|
||||
* When using future callbacks, inside actors you need to carefully avoid closing over
|
||||
* the containing actor’s object, i.e. do not call methods or access mutable state
|
||||
* on the enclosing actor from within the callback. This would break the actor
|
||||
* encapsulation and may introduce synchronization bugs and race conditions because
|
||||
* the callback will be scheduled concurrently to the enclosing actor. Unfortunately
|
||||
* there is not yet a way to detect these illegal accesses at compile time.
|
||||
*
|
||||
* <b>Recommended usage:</b>
|
||||
*
|
||||
* {{{
|
||||
* final CompletionStage<Object> f = Patterns.ask(worker, request, timeout);
|
||||
* f.onSuccess(new Procedure<Object>() {
|
||||
* public void apply(Object o) {
|
||||
* nextActor.tell(new EnrichedResult(request, o));
|
||||
* }
|
||||
* });
|
||||
* }}}
|
||||
*/
|
||||
def ask(actor: ActorRef, message: Any, timeout: Timeout): CompletionStage[AnyRef] =
|
||||
scalaAsk(actor, message)(timeout).toJava.asInstanceOf[CompletionStage[AnyRef]]
|
||||
|
||||
/**
|
||||
* A variation of ask which allows to implement "replyTo" pattern by including
|
||||
* sender reference in message.
|
||||
*
|
||||
* {{{
|
||||
* final CompletionStage<Object> f = Patterns.ask(
|
||||
* worker,
|
||||
* new akka.japi.Function<ActorRef, Object> {
|
||||
* Object apply(ActorRef askSender) {
|
||||
* return new Request(askSender);
|
||||
* }
|
||||
* },
|
||||
* timeout);
|
||||
* }}}
|
||||
*/
|
||||
def ask(actor: ActorRef, messageFactory: japi.Function[ActorRef, Any], timeout: Timeout): CompletionStage[AnyRef] =
|
||||
scalaAsk(actor, messageFactory.apply _)(timeout).toJava.asInstanceOf[CompletionStage[AnyRef]]
|
||||
|
||||
/**
|
||||
* <i>Java API for `akka.pattern.ask`:</i>
|
||||
* Sends a message asynchronously and returns a [[java.util.concurrent.CompletionStage]]
|
||||
* holding the eventual reply message; this means that the target actor
|
||||
* needs to send the result to the `sender` reference provided. The CompletionStage
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
*
|
||||
* <b>Warning:</b>
|
||||
* When using future callbacks, inside actors you need to carefully avoid closing over
|
||||
* the containing actor’s object, i.e. do not call methods or access mutable state
|
||||
* on the enclosing actor from within the callback. This would break the actor
|
||||
* encapsulation and may introduce synchronization bugs and race conditions because
|
||||
* the callback will be scheduled concurrently to the enclosing actor. Unfortunately
|
||||
* there is not yet a way to detect these illegal accesses at compile time.
|
||||
*
|
||||
* <b>Recommended usage:</b>
|
||||
*
|
||||
* {{{
|
||||
* final CompletionStage<Object> f = Patterns.ask(worker, request, timeout);
|
||||
* f.onSuccess(new Procedure<Object>() {
|
||||
* public void apply(Object o) {
|
||||
* nextActor.tell(new EnrichedResult(request, o));
|
||||
* }
|
||||
* });
|
||||
* }}}
|
||||
*/
|
||||
def ask(actor: ActorRef, message: Any, timeoutMillis: Long): CompletionStage[AnyRef] =
|
||||
scalaAsk(actor, message)(new Timeout(timeoutMillis, TimeUnit.MILLISECONDS)).toJava.asInstanceOf[CompletionStage[AnyRef]]
|
||||
|
||||
/**
|
||||
* A variation of ask which allows to implement "replyTo" pattern by including
|
||||
* sender reference in message.
|
||||
*
|
||||
* {{{
|
||||
* final CompletionStage<Object> f = Patterns.ask(
|
||||
* worker,
|
||||
* new akka.japi.Function<ActorRef, Object> {
|
||||
* Object apply(ActorRef askSender) {
|
||||
* return new Request(askSender);
|
||||
* }
|
||||
* },
|
||||
* timeout);
|
||||
* }}}
|
||||
*/
|
||||
def ask(actor: ActorRef, messageFactory: japi.Function[ActorRef, Any], timeoutMillis: Long): CompletionStage[AnyRef] =
|
||||
scalaAsk(actor, messageFactory.apply _)(Timeout(timeoutMillis.millis)).toJava.asInstanceOf[CompletionStage[AnyRef]]
|
||||
|
||||
/**
|
||||
* <i>Java API for `akka.pattern.ask`:</i>
|
||||
* Sends a message asynchronously and returns a [[java.util.concurrent.CompletionStage]]
|
||||
* holding the eventual reply message; this means that the target [[akka.actor.ActorSelection]]
|
||||
* needs to send the result to the `sender` reference provided. The CompletionStage
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
*
|
||||
* <b>Warning:</b>
|
||||
* When using future callbacks, inside actors you need to carefully avoid closing over
|
||||
* the containing actor’s object, i.e. do not call methods or access mutable state
|
||||
* on the enclosing actor from within the callback. This would break the actor
|
||||
* encapsulation and may introduce synchronization bugs and race conditions because
|
||||
* the callback will be scheduled concurrently to the enclosing actor. Unfortunately
|
||||
* there is not yet a way to detect these illegal accesses at compile time.
|
||||
*
|
||||
* <b>Recommended usage:</b>
|
||||
*
|
||||
* {{{
|
||||
* final CompletionStage<Object> f = Patterns.ask(selection, request, timeout);
|
||||
* f.onSuccess(new Procedure<Object>() {
|
||||
* public void apply(Object o) {
|
||||
* nextActor.tell(new EnrichedResult(request, o));
|
||||
* }
|
||||
* });
|
||||
* }}}
|
||||
*/
|
||||
def ask(selection: ActorSelection, message: Any, timeout: Timeout): CompletionStage[AnyRef] =
|
||||
scalaAsk(selection, message)(timeout).toJava.asInstanceOf[CompletionStage[AnyRef]]
|
||||
|
||||
/**
|
||||
* <i>Java API for `akka.pattern.ask`:</i>
|
||||
* Sends a message asynchronously and returns a [[java.util.concurrent.CompletionStage]]
|
||||
* holding the eventual reply message; this means that the target [[akka.actor.ActorSelection]]
|
||||
* needs to send the result to the `sender` reference provided. The CompletionStage
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
*
|
||||
* <b>Warning:</b>
|
||||
* When using future callbacks, inside actors you need to carefully avoid closing over
|
||||
* the containing actor’s object, i.e. do not call methods or access mutable state
|
||||
* on the enclosing actor from within the callback. This would break the actor
|
||||
* encapsulation and may introduce synchronization bugs and race conditions because
|
||||
* the callback will be scheduled concurrently to the enclosing actor. Unfortunately
|
||||
* there is not yet a way to detect these illegal accesses at compile time.
|
||||
*
|
||||
* <b>Recommended usage:</b>
|
||||
*
|
||||
* {{{
|
||||
* final CompletionStage<Object> f = Patterns.ask(selection, request, timeout);
|
||||
* f.onSuccess(new Procedure<Object>() {
|
||||
* public void apply(Object o) {
|
||||
* nextActor.tell(new EnrichedResult(request, o));
|
||||
* }
|
||||
* });
|
||||
* }}}
|
||||
*/
|
||||
def ask(selection: ActorSelection, message: Any, timeoutMillis: Long): CompletionStage[AnyRef] =
|
||||
scalaAsk(selection, message)(new Timeout(timeoutMillis, TimeUnit.MILLISECONDS)).toJava.asInstanceOf[CompletionStage[AnyRef]]
|
||||
|
||||
/**
|
||||
* A variation of ask which allows to implement "replyTo" pattern by including
|
||||
* sender reference in message.
|
||||
*
|
||||
* {{{
|
||||
* final CompletionStage<Object> f = Patterns.ask(
|
||||
* selection,
|
||||
* new akka.japi.Function<ActorRef, Object> {
|
||||
* Object apply(ActorRef askSender) {
|
||||
* return new Request(askSender);
|
||||
* }
|
||||
* },
|
||||
* timeout);
|
||||
* }}}
|
||||
*/
|
||||
def ask(selection: ActorSelection, messageFactory: japi.Function[ActorRef, Any], timeoutMillis: Long): CompletionStage[AnyRef] =
|
||||
scalaAsk(selection, messageFactory.apply _)(Timeout(timeoutMillis.millis)).toJava.asInstanceOf[CompletionStage[AnyRef]]
|
||||
|
||||
/**
|
||||
* Register an onComplete callback on this [[java.util.concurrent.CompletionStage]] to send
|
||||
* the result to the given [[akka.actor.ActorRef]] or [[akka.actor.ActorSelection]].
|
||||
* Returns the original CompletionStage to allow method chaining.
|
||||
* If the future was completed with failure it is sent as a [[akka.actor.Status.Failure]]
|
||||
* to the recipient.
|
||||
*
|
||||
* <b>Recommended usage example:</b>
|
||||
*
|
||||
* {{{
|
||||
* final CompletionStage<Object> f = Patterns.ask(worker, request, timeout);
|
||||
* // apply some transformation (i.e. enrich with request info)
|
||||
* final CompletionStage<Object> transformed = f.map(new akka.japi.Function<Object, Object>() { ... });
|
||||
* // send it on to the next stage
|
||||
* Patterns.pipe(transformed).to(nextActor);
|
||||
* }}}
|
||||
*/
|
||||
def pipe[T](future: CompletionStage[T], context: ExecutionContext): PipeableCompletionStage[T] = pipeCompletionStage(future)(context)
|
||||
|
||||
/**
|
||||
* Returns a [[java.util.concurrent.CompletionStage]] that will be completed with success (value `true`) when
|
||||
* existing messages of the target actor has been processed and the actor has been
|
||||
* terminated.
|
||||
*
|
||||
* Useful when you need to wait for termination or compose ordered termination of several actors.
|
||||
*
|
||||
* If the target actor isn't terminated within the timeout the [[java.util.concurrent.CompletionStage]]
|
||||
* is completed with failure [[akka.pattern.AskTimeoutException]].
|
||||
*/
|
||||
def gracefulStop(target: ActorRef, timeout: FiniteDuration): CompletionStage[java.lang.Boolean] =
|
||||
scalaGracefulStop(target, timeout).toJava.asInstanceOf[CompletionStage[java.lang.Boolean]]
|
||||
|
||||
/**
|
||||
* Returns a [[java.util.concurrent.CompletionStage]] that will be completed with success (value `true`) when
|
||||
* existing messages of the target actor has been processed and the actor has been
|
||||
* terminated.
|
||||
*
|
||||
* Useful when you need to wait for termination or compose ordered termination of several actors.
|
||||
*
|
||||
* If you want to invoke specialized stopping logic on your target actor instead of PoisonPill, you can pass your
|
||||
* stop command as `stopMessage` parameter
|
||||
*
|
||||
* If the target actor isn't terminated within the timeout the [[java.util.concurrent.CompletionStage]]
|
||||
* is completed with failure [[akka.pattern.AskTimeoutException]].
|
||||
*/
|
||||
def gracefulStop(target: ActorRef, timeout: FiniteDuration, stopMessage: Any): CompletionStage[java.lang.Boolean] =
|
||||
scalaGracefulStop(target, timeout, stopMessage).toJava.asInstanceOf[CompletionStage[java.lang.Boolean]]
|
||||
|
||||
/**
|
||||
* Returns a [[java.util.concurrent.CompletionStage]] that will be completed with the success or failure of the provided Callable
|
||||
* after the specified duration.
|
||||
*/
|
||||
def after[T](duration: FiniteDuration, scheduler: Scheduler, context: ExecutionContext, value: Callable[CompletionStage[T]]): CompletionStage[T] =
|
||||
afterCompletionStage(duration, scheduler)(value.call())(context)
|
||||
|
||||
/**
|
||||
* Returns a [[java.util.concurrent.CompletionStage]] that will be completed with the success or failure of the provided value
|
||||
* after the specified duration.
|
||||
*/
|
||||
def after[T](duration: FiniteDuration, scheduler: Scheduler, context: ExecutionContext, value: CompletionStage[T]): CompletionStage[T] =
|
||||
afterCompletionStage(duration, scheduler)(value)(context)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,23 +8,23 @@ import scala.concurrent.{ Future, ExecutionContext }
|
|||
import scala.util.{ Failure, Success }
|
||||
import akka.actor.{ Status, ActorRef, Actor }
|
||||
import akka.actor.ActorSelection
|
||||
import java.util.concurrent.CompletionStage
|
||||
import java.util.function.BiConsumer
|
||||
|
||||
trait PipeToSupport {
|
||||
|
||||
final class PipeableFuture[T](val future: Future[T])(implicit executionContext: ExecutionContext) {
|
||||
def pipeTo(recipient: ActorRef)(implicit sender: ActorRef = Actor.noSender): Future[T] = {
|
||||
future onComplete {
|
||||
future andThen {
|
||||
case Success(r) ⇒ recipient ! r
|
||||
case Failure(f) ⇒ recipient ! Status.Failure(f)
|
||||
}
|
||||
future
|
||||
}
|
||||
def pipeToSelection(recipient: ActorSelection)(implicit sender: ActorRef = Actor.noSender): Future[T] = {
|
||||
future onComplete {
|
||||
future andThen {
|
||||
case Success(r) ⇒ recipient ! r
|
||||
case Failure(f) ⇒ recipient ! Status.Failure(f)
|
||||
}
|
||||
future
|
||||
}
|
||||
def to(recipient: ActorRef): PipeableFuture[T] = to(recipient, Actor.noSender)
|
||||
def to(recipient: ActorRef, sender: ActorRef): PipeableFuture[T] = {
|
||||
|
|
@ -38,6 +38,35 @@ trait PipeToSupport {
|
|||
}
|
||||
}
|
||||
|
||||
final class PipeableCompletionStage[T](val future: CompletionStage[T])(implicit executionContext: ExecutionContext) {
|
||||
def pipeTo(recipient: ActorRef)(implicit sender: ActorRef = Actor.noSender): CompletionStage[T] = {
|
||||
future whenComplete new BiConsumer[T, Throwable] {
|
||||
override def accept(t: T, ex: Throwable) {
|
||||
if (t != null) recipient ! t
|
||||
if (ex != null) recipient ! Status.Failure(ex)
|
||||
}
|
||||
}
|
||||
}
|
||||
def pipeToSelection(recipient: ActorSelection)(implicit sender: ActorRef = Actor.noSender): CompletionStage[T] = {
|
||||
future whenComplete new BiConsumer[T, Throwable] {
|
||||
override def accept(t: T, ex: Throwable) {
|
||||
if (t != null) recipient ! t
|
||||
if (ex != null) recipient ! Status.Failure(ex)
|
||||
}
|
||||
}
|
||||
}
|
||||
def to(recipient: ActorRef): PipeableCompletionStage[T] = to(recipient, Actor.noSender)
|
||||
def to(recipient: ActorRef, sender: ActorRef): PipeableCompletionStage[T] = {
|
||||
pipeTo(recipient)(sender)
|
||||
this
|
||||
}
|
||||
def to(recipient: ActorSelection): PipeableCompletionStage[T] = to(recipient, Actor.noSender)
|
||||
def to(recipient: ActorSelection, sender: ActorRef): PipeableCompletionStage[T] = {
|
||||
pipeToSelection(recipient)(sender)
|
||||
this
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Import this implicit conversion to gain the `pipeTo` method on [[scala.concurrent.Future]]:
|
||||
*
|
||||
|
|
@ -56,4 +85,23 @@ trait PipeToSupport {
|
|||
* the failure is sent in a [[akka.actor.Status.Failure]] to the recipient.
|
||||
*/
|
||||
implicit def pipe[T](future: Future[T])(implicit executionContext: ExecutionContext): PipeableFuture[T] = new PipeableFuture(future)
|
||||
|
||||
/**
|
||||
* Import this implicit conversion to gain the `pipeTo` method on [[scala.concurrent.Future]]:
|
||||
*
|
||||
* {{{
|
||||
* import akka.pattern.pipe
|
||||
*
|
||||
* Future { doExpensiveCalc() } pipeTo nextActor
|
||||
*
|
||||
* or
|
||||
*
|
||||
* pipe(someFuture) to nextActor
|
||||
*
|
||||
* }}}
|
||||
*
|
||||
* The successful result of the future is sent as a message to the recipient, or
|
||||
* the failure is sent in a [[akka.actor.Status.Failure]] to the recipient.
|
||||
*/
|
||||
implicit def pipeCompletionStage[T](future: CompletionStage[T])(implicit executionContext: ExecutionContext): PipeableCompletionStage[T] = new PipeableCompletionStage(future)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -22,7 +22,9 @@ import akka.http.javadsl.Http;
|
|||
import scala.util.Try;
|
||||
|
||||
import static akka.http.javadsl.ConnectHttp.toHost;
|
||||
import static akka.pattern.Patterns.*;
|
||||
import static akka.pattern.PatternsCS.*;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public class HttpClientExampleDocTest {
|
||||
|
|
@ -34,9 +36,9 @@ public class HttpClientExampleDocTest {
|
|||
final ActorSystem system = ActorSystem.create();
|
||||
final ActorMaterializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
final Flow<HttpRequest, HttpResponse, Future<OutgoingConnection>> connectionFlow =
|
||||
final Flow<HttpRequest, HttpResponse, CompletionStage<OutgoingConnection>> connectionFlow =
|
||||
Http.get(system).outgoingConnection(toHost("akka.io", 80));
|
||||
final Future<HttpResponse> responseFuture =
|
||||
final CompletionStage<HttpResponse> responseFuture =
|
||||
Source.single(HttpRequest.create("/"))
|
||||
.via(connectionFlow)
|
||||
.runWith(Sink.<HttpResponse>head(), materializer);
|
||||
|
|
@ -58,7 +60,7 @@ public class HttpClientExampleDocTest {
|
|||
|
||||
// construct a pool client flow with context type `Integer`
|
||||
|
||||
final Future<Pair<Try<HttpResponse>, Integer>> responseFuture =
|
||||
final CompletionStage<Pair<Try<HttpResponse>, Integer>> responseFuture =
|
||||
Source
|
||||
.single(Pair.create(HttpRequest.create("/"), 42))
|
||||
.via(poolClientFlow)
|
||||
|
|
@ -72,7 +74,7 @@ public class HttpClientExampleDocTest {
|
|||
final ActorSystem system = ActorSystem.create();
|
||||
final Materializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
final Future<HttpResponse> responseFuture =
|
||||
final CompletionStage<HttpResponse> responseFuture =
|
||||
Http.get(system)
|
||||
.singleRequest(HttpRequest.create("http://akka.io"), materializer);
|
||||
//#single-request-example
|
||||
|
|
@ -92,7 +94,7 @@ public class HttpClientExampleDocTest {
|
|||
}).build());
|
||||
}
|
||||
|
||||
Future<HttpResponse> fetch(String url) {
|
||||
CompletionStage<HttpResponse> fetch(String url) {
|
||||
return http.singleRequest(HttpRequest.create(url), materializer);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,32 +6,25 @@ package docs.http.javadsl.server;
|
|||
|
||||
//#binding-failure-high-level-example
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.dispatch.OnFailure;
|
||||
import akka.http.javadsl.model.ContentTypes;
|
||||
import akka.http.javadsl.server.*;
|
||||
import akka.http.javadsl.server.values.Parameters;
|
||||
import akka.http.scaladsl.Http;
|
||||
import scala.concurrent.Future;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public class HighLevelServerBindFailureExample {
|
||||
public static void main(String[] args) throws IOException {
|
||||
// boot up server using the route as defined below
|
||||
final ActorSystem system = ActorSystem.create();
|
||||
|
||||
// HttpApp.bindRoute expects a route being provided by HttpApp.createRoute
|
||||
Future<Http.ServerBinding> bindingFuture =
|
||||
CompletionStage<Http.ServerBinding> bindingFuture =
|
||||
new HighLevelServerExample().bindRoute("localhost", 8080, system);
|
||||
|
||||
bindingFuture.onFailure(new OnFailure() {
|
||||
@Override
|
||||
public void onFailure(Throwable failure) throws Throwable {
|
||||
bindingFuture.exceptionally(failure -> {
|
||||
System.err.println("Something very bad happened! " + failure.getMessage());
|
||||
system.terminate();
|
||||
}
|
||||
}, system.dispatcher());
|
||||
return null;
|
||||
});
|
||||
|
||||
system.terminate();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -13,6 +13,10 @@ import akka.http.javadsl.server.values.BasicCredentials;
|
|||
import akka.http.javadsl.server.values.HttpBasicAuthenticator;
|
||||
import akka.http.javadsl.testkit.JUnitRouteTest;
|
||||
import akka.http.scaladsl.model.headers.Authorization;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import org.junit.Test;
|
||||
import scala.Option;
|
||||
import scala.concurrent.Future;
|
||||
|
|
@ -27,7 +31,7 @@ public class HttpBasicAuthenticatorExample extends JUnitRouteTest {
|
|||
|
||||
private final String hardcodedPassword = "correcthorsebatterystaple";
|
||||
|
||||
public Future<Option<String>> authenticate(BasicCredentials credentials) {
|
||||
public CompletionStage<Optional<String>> authenticate(BasicCredentials credentials) {
|
||||
// this is where your actual authentication logic would go
|
||||
if (credentials.available() && // no anonymous access
|
||||
credentials.verify(hardcodedPassword)) {
|
||||
|
|
|
|||
|
|
@ -30,13 +30,10 @@ import akka.stream.stage.PushStage;
|
|||
import akka.stream.stage.SyncDirective;
|
||||
import akka.stream.stage.TerminationDirective;
|
||||
import akka.util.ByteString;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Future;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
import scala.runtime.BoxedUnit;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
|
|
@ -47,20 +44,17 @@ public class HttpServerExampleDocTest {
|
|||
ActorSystem system = ActorSystem.create();
|
||||
Materializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
Source<IncomingConnection, Future<ServerBinding>> serverSource =
|
||||
Source<IncomingConnection, CompletionStage<ServerBinding>> serverSource =
|
||||
Http.get(system).bind("localhost", 8080, materializer);
|
||||
|
||||
Future<ServerBinding> serverBindingFuture =
|
||||
serverSource.to(Sink.foreach(
|
||||
new Procedure<IncomingConnection>() {
|
||||
@Override
|
||||
public void apply(IncomingConnection connection) throws Exception {
|
||||
CompletionStage<ServerBinding> serverBindingFuture =
|
||||
serverSource.to(Sink.foreach(connection -> {
|
||||
System.out.println("Accepted new connection from " + connection.remoteAddress());
|
||||
// ... and then actually handle the connection
|
||||
}
|
||||
})).run(materializer);
|
||||
)).run(materializer);
|
||||
//#binding-example
|
||||
Await.result(serverBindingFuture, new FiniteDuration(3, TimeUnit.SECONDS));
|
||||
serverBindingFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
public static void bindingFailureExample() throws Exception {
|
||||
|
|
@ -68,27 +62,21 @@ public class HttpServerExampleDocTest {
|
|||
ActorSystem system = ActorSystem.create();
|
||||
Materializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
Source<IncomingConnection, Future<ServerBinding>> serverSource =
|
||||
Source<IncomingConnection, CompletionStage<ServerBinding>> serverSource =
|
||||
Http.get(system).bind("localhost", 80, materializer);
|
||||
|
||||
Future<ServerBinding> serverBindingFuture =
|
||||
serverSource.to(Sink.foreach(
|
||||
new Procedure<IncomingConnection>() {
|
||||
@Override
|
||||
public void apply(IncomingConnection connection) throws Exception {
|
||||
CompletionStage<ServerBinding> serverBindingFuture =
|
||||
serverSource.to(Sink.foreach(connection -> {
|
||||
System.out.println("Accepted new connection from " + connection.remoteAddress());
|
||||
// ... and then actually handle the connection
|
||||
}
|
||||
})).run(materializer);
|
||||
)).run(materializer);
|
||||
|
||||
serverBindingFuture.onFailure(new OnFailure() {
|
||||
@Override
|
||||
public void onFailure(Throwable failure) throws Throwable {
|
||||
serverBindingFuture.whenCompleteAsync((binding, failure) -> {
|
||||
// possibly report the failure somewhere...
|
||||
}
|
||||
}, system.dispatcher());
|
||||
//#binding-failure-handling
|
||||
Await.result(serverBindingFuture, new FiniteDuration(3, TimeUnit.SECONDS));
|
||||
serverBindingFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
public static void connectionSourceFailureExample() throws Exception {
|
||||
|
|
@ -96,7 +84,7 @@ public class HttpServerExampleDocTest {
|
|||
ActorSystem system = ActorSystem.create();
|
||||
Materializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
Source<IncomingConnection, Future<ServerBinding>> serverSource =
|
||||
Source<IncomingConnection, CompletionStage<ServerBinding>> serverSource =
|
||||
Http.get(system).bind("localhost", 8080, materializer);
|
||||
|
||||
Flow<IncomingConnection, IncomingConnection, NotUsed> failureDetection =
|
||||
|
|
@ -114,19 +102,16 @@ public class HttpServerExampleDocTest {
|
|||
}
|
||||
});
|
||||
|
||||
Future<ServerBinding> serverBindingFuture =
|
||||
CompletionStage<ServerBinding> serverBindingFuture =
|
||||
serverSource
|
||||
.via(failureDetection) // feed signals through our custom stage
|
||||
.to(Sink.foreach(
|
||||
new Procedure<IncomingConnection>() {
|
||||
@Override
|
||||
public void apply(IncomingConnection connection) throws Exception {
|
||||
.to(Sink.foreach(connection -> {
|
||||
System.out.println("Accepted new connection from " + connection.remoteAddress());
|
||||
// ... and then actually handle the connection
|
||||
}
|
||||
})).run(materializer);
|
||||
}))
|
||||
.run(materializer);
|
||||
//#incoming-connections-source-failure-handling
|
||||
Await.result(serverBindingFuture, new FiniteDuration(3, TimeUnit.SECONDS));
|
||||
serverBindingFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
public static void connectionStreamFailureExample() throws Exception {
|
||||
|
|
@ -134,7 +119,7 @@ public class HttpServerExampleDocTest {
|
|||
ActorSystem system = ActorSystem.create();
|
||||
Materializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
Source<IncomingConnection, Future<ServerBinding>> serverSource =
|
||||
Source<IncomingConnection, CompletionStage<ServerBinding>> serverSource =
|
||||
Http.get(system).bind("localhost", 8080, materializer);
|
||||
|
||||
Flow<HttpRequest, HttpRequest, NotUsed> failureDetection =
|
||||
|
|
@ -163,14 +148,14 @@ public class HttpServerExampleDocTest {
|
|||
.withEntity(entity);
|
||||
});
|
||||
|
||||
Future<ServerBinding> serverBindingFuture =
|
||||
serverSource.to(Sink.foreach(con -> {
|
||||
System.out.println("Accepted new connection from " + con.remoteAddress());
|
||||
con.handleWith(httpEcho, materializer);
|
||||
CompletionStage<ServerBinding> serverBindingFuture =
|
||||
serverSource.to(Sink.foreach(conn -> {
|
||||
System.out.println("Accepted new connection from " + conn.remoteAddress());
|
||||
conn.handleWith(httpEcho, materializer);
|
||||
}
|
||||
)).run(materializer);
|
||||
//#connection-stream-failure-handling
|
||||
Await.result(serverBindingFuture, new FiniteDuration(3, TimeUnit.SECONDS));
|
||||
serverBindingFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
public static void fullServerExample() throws Exception {
|
||||
|
|
@ -181,7 +166,7 @@ public class HttpServerExampleDocTest {
|
|||
//#full-server-example
|
||||
final Materializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
Source<IncomingConnection, Future<ServerBinding>> serverSource =
|
||||
Source<IncomingConnection, CompletionStage<ServerBinding>> serverSource =
|
||||
Http.get(system).bind("localhost", 8080, materializer);
|
||||
|
||||
//#request-handler
|
||||
|
|
@ -219,21 +204,17 @@ public class HttpServerExampleDocTest {
|
|||
};
|
||||
//#request-handler
|
||||
|
||||
Future<ServerBinding> serverBindingFuture =
|
||||
serverSource.to(Sink.foreach(
|
||||
new Procedure<IncomingConnection>() {
|
||||
@Override
|
||||
public void apply(IncomingConnection connection) throws Exception {
|
||||
CompletionStage<ServerBinding> serverBindingFuture =
|
||||
serverSource.to(Sink.foreach(connection -> {
|
||||
System.out.println("Accepted new connection from " + connection.remoteAddress());
|
||||
|
||||
connection.handleWithSyncHandler(requestHandler, materializer);
|
||||
// this is equivalent to
|
||||
//connection.handleWith(Flow.of(HttpRequest.class).map(requestHandler), materializer);
|
||||
}
|
||||
})).run(materializer);
|
||||
//#full-server-example
|
||||
|
||||
Await.result(serverBindingFuture, new FiniteDuration(1, TimeUnit.SECONDS)); // will throw if binding fails
|
||||
serverBindingFuture.toCompletableFuture().get(1, TimeUnit.SECONDS); // will throw if binding fails
|
||||
System.out.println("Press ENTER to stop.");
|
||||
new BufferedReader(new InputStreamReader(System.in)).readLine();
|
||||
} finally {
|
||||
|
|
|
|||
|
|
@ -16,6 +16,10 @@ import akka.http.javadsl.server.values.OAuth2Authenticator;
|
|||
import akka.http.javadsl.server.values.OAuth2Credentials;
|
||||
import akka.http.javadsl.testkit.JUnitRouteTest;
|
||||
import akka.http.scaladsl.model.headers.Authorization;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import org.junit.Test;
|
||||
import scala.Option;
|
||||
import scala.concurrent.Future;
|
||||
|
|
@ -31,7 +35,7 @@ public class OAuth2AuthenticatorExample extends JUnitRouteTest {
|
|||
private final String hardcodedToken = "token";
|
||||
|
||||
@Override
|
||||
public Future<Option<String>> authenticate(OAuth2Credentials credentials) {
|
||||
public CompletionStage<Optional<String>> authenticate(OAuth2Credentials credentials) {
|
||||
// this is where your actual authentication logic would go, looking up the user
|
||||
// based on the token or something in that direction
|
||||
if (credentials.available() && // no anonymous access
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package docs.http.javadsl.server;
|
|||
//#websocket-example-using-core
|
||||
import java.io.BufferedReader;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.NotUsed;
|
||||
|
|
@ -50,7 +51,7 @@ public class WebSocketCoreExample {
|
|||
try {
|
||||
final Materializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
Future<ServerBinding> serverBindingFuture =
|
||||
CompletionStage<ServerBinding> serverBindingFuture =
|
||||
Http.get(system).bindAndHandleSync(
|
||||
new Function<HttpRequest, HttpResponse>() {
|
||||
public HttpResponse apply(HttpRequest request) throws Exception {
|
||||
|
|
@ -59,7 +60,7 @@ public class WebSocketCoreExample {
|
|||
}, "localhost", 8080, materializer);
|
||||
|
||||
// will throw if binding fails
|
||||
Await.result(serverBindingFuture, new FiniteDuration(1, TimeUnit.SECONDS));
|
||||
serverBindingFuture.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
System.out.println("Press ENTER to stop.");
|
||||
new BufferedReader(new InputStreamReader(System.in)).readLine();
|
||||
} finally {
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
|
||||
package docs.persistence;
|
||||
|
||||
import static akka.pattern.Patterns.ask;
|
||||
import static akka.pattern.PatternsCS.ask;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.Iterator;
|
||||
|
|
@ -42,6 +42,7 @@ import java.io.Serializable;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class PersistenceQueryDocTest {
|
||||
|
|
@ -343,7 +344,7 @@ public class PersistenceQueryDocTest {
|
|||
|
||||
//#projection-into-different-store-simple-classes
|
||||
class ExampleStore {
|
||||
Future<Void> save(Object any) {
|
||||
CompletionStage<Void> save(Object any) {
|
||||
// ...
|
||||
//#projection-into-different-store-simple-classes
|
||||
return null;
|
||||
|
|
@ -379,13 +380,13 @@ public class PersistenceQueryDocTest {
|
|||
this.name = name;
|
||||
}
|
||||
|
||||
public Future<Long> saveProgress(long offset) {
|
||||
public CompletionStage<Long> saveProgress(long offset) {
|
||||
// ...
|
||||
//#projection-into-different-store
|
||||
return null;
|
||||
//#projection-into-different-store
|
||||
}
|
||||
public Future<Long> latestOffset() {
|
||||
public CompletionStage<Long> latestOffset() {
|
||||
// ...
|
||||
//#projection-into-different-store
|
||||
return null;
|
||||
|
|
@ -412,17 +413,13 @@ public class PersistenceQueryDocTest {
|
|||
final Props writerProps = Props.create(TheOneWhoWritesToQueryJournal.class, "bid");
|
||||
final ActorRef writer = system.actorOf(writerProps, "bid-projection-writer");
|
||||
|
||||
long startFromOffset = Await.result(bidProjection.latestOffset(), timeout.duration());
|
||||
long startFromOffset = bidProjection.latestOffset().toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
readJournal
|
||||
.eventsByTag("bid", startFromOffset)
|
||||
.<Long>mapAsync(8, envelope -> {
|
||||
final Future<Object> f = ask(writer, envelope.event(), timeout);
|
||||
return f.<Long>map(new Mapper<Object, Long>() {
|
||||
@Override public Long apply(Object in) {
|
||||
return envelope.offset();
|
||||
}
|
||||
}, system.dispatcher());
|
||||
.mapAsync(8, envelope -> {
|
||||
final CompletionStage<Object> f = ask(writer, envelope.event(), timeout);
|
||||
return f.thenApplyAsync(in -> envelope.offset(), system.dispatcher());
|
||||
})
|
||||
.mapAsync(1, offset -> bidProjection.saveProgress(offset))
|
||||
.runWith(Sink.ignore(), mat);
|
||||
|
|
|
|||
|
|
@ -6,6 +6,7 @@ package docs.stream;
|
|||
import java.nio.ByteOrder;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.NotUsed;
|
||||
|
|
@ -221,16 +222,15 @@ public class BidiFlowDocTest {
|
|||
);
|
||||
final Flow<Message, Message, NotUsed> flow =
|
||||
stack.atop(stack.reversed()).join(pingpong);
|
||||
final Future<List<Message>> result = Source
|
||||
final CompletionStage<List<Message>> result = Source
|
||||
.from(Arrays.asList(0, 1, 2))
|
||||
.<Message> map(id -> new Ping(id))
|
||||
.via(flow)
|
||||
.grouped(10)
|
||||
.runWith(Sink.<List<Message>> head(), mat);
|
||||
final FiniteDuration oneSec = Duration.create(1, TimeUnit.SECONDS);
|
||||
assertArrayEquals(
|
||||
new Message[] { new Pong(0), new Pong(1), new Pong(2) },
|
||||
Await.result(result, oneSec).toArray(new Message[0]));
|
||||
result.toCompletableFuture().get(1, TimeUnit.SECONDS).toArray(new Message[0]));
|
||||
//#compose
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,6 +5,8 @@ package docs.stream;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import akka.NotUsed;
|
||||
import akka.stream.ClosedShape;
|
||||
|
|
@ -215,27 +217,23 @@ public class CompositionDocTest {
|
|||
|
||||
//#mat-combine-4a
|
||||
static class MyClass {
|
||||
private Promise<Optional<Integer>> p;
|
||||
private CompletableFuture<Optional<Integer>> p;
|
||||
private OutgoingConnection conn;
|
||||
|
||||
public MyClass(Promise<Optional<Integer>> p, OutgoingConnection conn) {
|
||||
public MyClass(CompletableFuture<Optional<Integer>> p, OutgoingConnection conn) {
|
||||
this.p = p;
|
||||
this.conn = conn;
|
||||
}
|
||||
|
||||
public void close() {
|
||||
p.success(Optional.empty());
|
||||
p.complete(Optional.empty());
|
||||
}
|
||||
}
|
||||
|
||||
static class Combiner {
|
||||
static Future<MyClass> f(Promise<Optional<Integer>> p,
|
||||
Pair<Future<OutgoingConnection>, Future<String>> rest) {
|
||||
return rest.first().map(new Mapper<OutgoingConnection, MyClass>() {
|
||||
public MyClass apply(OutgoingConnection c) {
|
||||
return new MyClass(p, c);
|
||||
}
|
||||
}, system.dispatcher());
|
||||
static CompletionStage<MyClass> f(CompletableFuture<Optional<Integer>> p,
|
||||
Pair<CompletionStage<OutgoingConnection>, CompletionStage<String>> rest) {
|
||||
return rest.first().thenApply(c -> new MyClass(p, c));
|
||||
}
|
||||
}
|
||||
//#mat-combine-4a
|
||||
|
|
@ -244,13 +242,13 @@ public class CompositionDocTest {
|
|||
public void materializedValues() throws Exception {
|
||||
//#mat-combine-1
|
||||
// Materializes to Promise<BoxedUnit> (red)
|
||||
final Source<Integer, Promise<Optional<Integer>>> source = Source.<Integer>maybe();
|
||||
final Source<Integer, CompletableFuture<Optional<Integer>>> source = Source.<Integer>maybe();
|
||||
|
||||
// Materializes to BoxedUnit (black)
|
||||
final Flow<Integer, Integer, NotUsed> flow1 = Flow.of(Integer.class).take(100);
|
||||
|
||||
// Materializes to Promise<Option<>> (red)
|
||||
final Source<Integer, Promise<Optional<Integer>>> nestedSource =
|
||||
final Source<Integer, CompletableFuture<Optional<Integer>>> nestedSource =
|
||||
source.viaMat(flow1, Keep.left()).named("nestedSource");
|
||||
//#mat-combine-1
|
||||
|
||||
|
|
@ -260,27 +258,27 @@ public class CompositionDocTest {
|
|||
.map(i -> ByteString.fromString(i.toString()));
|
||||
|
||||
// Materializes to Future<OutgoingConnection> (yellow)
|
||||
final Flow<ByteString, ByteString, Future<OutgoingConnection>> flow3 =
|
||||
final Flow<ByteString, ByteString, CompletionStage<OutgoingConnection>> flow3 =
|
||||
Tcp.get(system).outgoingConnection("localhost", 8080);
|
||||
|
||||
// Materializes to Future<OutgoingConnection> (yellow)
|
||||
final Flow<Integer, ByteString, Future<OutgoingConnection>> nestedFlow =
|
||||
final Flow<Integer, ByteString, CompletionStage<OutgoingConnection>> nestedFlow =
|
||||
flow2.viaMat(flow3, Keep.right()).named("nestedFlow");
|
||||
//#mat-combine-2
|
||||
|
||||
//#mat-combine-3
|
||||
// Materializes to Future<String> (green)
|
||||
final Sink<ByteString, Future<String>> sink = Sink
|
||||
.fold("", (acc, i) -> acc + i.utf8String());
|
||||
final Sink<ByteString, CompletionStage<String>> sink =
|
||||
Sink.<String, ByteString> fold("", (acc, i) -> acc + i.utf8String());
|
||||
|
||||
// Materializes to Pair<Future<OutgoingConnection>, Future<String>> (blue)
|
||||
final Sink<Integer, Pair<Future<OutgoingConnection>, Future<String>>> nestedSink =
|
||||
final Sink<Integer, Pair<CompletionStage<OutgoingConnection>, CompletionStage<String>>> nestedSink =
|
||||
nestedFlow.toMat(sink, Keep.both());
|
||||
//#mat-combine-3
|
||||
|
||||
//#mat-combine-4b
|
||||
// Materializes to Future<MyClass> (purple)
|
||||
final RunnableGraph<Future<MyClass>> runnableGraph =
|
||||
final RunnableGraph<CompletionStage<MyClass>> runnableGraph =
|
||||
nestedSource.toMat(nestedSink, Combiner::f);
|
||||
//#mat-combine-4b
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,6 +6,8 @@ package docs.stream;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
|
|
@ -55,15 +57,12 @@ public class FlowDocTest {
|
|||
|
||||
// returns new Source<Integer>, with `map()` appended
|
||||
final Source<Integer, NotUsed> zeroes = source.map(x -> 0);
|
||||
final Sink<Integer, Future<Integer>> fold =
|
||||
Sink.fold(0, (agg, next) -> agg + next);
|
||||
final Sink<Integer, CompletionStage<Integer>> fold =
|
||||
Sink.<Integer, Integer> fold(0, (agg, next) -> agg + next);
|
||||
zeroes.runWith(fold, mat); // 0
|
||||
//#source-immutable
|
||||
|
||||
int result = Await.result(
|
||||
zeroes.runWith(fold, mat),
|
||||
Duration.create(3, TimeUnit.SECONDS)
|
||||
);
|
||||
int result = zeroes.runWith(fold, mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(0, result);
|
||||
}
|
||||
|
||||
|
|
@ -73,18 +72,18 @@ public class FlowDocTest {
|
|||
final Source<Integer, NotUsed> source =
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
|
||||
// note that the Future is scala.concurrent.Future
|
||||
final Sink<Integer, Future<Integer>> sink =
|
||||
Sink.fold(0, (aggr, next) -> aggr + next);
|
||||
final Sink<Integer, CompletionStage<Integer>> sink =
|
||||
Sink.<Integer, Integer> fold(0, (aggr, next) -> aggr + next);
|
||||
|
||||
// connect the Source to the Sink, obtaining a RunnableFlow
|
||||
final RunnableGraph<Future<Integer>> runnable =
|
||||
final RunnableGraph<CompletionStage<Integer>> runnable =
|
||||
source.toMat(sink, Keep.right());
|
||||
|
||||
// materialize the flow
|
||||
final Future<Integer> sum = runnable.run(mat);
|
||||
final CompletionStage<Integer> sum = runnable.run(mat);
|
||||
//#materialization-in-steps
|
||||
|
||||
int result = Await.result(sum, Duration.create(3, TimeUnit.SECONDS));
|
||||
int result = sum.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(55, result);
|
||||
}
|
||||
|
||||
|
|
@ -93,14 +92,14 @@ public class FlowDocTest {
|
|||
//#materialization-runWith
|
||||
final Source<Integer, NotUsed> source =
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
|
||||
final Sink<Integer, Future<Integer>> sink =
|
||||
Sink.fold(0, (aggr, next) -> aggr + next);
|
||||
final Sink<Integer, CompletionStage<Integer>> sink =
|
||||
Sink.<Integer, Integer> fold(0, (aggr, next) -> aggr + next);
|
||||
|
||||
// materialize the flow, getting the Sinks materialized value
|
||||
final Future<Integer> sum = source.runWith(sink, mat);
|
||||
final CompletionStage<Integer> sum = source.runWith(sink, mat);
|
||||
//#materialization-runWith
|
||||
|
||||
int result = Await.result(sum, Duration.create(3, TimeUnit.SECONDS));
|
||||
int result = sum.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(55, result);
|
||||
}
|
||||
|
||||
|
|
@ -108,21 +107,21 @@ public class FlowDocTest {
|
|||
public void materializedMapUnique() throws Exception {
|
||||
//#stream-reuse
|
||||
// connect the Source to the Sink, obtaining a RunnableGraph
|
||||
final Sink<Integer, Future<Integer>> sink =
|
||||
Sink.fold(0, (aggr, next) -> aggr + next);
|
||||
final RunnableGraph<Future<Integer>> runnable =
|
||||
final Sink<Integer, CompletionStage<Integer>> sink =
|
||||
Sink.<Integer, Integer> fold(0, (aggr, next) -> aggr + next);
|
||||
final RunnableGraph<CompletionStage<Integer>> runnable =
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)).toMat(sink, Keep.right());
|
||||
|
||||
// get the materialized value of the FoldSink
|
||||
final Future<Integer> sum1 = runnable.run(mat);
|
||||
final Future<Integer> sum2 = runnable.run(mat);
|
||||
final CompletionStage<Integer> sum1 = runnable.run(mat);
|
||||
final CompletionStage<Integer> sum2 = runnable.run(mat);
|
||||
|
||||
// sum1 and sum2 are different Futures!
|
||||
//#stream-reuse
|
||||
|
||||
int result1 = Await.result(sum1, Duration.create(3, TimeUnit.SECONDS));
|
||||
int result1 = sum1.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(55, result1);
|
||||
int result2 = Await.result(sum2, Duration.create(3, TimeUnit.SECONDS));
|
||||
int result2 = sum2.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(55, result2);
|
||||
}
|
||||
|
||||
|
|
@ -222,40 +221,40 @@ public class FlowDocTest {
|
|||
//#flow-mat-combine
|
||||
|
||||
// An empty source that can be shut down explicitly from the outside
|
||||
Source<Integer, Promise<Optional<Integer>>> source = Source.<Integer>maybe();
|
||||
Source<Integer, CompletableFuture<Optional<Integer>>> source = Source.<Integer>maybe();
|
||||
|
||||
// A flow that internally throttles elements to 1/second, and returns a Cancellable
|
||||
// which can be used to shut down the stream
|
||||
Flow<Integer, Integer, Cancellable> flow = throttler;
|
||||
|
||||
// A sink that returns the first element of a stream in the returned Future
|
||||
Sink<Integer, Future<Integer>> sink = Sink.head();
|
||||
Sink<Integer, CompletionStage<Integer>> sink = Sink.head();
|
||||
|
||||
|
||||
// By default, the materialized value of the leftmost stage is preserved
|
||||
RunnableGraph<Promise<Optional<Integer>>> r1 = source.via(flow).to(sink);
|
||||
RunnableGraph<CompletableFuture<Optional<Integer>>> r1 = source.via(flow).to(sink);
|
||||
|
||||
// Simple selection of materialized values by using Keep.right
|
||||
RunnableGraph<Cancellable> r2 = source.viaMat(flow, Keep.right()).to(sink);
|
||||
RunnableGraph<Future<Integer>> r3 = source.via(flow).toMat(sink, Keep.right());
|
||||
RunnableGraph<CompletionStage<Integer>> r3 = source.via(flow).toMat(sink, Keep.right());
|
||||
|
||||
// Using runWith will always give the materialized values of the stages added
|
||||
// by runWith() itself
|
||||
Future<Integer> r4 = source.via(flow).runWith(sink, mat);
|
||||
Promise<Optional<Integer>> r5 = flow.to(sink).runWith(source, mat);
|
||||
Pair<Promise<Optional<Integer>>, Future<Integer>> r6 = flow.runWith(source, sink, mat);
|
||||
CompletionStage<Integer> r4 = source.via(flow).runWith(sink, mat);
|
||||
CompletableFuture<Optional<Integer>> r5 = flow.to(sink).runWith(source, mat);
|
||||
Pair<CompletableFuture<Optional<Integer>>, CompletionStage<Integer>> r6 = flow.runWith(source, sink, mat);
|
||||
|
||||
// Using more complext combinations
|
||||
RunnableGraph<Pair<Promise<Optional<Integer>>, Cancellable>> r7 =
|
||||
RunnableGraph<Pair<CompletableFuture<Optional<Integer>>, Cancellable>> r7 =
|
||||
source.viaMat(flow, Keep.both()).to(sink);
|
||||
|
||||
RunnableGraph<Pair<Promise<Optional<Integer>>, Future<Integer>>> r8 =
|
||||
RunnableGraph<Pair<CompletableFuture<Optional<Integer>>, CompletionStage<Integer>>> r8 =
|
||||
source.via(flow).toMat(sink, Keep.both());
|
||||
|
||||
RunnableGraph<Pair<Pair<Promise<Optional<Integer>>, Cancellable>, Future<Integer>>> r9 =
|
||||
RunnableGraph<Pair<Pair<CompletableFuture<Optional<Integer>>, Cancellable>, CompletionStage<Integer>>> r9 =
|
||||
source.viaMat(flow, Keep.both()).toMat(sink, Keep.both());
|
||||
|
||||
RunnableGraph<Pair<Cancellable, Future<Integer>>> r10 =
|
||||
RunnableGraph<Pair<Cancellable, CompletionStage<Integer>>> r10 =
|
||||
source.viaMat(flow, Keep.right()).toMat(sink, Keep.both());
|
||||
|
||||
// It is also possible to map over the materialized values. In r9 we had a
|
||||
|
|
@ -264,9 +263,9 @@ public class FlowDocTest {
|
|||
|
||||
RunnableGraph<Cancellable> r11 =
|
||||
r9.mapMaterializedValue( (nestedTuple) -> {
|
||||
Promise<Optional<Integer>> p = nestedTuple.first().first();
|
||||
CompletableFuture<Optional<Integer>> p = nestedTuple.first().first();
|
||||
Cancellable c = nestedTuple.first().second();
|
||||
Future<Integer> f = nestedTuple.second();
|
||||
CompletionStage<Integer> f = nestedTuple.second();
|
||||
|
||||
// Picking the Cancellable, but we could also construct a domain class here
|
||||
return c;
|
||||
|
|
|
|||
|
|
@ -6,6 +6,7 @@ package docs.stream;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.NotUsed;
|
||||
|
|
@ -49,14 +50,14 @@ public class FlowErrorDocTest {
|
|||
final Materializer mat = ActorMaterializer.create(system);
|
||||
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
|
||||
.map(elem -> 100 / elem);
|
||||
final Sink<Integer, Future<Integer>> fold =
|
||||
Sink.fold(0, (acc, elem) -> acc + elem);
|
||||
final Future<Integer> result = source.runWith(fold, mat);
|
||||
final Sink<Integer, CompletionStage<Integer>> fold =
|
||||
Sink.<Integer, Integer> fold(0, (acc, elem) -> acc + elem);
|
||||
final CompletionStage<Integer> result = source.runWith(fold, mat);
|
||||
// division by zero will fail the stream and the
|
||||
// result here will be a Future completed with Failure(ArithmeticException)
|
||||
//#stop
|
||||
|
||||
Await.result(result, Duration.create(3, TimeUnit.SECONDS));
|
||||
result.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -73,14 +74,14 @@ public class FlowErrorDocTest {
|
|||
system);
|
||||
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
|
||||
.map(elem -> 100 / elem);
|
||||
final Sink<Integer, Future<Integer>> fold =
|
||||
final Sink<Integer, CompletionStage<Integer>> fold =
|
||||
Sink.fold(0, (acc, elem) -> acc + elem);
|
||||
final Future<Integer> result = source.runWith(fold, mat);
|
||||
final CompletionStage<Integer> result = source.runWith(fold, mat);
|
||||
// the element causing division by zero will be dropped
|
||||
// result here will be a Future completed with Success(228)
|
||||
//#resume
|
||||
|
||||
assertEquals(Integer.valueOf(228), Await.result(result, Duration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals(Integer.valueOf(228), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -98,14 +99,14 @@ public class FlowErrorDocTest {
|
|||
.withAttributes(ActorAttributes.withSupervisionStrategy(decider));
|
||||
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
|
||||
.via(flow);
|
||||
final Sink<Integer, Future<Integer>> fold =
|
||||
Sink.fold(0, (acc, elem) -> acc + elem);
|
||||
final Future<Integer> result = source.runWith(fold, mat);
|
||||
final Sink<Integer, CompletionStage<Integer>> fold =
|
||||
Sink.<Integer, Integer> fold(0, (acc, elem) -> acc + elem);
|
||||
final CompletionStage<Integer> result = source.runWith(fold, mat);
|
||||
// the elements causing division by zero will be dropped
|
||||
// result here will be a Future completed with Success(150)
|
||||
//#resume-section
|
||||
|
||||
assertEquals(Integer.valueOf(150), Await.result(result, Duration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals(Integer.valueOf(150), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -126,7 +127,7 @@ public class FlowErrorDocTest {
|
|||
.withAttributes(ActorAttributes.withSupervisionStrategy(decider));
|
||||
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(1, 3, -1, 5, 7))
|
||||
.via(flow);
|
||||
final Future<List<Integer>> result = source.grouped(1000)
|
||||
final CompletionStage<List<Integer>> result = source.grouped(1000)
|
||||
.runWith(Sink.<List<Integer>>head(), mat);
|
||||
// the negative element cause the scan stage to be restarted,
|
||||
// i.e. start from 0 again
|
||||
|
|
@ -135,7 +136,7 @@ public class FlowErrorDocTest {
|
|||
|
||||
assertEquals(
|
||||
Arrays.asList(0, 1, 4, 0, 5, 12),
|
||||
Await.result(result, Duration.create(3, TimeUnit.SECONDS)));
|
||||
result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ import static org.junit.Assert.*;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.NotUsed;
|
||||
|
|
@ -46,15 +47,15 @@ public class FlowGraphDocTest {
|
|||
public void demonstrateBuildSimpleGraph() throws Exception {
|
||||
//#simple-flow-graph
|
||||
final Source<Integer, NotUsed> in = Source.from(Arrays.asList(1, 2, 3, 4, 5));
|
||||
final Sink<List<String>, Future<List<String>>> sink = Sink.head();
|
||||
final Sink<List<Integer>, Future<List<Integer>>> sink2 = Sink.head();
|
||||
final Sink<List<String>, CompletionStage<List<String>>> sink = Sink.head();
|
||||
final Sink<List<Integer>, CompletionStage<List<Integer>>> sink2 = Sink.head();
|
||||
final Flow<Integer, Integer, NotUsed> f1 = Flow.of(Integer.class).map(elem -> elem + 10);
|
||||
final Flow<Integer, Integer, NotUsed> f2 = Flow.of(Integer.class).map(elem -> elem + 20);
|
||||
final Flow<Integer, String, NotUsed> f3 = Flow.of(Integer.class).map(elem -> elem.toString());
|
||||
final Flow<Integer, Integer, NotUsed> f4 = Flow.of(Integer.class).map(elem -> elem + 30);
|
||||
|
||||
final RunnableGraph<Future<List<String>>> result =
|
||||
RunnableGraph.<Future<List<String>>>fromGraph(
|
||||
final RunnableGraph<CompletionStage<List<String>>> result =
|
||||
RunnableGraph.<CompletionStage<List<String>>>fromGraph(
|
||||
GraphDSL
|
||||
.create(
|
||||
sink,
|
||||
|
|
@ -70,7 +71,7 @@ public class FlowGraphDocTest {
|
|||
return ClosedShape.getInstance();
|
||||
}));
|
||||
//#simple-flow-graph
|
||||
final List<String> list = Await.result(result.run(mat), Duration.create(3, TimeUnit.SECONDS));
|
||||
final List<String> list = result.run(mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
final String[] res = list.toArray(new String[] {});
|
||||
Arrays.sort(res, null);
|
||||
assertArrayEquals(new String[] { "31", "32", "33", "34", "35", "41", "42", "43", "44", "45" }, res);
|
||||
|
|
@ -105,12 +106,12 @@ public class FlowGraphDocTest {
|
|||
@Test
|
||||
public void demonstrateReusingFlowInGraph() throws Exception {
|
||||
//#flow-graph-reusing-a-flow
|
||||
final Sink<Integer, Future<Integer>> topHeadSink = Sink.head();
|
||||
final Sink<Integer, Future<Integer>> bottomHeadSink = Sink.head();
|
||||
final Sink<Integer, CompletionStage<Integer>> topHeadSink = Sink.head();
|
||||
final Sink<Integer, CompletionStage<Integer>> bottomHeadSink = Sink.head();
|
||||
final Flow<Integer, Integer, NotUsed> sharedDoubler = Flow.of(Integer.class).map(elem -> elem * 2);
|
||||
|
||||
final RunnableGraph<Pair<Future<Integer>, Future<Integer>>> g =
|
||||
RunnableGraph.<Pair<Future<Integer>, Future<Integer>>>fromGraph(
|
||||
final RunnableGraph<Pair<CompletionStage<Integer>, CompletionStage<Integer>>> g =
|
||||
RunnableGraph.<Pair<CompletionStage<Integer>, CompletionStage<Integer>>>fromGraph(
|
||||
GraphDSL.create(
|
||||
topHeadSink, // import this sink into the graph
|
||||
bottomHeadSink, // and this as well
|
||||
|
|
@ -127,24 +128,22 @@ public class FlowGraphDocTest {
|
|||
)
|
||||
);
|
||||
//#flow-graph-reusing-a-flow
|
||||
final Pair<Future<Integer>, Future<Integer>> pair = g.run(mat);
|
||||
assertEquals(Integer.valueOf(2), Await.result(pair.first(), Duration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals(Integer.valueOf(2), Await.result(pair.second(), Duration.create(3, TimeUnit.SECONDS)));
|
||||
final Pair<CompletionStage<Integer>, CompletionStage<Integer>> pair = g.run(mat);
|
||||
assertEquals(Integer.valueOf(2), pair.first().toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
assertEquals(Integer.valueOf(2), pair.second().toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateMatValue() throws Exception {
|
||||
//#flow-graph-matvalue
|
||||
final Sink<Integer, Future<Integer>> foldSink = Sink.<Integer, Integer> fold(0, (a, b) -> {
|
||||
final Sink<Integer, CompletionStage<Integer>> foldSink = Sink.<Integer, Integer> fold(0, (a, b) -> {
|
||||
return a + b;
|
||||
});
|
||||
|
||||
final Flow<Future<Integer>, Integer, NotUsed> flatten = Flow.<Future<Integer>>create()
|
||||
.mapAsync(4, x -> {
|
||||
return x;
|
||||
});
|
||||
final Flow<CompletionStage<Integer>, Integer, NotUsed> flatten =
|
||||
Flow.<CompletionStage<Integer>>create().mapAsync(4, x -> x);
|
||||
|
||||
final Flow<Integer, Integer, Future<Integer>> foldingFlow = Flow.fromGraph(
|
||||
final Flow<Integer, Integer, CompletionStage<Integer>> foldingFlow = Flow.fromGraph(
|
||||
GraphDSL.create(foldSink,
|
||||
(b, fold) -> {
|
||||
return FlowShape.of(
|
||||
|
|
@ -155,7 +154,7 @@ public class FlowGraphDocTest {
|
|||
|
||||
//#flow-graph-matvalue-cycle
|
||||
// This cannot produce any value:
|
||||
final Source<Integer, Future<Integer>> cyclicSource = Source.fromGraph(
|
||||
final Source<Integer, CompletionStage<Integer>> cyclicSource = Source.fromGraph(
|
||||
GraphDSL.create(foldSink,
|
||||
(b, fold) -> {
|
||||
// - Fold cannot complete until its upstream mapAsync completes
|
||||
|
|
|
|||
|
|
@ -9,6 +9,7 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Predicate;
|
||||
|
|
@ -154,11 +155,11 @@ public class FlowStagesDocTest {
|
|||
|
||||
@Test
|
||||
public void demonstrateVariousPushPullStages() throws Exception {
|
||||
final Sink<Integer, Future<List<Integer>>> sink =
|
||||
final Sink<Integer, CompletionStage<List<Integer>>> sink =
|
||||
Flow.of(Integer.class).grouped(10).toMat(Sink.head(), Keep.right());
|
||||
|
||||
//#stage-chain
|
||||
final RunnableGraph<Future<List<Integer>>> runnable =
|
||||
final RunnableGraph<CompletionStage<List<Integer>>> runnable =
|
||||
Source
|
||||
.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
|
||||
.transform(() -> new Filter<Integer>(elem -> elem % 2 == 0))
|
||||
|
|
@ -168,7 +169,7 @@ public class FlowStagesDocTest {
|
|||
//#stage-chain
|
||||
|
||||
assertEquals(Arrays.asList(1, 1, 2, 2, 3, 3, 4, 4, 5, 5),
|
||||
Await.result(runnable.run(mat), FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
runnable.run(mat).toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
//#detached
|
||||
|
|
|
|||
|
|
@ -31,6 +31,9 @@ import scala.concurrent.duration.Duration;
|
|||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
|
|
@ -101,14 +104,14 @@ public class GraphStageDocTest {
|
|||
Source<Integer, NotUsed> mySource = Source.fromGraph(sourceGraph);
|
||||
|
||||
// Returns 55
|
||||
Future<Integer> result1 = mySource.take(10).runFold(0, (sum, next) -> sum + next, mat);
|
||||
CompletionStage<Integer> result1 = mySource.take(10).runFold(0, (sum, next) -> sum + next, mat);
|
||||
|
||||
// The source is reusable. This returns 5050
|
||||
Future<Integer> result2 = mySource.take(100).runFold(0, (sum, next) -> sum + next, mat);
|
||||
CompletionStage<Integer> result2 = mySource.take(100).runFold(0, (sum, next) -> sum + next, mat);
|
||||
//#simple-source-usage
|
||||
|
||||
assertEquals(Await.result(result1, Duration.create(3, "seconds")), (Integer) 55);
|
||||
assertEquals(Await.result(result2, Duration.create(3, "seconds")), (Integer) 5050);
|
||||
assertEquals(result1.toCompletableFuture().get(3, TimeUnit.SECONDS), (Integer) 55);
|
||||
assertEquals(result2.toCompletableFuture().get(3, TimeUnit.SECONDS), (Integer) 5050);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -169,12 +172,12 @@ public class GraphStageDocTest {
|
|||
}
|
||||
}));
|
||||
|
||||
Future<Integer> result =
|
||||
CompletionStage<Integer> result =
|
||||
Source.from(Arrays.asList("one", "two", "three"))
|
||||
.via(stringLength)
|
||||
.runFold(0, (sum, n) -> sum + n, mat);
|
||||
|
||||
assertEquals(new Integer(11), Await.result(result, Duration.create(3, "seconds")));
|
||||
assertEquals(new Integer(11), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
//#many-to-one
|
||||
|
|
@ -231,12 +234,12 @@ public class GraphStageDocTest {
|
|||
Graph<FlowShape<Integer, Integer>, NotUsed> evenFilter =
|
||||
Flow.fromGraph(new Filter<Integer>(n -> n % 2 == 0));
|
||||
|
||||
Future<Integer> result =
|
||||
CompletionStage<Integer> result =
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6))
|
||||
.via(evenFilter)
|
||||
.runFold(0, (elem, sum) -> sum + elem, mat);
|
||||
|
||||
assertEquals(new Integer(12), Await.result(result, Duration.create(3, "seconds")));
|
||||
assertEquals(new Integer(12), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
//#one-to-many
|
||||
|
|
@ -300,12 +303,12 @@ public class GraphStageDocTest {
|
|||
Graph<FlowShape<Integer, Integer>, NotUsed> duplicator =
|
||||
Flow.fromGraph(new Duplicator<Integer>());
|
||||
|
||||
Future<Integer> result =
|
||||
CompletionStage<Integer> result =
|
||||
Source.from(Arrays.asList(1, 2, 3))
|
||||
.via(duplicator)
|
||||
.runFold(0, (n, sum) -> n + sum, mat);
|
||||
|
||||
assertEquals(new Integer(12), Await.result(result, Duration.create(3, "seconds")));
|
||||
assertEquals(new Integer(12), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -357,20 +360,20 @@ public class GraphStageDocTest {
|
|||
Graph<FlowShape<Integer, Integer>, NotUsed> duplicator =
|
||||
Flow.fromGraph(new Duplicator2<Integer>());
|
||||
|
||||
Future<Integer> result =
|
||||
CompletionStage<Integer> result =
|
||||
Source.from(Arrays.asList(1, 2, 3))
|
||||
.via(duplicator)
|
||||
.runFold(0, (n, sum) -> n + sum, mat);
|
||||
|
||||
assertEquals(new Integer(12), Await.result(result, Duration.create(3, "seconds")));
|
||||
assertEquals(new Integer(12), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateChainingOfGraphStages() throws Exception {
|
||||
Graph<SinkShape<Integer>, Future<String>> sink = Sink.fold("", (acc, n) -> acc + n.toString());
|
||||
Graph<SinkShape<Integer>, CompletionStage<String>> sink = Sink.fold("", (acc, n) -> acc + n.toString());
|
||||
|
||||
//#graph-stage-chain
|
||||
Future<String> resultFuture = Source.from(Arrays.asList(1,2,3,4,5))
|
||||
CompletionStage<String> resultFuture = Source.from(Arrays.asList(1,2,3,4,5))
|
||||
.via(new Filter<Integer>((n) -> n % 2 == 0))
|
||||
.via(new Duplicator<Integer>())
|
||||
.via(new Map<Integer, Integer>((n) -> n / 2))
|
||||
|
|
@ -378,7 +381,7 @@ public class GraphStageDocTest {
|
|||
|
||||
//#graph-stage-chain
|
||||
|
||||
assertEquals("1122", Await.result(resultFuture, Duration.create(3, "seconds")));
|
||||
assertEquals("1122", resultFuture.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -386,9 +389,9 @@ public class GraphStageDocTest {
|
|||
// will close upstream when the future completes
|
||||
public class KillSwitch<A> extends GraphStage<FlowShape<A, A>> {
|
||||
|
||||
private final Future<Done> switchF;
|
||||
private final CompletionStage<Done> switchF;
|
||||
|
||||
public KillSwitch(Future<Done> switchF) {
|
||||
public KillSwitch(CompletionStage<Done> switchF) {
|
||||
this.switchF = switchF;
|
||||
}
|
||||
|
||||
|
|
@ -430,12 +433,7 @@ public class GraphStageDocTest {
|
|||
});
|
||||
|
||||
ExecutionContext ec = system.dispatcher();
|
||||
switchF.onSuccess(new OnSuccess<Done>() {
|
||||
@Override
|
||||
public void onSuccess(Done result) throws Throwable {
|
||||
callback.invoke(Done.getInstance());
|
||||
}
|
||||
}, ec);
|
||||
switchF.thenAccept(callback::invoke);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
@ -446,29 +444,23 @@ public class GraphStageDocTest {
|
|||
public void demonstrateAnAsynchronousSideChannel() throws Exception{
|
||||
|
||||
// tests:
|
||||
Promise<Done> switchF = Futures.promise();
|
||||
CompletableFuture<Done> switchF = new CompletableFuture<>();
|
||||
Graph<FlowShape<Integer, Integer>, NotUsed> killSwitch =
|
||||
Flow.fromGraph(new KillSwitch<>(switchF.future()));
|
||||
Flow.fromGraph(new KillSwitch<>(switchF));
|
||||
|
||||
ExecutionContext ec = system.dispatcher();
|
||||
|
||||
// TODO this is probably racey, is there a way to make sure it happens after?
|
||||
Future<Integer> valueAfterKill = switchF.future().flatMap(new Mapper<Done, Future<Integer>>() {
|
||||
@Override
|
||||
public Future<Integer> apply(Done parameter) {
|
||||
return Futures.successful(4);
|
||||
}
|
||||
}, ec);
|
||||
CompletionStage<Integer> valueAfterKill = switchF.thenApply(in -> 4);
|
||||
|
||||
|
||||
Future<Integer> result =
|
||||
Source.from(Arrays.asList(1, 2, 3)).concat(Source.fromFuture(valueAfterKill))
|
||||
CompletionStage<Integer> result =
|
||||
Source.from(Arrays.asList(1, 2, 3)).concat(Source.fromCompletionStage(valueAfterKill))
|
||||
.via(killSwitch)
|
||||
.runFold(0, (n, sum) -> n + sum, mat);
|
||||
|
||||
switchF.success(Done.getInstance());
|
||||
switchF.complete(Done.getInstance());
|
||||
|
||||
assertEquals(new Integer(6), Await.result(result, Duration.create(3, "seconds")));
|
||||
assertEquals(new Integer(6), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -531,18 +523,18 @@ public class GraphStageDocTest {
|
|||
|
||||
public void demonstrateAGraphStageWithATimer() throws Exception {
|
||||
// tests:
|
||||
Future<Integer> result =
|
||||
CompletionStage<Integer> result =
|
||||
Source.from(Arrays.asList(1, 2, 3))
|
||||
.via(new TimedGate<>(Duration.create(2, "seconds")))
|
||||
.takeWithin(Duration.create(250, "millis"))
|
||||
.runFold(0, (n, sum) -> n + sum, mat);
|
||||
|
||||
assertEquals(new Integer(1), Await.result(result, Duration.create(3, "seconds")));
|
||||
assertEquals(new Integer(1), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
|
||||
//#materialized
|
||||
public class FirstValue<A> extends GraphStageWithMaterializedValue<FlowShape<A, A>, Future<A>> {
|
||||
public class FirstValue<A> extends GraphStageWithMaterializedValue<FlowShape<A, A>, CompletionStage<A>> {
|
||||
|
||||
public final Inlet<A> in = Inlet.create("FirstValue.in");
|
||||
public final Outlet<A> out = Outlet.create("FirstValue.out");
|
||||
|
|
@ -554,7 +546,7 @@ public class GraphStageDocTest {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Tuple2<GraphStageLogic, Future<A>> createLogicAndMaterializedValue(Attributes inheritedAttributes) {
|
||||
public Tuple2<GraphStageLogic, CompletionStage<A>> createLogicAndMaterializedValue(Attributes inheritedAttributes) {
|
||||
Promise<A> promise = Futures.promise();
|
||||
|
||||
GraphStageLogic logic = new GraphStageLogic(shape) {
|
||||
|
|
@ -592,13 +584,13 @@ public class GraphStageDocTest {
|
|||
|
||||
public void demonstrateACustomMaterializedValue() throws Exception {
|
||||
// tests:
|
||||
RunnableGraph<Future<Integer>> flow = Source.from(Arrays.asList(1, 2, 3))
|
||||
RunnableGraph<CompletionStage<Integer>> flow = Source.from(Arrays.asList(1, 2, 3))
|
||||
.viaMat(new FirstValue(), Keep.right())
|
||||
.to(Sink.ignore());
|
||||
|
||||
Future<Integer> result = flow.run(mat);
|
||||
CompletionStage<Integer> result = flow.run(mat);
|
||||
|
||||
assertEquals(new Integer(1), Await.result(result, Duration.create(3, "seconds")));
|
||||
assertEquals(new Integer(1), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -685,11 +677,11 @@ public class GraphStageDocTest {
|
|||
|
||||
public void demonstrateADetachedGraphStage() throws Exception {
|
||||
// tests:
|
||||
Future<Integer> result1 = Source.from(Arrays.asList(1, 2, 3))
|
||||
CompletionStage<Integer> result1 = Source.from(Arrays.asList(1, 2, 3))
|
||||
.via(new TwoBuffer<>())
|
||||
.runFold(0, (acc, n) -> acc + n, mat);
|
||||
|
||||
assertEquals(new Integer(6), Await.result(result1, Duration.create(3, "seconds")));
|
||||
assertEquals(new Integer(6), result1.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
|
||||
TestSubscriber.ManualProbe<Integer> subscriber = TestSubscriber.manualProbe(system);
|
||||
TestPublisher.Probe<Integer> publisher = TestPublisher.probe(0, system);
|
||||
|
|
|
|||
|
|
@ -6,8 +6,6 @@ package docs.stream;
|
|||
|
||||
import akka.NotUsed;
|
||||
import akka.actor.*;
|
||||
import akka.dispatch.Futures;
|
||||
import akka.dispatch.MessageDispatcher;
|
||||
import akka.japi.pf.ReceiveBuilder;
|
||||
import akka.stream.*;
|
||||
import akka.stream.javadsl.*;
|
||||
|
|
@ -21,14 +19,15 @@ import docs.stream.TwitterStreamQuickstartDocTest.Model.Tweet;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.ExecutionContext;
|
||||
import scala.concurrent.Future;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import static akka.pattern.Patterns.ask;
|
||||
import static akka.pattern.PatternsCS.ask;
|
||||
import static docs.stream.TwitterStreamQuickstartDocTest.Model.AKKA;
|
||||
import static docs.stream.TwitterStreamQuickstartDocTest.Model.tweets;
|
||||
import static junit.framework.TestCase.assertTrue;
|
||||
|
|
@ -64,26 +63,26 @@ public class IntegrationDocTest {
|
|||
|
||||
class AddressSystem {
|
||||
//#email-address-lookup
|
||||
public Future<Optional<String>> lookupEmail(String handle)
|
||||
public CompletionStage<Optional<String>> lookupEmail(String handle)
|
||||
//#email-address-lookup
|
||||
{
|
||||
return Futures.successful(Optional.of(handle + "@somewhere.com"));
|
||||
return CompletableFuture.completedFuture(Optional.of(handle + "@somewhere.com"));
|
||||
}
|
||||
|
||||
//#phone-lookup
|
||||
public Future<Optional<String>> lookupPhoneNumber(String handle)
|
||||
public CompletionStage<Optional<String>> lookupPhoneNumber(String handle)
|
||||
//#phone-lookup
|
||||
{
|
||||
return Futures.successful(Optional.of("" + handle.hashCode()));
|
||||
return CompletableFuture.completedFuture(Optional.of("" + handle.hashCode()));
|
||||
}
|
||||
}
|
||||
|
||||
class AddressSystem2 {
|
||||
//#email-address-lookup2
|
||||
public Future<String> lookupEmail(String handle)
|
||||
public CompletionStage<String> lookupEmail(String handle)
|
||||
//#email-address-lookup2
|
||||
{
|
||||
return Futures.successful(handle + "@somewhere.com");
|
||||
return CompletableFuture.completedFuture(handle + "@somewhere.com");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -177,11 +176,11 @@ public class IntegrationDocTest {
|
|||
}
|
||||
|
||||
//#email-server-send
|
||||
public Future<Email> send(Email email) {
|
||||
public CompletionStage<Email> send(Email email) {
|
||||
// ...
|
||||
//#email-server-send
|
||||
probe.tell(email.to, ActorRef.noSender());
|
||||
return Futures.successful(email);
|
||||
return CompletableFuture.completedFuture(email);
|
||||
//#email-server-send
|
||||
}
|
||||
//#email-server-send
|
||||
|
|
@ -258,21 +257,21 @@ public class IntegrationDocTest {
|
|||
|
||||
//#sometimes-slow-service
|
||||
static class SometimesSlowService {
|
||||
private final ExecutionContext ec;
|
||||
private final Executor ec;
|
||||
|
||||
public SometimesSlowService(ExecutionContext ec) {
|
||||
public SometimesSlowService(Executor ec) {
|
||||
this.ec = ec;
|
||||
}
|
||||
|
||||
private final AtomicInteger runningCount = new AtomicInteger();
|
||||
|
||||
public Future<String> convert(String s) {
|
||||
public CompletionStage<String> convert(String s) {
|
||||
System.out.println("running: " + s + "(" + runningCount.incrementAndGet() + ")");
|
||||
return Futures.future(() -> {
|
||||
return CompletableFuture.supplyAsync(() -> {
|
||||
if (!s.isEmpty() && Character.isLowerCase(s.charAt(0)))
|
||||
Thread.sleep(500);
|
||||
try { Thread.sleep(500); } catch (InterruptedException e) {}
|
||||
else
|
||||
Thread.sleep(20);
|
||||
try { Thread.sleep(20); } catch (InterruptedException e) {}
|
||||
System.out.println("completed: " + s + "(" + runningCount.decrementAndGet() + ")");
|
||||
return s.toUpperCase();
|
||||
}, ec);
|
||||
|
|
@ -399,15 +398,12 @@ public class IntegrationDocTest {
|
|||
.map(o -> o.get());
|
||||
|
||||
//#blocking-mapAsync
|
||||
final MessageDispatcher blockingEc = system.dispatchers().lookup("blocking-dispatcher");
|
||||
final Executor blockingEc = system.dispatchers().lookup("blocking-dispatcher");
|
||||
|
||||
final RunnableGraph<NotUsed> sendTextMessages =
|
||||
phoneNumbers
|
||||
.mapAsync(4, phoneNo ->
|
||||
Futures.future(() ->
|
||||
smsServer.send(new TextMessage(phoneNo, "I like your tweet")),
|
||||
blockingEc)
|
||||
)
|
||||
.mapAsync(4, phoneNo -> CompletableFuture.supplyAsync(() ->
|
||||
smsServer.send(new TextMessage(phoneNo, "I like your tweet")), blockingEc))
|
||||
.to(Sink.ignore());
|
||||
|
||||
sendTextMessages.run(mat);
|
||||
|
|
@ -518,7 +514,7 @@ public class IntegrationDocTest {
|
|||
|
||||
{
|
||||
//#sometimes-slow-mapAsync
|
||||
final MessageDispatcher blockingEc = system.dispatchers().lookup("blocking-dispatcher");
|
||||
final Executor blockingEc = system.dispatchers().lookup("blocking-dispatcher");
|
||||
final SometimesSlowService service = new SometimesSlowService(blockingEc);
|
||||
|
||||
final ActorMaterializer mat = ActorMaterializer.create(
|
||||
|
|
@ -563,7 +559,7 @@ public class IntegrationDocTest {
|
|||
|
||||
{
|
||||
//#sometimes-slow-mapAsyncUnordered
|
||||
final MessageDispatcher blockingEc = system.dispatchers().lookup("blocking-dispatcher");
|
||||
final Executor blockingEc = system.dispatchers().lookup("blocking-dispatcher");
|
||||
final SometimesSlowService service = new SometimesSlowService(blockingEc);
|
||||
|
||||
final ActorMaterializer mat = ActorMaterializer.create(
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package docs.stream;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.DoubleStream;
|
||||
|
|
@ -72,13 +73,12 @@ public class RateTransformationDocTest {
|
|||
});
|
||||
//#conflate-summarize
|
||||
|
||||
final Future<List<Tuple3<Double, Double, Integer>>> fut = Source.repeat(0).map(i -> r.nextGaussian())
|
||||
final CompletionStage<List<Tuple3<Double, Double, Integer>>> fut = Source.repeat(0).map(i -> r.nextGaussian())
|
||||
.via(statsFlow)
|
||||
.grouped(10)
|
||||
.runWith(Sink.head(), mat);
|
||||
|
||||
final Duration timeout = Duration.create(100, TimeUnit.MILLISECONDS);
|
||||
Await.result(fut, timeout);
|
||||
fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -97,12 +97,11 @@ public class RateTransformationDocTest {
|
|||
.mapConcat(d -> d);
|
||||
//#conflate-sample
|
||||
|
||||
final Future<Double> fut = Source.from(new ArrayList<Double>(Collections.nCopies(1000, 1.0)))
|
||||
final CompletionStage<Double> fut = Source.from(new ArrayList<Double>(Collections.nCopies(1000, 1.0)))
|
||||
.via(sampleFlow)
|
||||
.runWith(Sink.fold(0.0, (agg, next) -> agg + next), mat);
|
||||
|
||||
final Duration timeout = Duration.create(1, TimeUnit.SECONDS);
|
||||
final Double count = Await.result(fut, timeout);
|
||||
final Double count = fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -112,17 +111,16 @@ public class RateTransformationDocTest {
|
|||
.expand(in -> Stream.iterate(in, i -> i).iterator());
|
||||
//#expand-last
|
||||
|
||||
final Pair<TestPublisher.Probe<Double>, Future<List<Double>>> probeFut = TestSource.<Double> probe(system)
|
||||
final Pair<TestPublisher.Probe<Double>, CompletionStage<List<Double>>> probeFut = TestSource.<Double> probe(system)
|
||||
.via(lastFlow)
|
||||
.grouped(10)
|
||||
.toMat(Sink.head(), Keep.both())
|
||||
.run(mat);
|
||||
|
||||
final TestPublisher.Probe<Double> probe = probeFut.first();
|
||||
final Future<List<Double>> fut = probeFut.second();
|
||||
final CompletionStage<List<Double>> fut = probeFut.second();
|
||||
probe.sendNext(1.0);
|
||||
final Duration timeout = Duration.create(1, TimeUnit.SECONDS);
|
||||
final List<Double> expanded = Await.result(fut, timeout);
|
||||
final List<Double> expanded = fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertEquals(expanded.size(), 10);
|
||||
assertEquals(expanded.stream().mapToDouble(d -> d).sum(), 10, 0.1);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,6 +6,7 @@ package docs.stream;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.Done;
|
||||
|
|
@ -58,10 +59,10 @@ public class StreamPartialFlowGraphDocTest {
|
|||
new Inlet[] {zip1.in0(), zip1.in1(), zip2.in1()});
|
||||
});
|
||||
|
||||
final Sink<Integer, Future<Integer>> resultSink = Sink.<Integer>head();
|
||||
final Sink<Integer, CompletionStage<Integer>> resultSink = Sink.<Integer>head();
|
||||
|
||||
final RunnableGraph<Future<Integer>> g =
|
||||
RunnableGraph.<Future<Integer>>fromGraph(
|
||||
final RunnableGraph<CompletionStage<Integer>> g =
|
||||
RunnableGraph.<CompletionStage<Integer>>fromGraph(
|
||||
GraphDSL.create(resultSink, (builder, sink) -> {
|
||||
// import the partial flow graph explicitly
|
||||
final UniformFanInShape<Integer, Integer> pm = builder.add(pickMaxOfThree);
|
||||
|
|
@ -73,9 +74,9 @@ public class StreamPartialFlowGraphDocTest {
|
|||
return ClosedShape.getInstance();
|
||||
}));
|
||||
|
||||
final Future<Integer> max = g.run(mat);
|
||||
final CompletionStage<Integer> max = g.run(mat);
|
||||
//#simple-partial-flow-graph
|
||||
assertEquals(Integer.valueOf(3), Await.result(max, Duration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals(Integer.valueOf(3), max.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
//#source-from-partial-flow-graph
|
||||
|
|
@ -110,10 +111,10 @@ public class StreamPartialFlowGraphDocTest {
|
|||
return SourceShape.of(zip.out());
|
||||
}));
|
||||
|
||||
final Future<Pair<Integer, Integer>> firstPair =
|
||||
final CompletionStage<Pair<Integer, Integer>> firstPair =
|
||||
pairs.runWith(Sink.<Pair<Integer, Integer>>head(), mat);
|
||||
//#source-from-partial-flow-graph
|
||||
assertEquals(new Pair<>(0, 1), Await.result(firstPair, Duration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals(new Pair<>(0, 1), firstPair.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -132,12 +133,12 @@ public class StreamPartialFlowGraphDocTest {
|
|||
}));
|
||||
|
||||
//#flow-from-partial-flow-graph
|
||||
final Future<Pair<Integer, String>> matSink =
|
||||
final CompletionStage<Pair<Integer, String>> matSink =
|
||||
//#flow-from-partial-flow-graph
|
||||
Source.single(1).via(pairs).runWith(Sink.<Pair<Integer, String>>head(), mat);
|
||||
//#flow-from-partial-flow-graph
|
||||
|
||||
assertEquals(new Pair<>(1, "1"), Await.result(matSink, Duration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals(new Pair<>(1, "1"), matSink.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -150,12 +151,12 @@ public class StreamPartialFlowGraphDocTest {
|
|||
final Source<Integer, NotUsed> sources = Source.combine(source1, source2, new ArrayList<>(),
|
||||
i -> Merge.<Integer>create(i));
|
||||
//#source-combine
|
||||
final Future<Integer> result=
|
||||
final CompletionStage<Integer> result=
|
||||
//#source-combine
|
||||
sources.runWith(Sink.<Integer, Integer>fold(0, (a,b) -> a + b), mat);
|
||||
//#source-combine
|
||||
|
||||
assertEquals(Integer.valueOf(3), Await.result(result, Duration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals(Integer.valueOf(3), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -165,7 +166,7 @@ public class StreamPartialFlowGraphDocTest {
|
|||
|
||||
//#sink-combine
|
||||
Sink<Integer, NotUsed> sendRemotely = Sink.actorRef(actorRef, "Done");
|
||||
Sink<Integer, Future<Done>> localProcessing = Sink.<Integer>foreach(a -> { /*do something useful*/ } );
|
||||
Sink<Integer, CompletionStage<Done>> localProcessing = Sink.<Integer>foreach(a -> { /*do something useful*/ } );
|
||||
Sink<Integer, NotUsed> sinks = Sink.combine(sendRemotely,localProcessing, new ArrayList<>(), a -> Broadcast.create(a));
|
||||
|
||||
Source.<Integer>from(Arrays.asList(new Integer[]{0, 1, 2})).runWith(sinks, mat);
|
||||
|
|
|
|||
|
|
@ -6,6 +6,9 @@ package docs.stream;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.NotUsed;
|
||||
|
|
@ -13,7 +16,6 @@ import org.junit.*;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import akka.actor.*;
|
||||
import akka.dispatch.Futures;
|
||||
import akka.testkit.*;
|
||||
import akka.japi.Pair;
|
||||
import akka.stream.*;
|
||||
|
|
@ -23,7 +25,6 @@ import akka.stream.testkit.javadsl.*;
|
|||
import akka.testkit.TestProbe;
|
||||
import scala.util.*;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Future;
|
||||
import scala.concurrent.duration.Duration;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
|
|
@ -48,13 +49,13 @@ public class StreamTestKitDocTest {
|
|||
@Test
|
||||
public void strictCollection() throws Exception {
|
||||
//#strict-collection
|
||||
final Sink<Integer, Future<Integer>> sinkUnderTest = Flow.of(Integer.class)
|
||||
final Sink<Integer, CompletionStage<Integer>> sinkUnderTest = Flow.of(Integer.class)
|
||||
.map(i -> i * 2)
|
||||
.toMat(Sink.fold(0, (agg, next) -> agg + next), Keep.right());
|
||||
|
||||
final Future<Integer> future = Source.from(Arrays.asList(1, 2, 3, 4))
|
||||
final CompletionStage<Integer> future = Source.from(Arrays.asList(1, 2, 3, 4))
|
||||
.runWith(sinkUnderTest, mat);
|
||||
final Integer result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final Integer result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assert(result == 20);
|
||||
//#strict-collection
|
||||
}
|
||||
|
|
@ -65,11 +66,10 @@ public class StreamTestKitDocTest {
|
|||
final Source<Integer, NotUsed> sourceUnderTest = Source.repeat(1)
|
||||
.map(i -> i * 2);
|
||||
|
||||
final Future<List<Integer>> future = sourceUnderTest
|
||||
final CompletionStage<List<Integer>> future = sourceUnderTest
|
||||
.grouped(10)
|
||||
.runWith(Sink.head(), mat);
|
||||
final List<Integer> result =
|
||||
Await.result(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final List<Integer> result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertEquals(result, Collections.nCopies(10, 2));
|
||||
//#grouped-infinite
|
||||
}
|
||||
|
|
@ -80,9 +80,9 @@ public class StreamTestKitDocTest {
|
|||
final Flow<Integer, Integer, NotUsed> flowUnderTest = Flow.of(Integer.class)
|
||||
.takeWhile(i -> i < 5);
|
||||
|
||||
final Future<Integer> future = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6))
|
||||
final CompletionStage<Integer> future = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6))
|
||||
.via(flowUnderTest).runWith(Sink.fold(0, (agg, next) -> agg + next), mat);
|
||||
final Integer result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final Integer result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assert(result == 10);
|
||||
//#folded-stream
|
||||
}
|
||||
|
|
@ -95,10 +95,10 @@ public class StreamTestKitDocTest {
|
|||
.grouped(2);
|
||||
|
||||
final TestProbe probe = new TestProbe(system);
|
||||
final Future<List<List<Integer>>> future = sourceUnderTest
|
||||
final CompletionStage<List<List<Integer>>> future = sourceUnderTest
|
||||
.grouped(2)
|
||||
.runWith(Sink.head(), mat);
|
||||
akka.pattern.Patterns.pipe(future, system.dispatcher()).to(probe.ref());
|
||||
akka.pattern.PatternsCS.pipe(future, system.dispatcher()).to(probe.ref());
|
||||
probe.expectMsg(Duration.create(1, TimeUnit.SECONDS),
|
||||
Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4))
|
||||
);
|
||||
|
|
@ -129,23 +129,23 @@ public class StreamTestKitDocTest {
|
|||
@Test
|
||||
public void sourceActorRef() throws Exception {
|
||||
//#source-actorref
|
||||
final Sink<Integer, Future<String>> sinkUnderTest = Flow.of(Integer.class)
|
||||
final Sink<Integer, CompletionStage<String>> sinkUnderTest = Flow.of(Integer.class)
|
||||
.map(i -> i.toString())
|
||||
.toMat(Sink.fold("", (agg, next) -> agg + next), Keep.right());
|
||||
|
||||
final Pair<ActorRef, Future<String>> refAndFuture =
|
||||
final Pair<ActorRef, CompletionStage<String>> refAndCompletionStage =
|
||||
Source.<Integer>actorRef(8, OverflowStrategy.fail())
|
||||
.toMat(sinkUnderTest, Keep.both())
|
||||
.run(mat);
|
||||
final ActorRef ref = refAndFuture.first();
|
||||
final Future<String> future = refAndFuture.second();
|
||||
final ActorRef ref = refAndCompletionStage.first();
|
||||
final CompletionStage<String> future = refAndCompletionStage.second();
|
||||
|
||||
ref.tell(1, ActorRef.noSender());
|
||||
ref.tell(2, ActorRef.noSender());
|
||||
ref.tell(3, ActorRef.noSender());
|
||||
ref.tell(new akka.actor.Status.Success("done"), ActorRef.noSender());
|
||||
|
||||
final String result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final String result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertEquals(result, "123");
|
||||
//#source-actorref
|
||||
}
|
||||
|
|
@ -180,19 +180,23 @@ public class StreamTestKitDocTest {
|
|||
@Test
|
||||
public void injectingFailure() throws Exception {
|
||||
//#injecting-failure
|
||||
final Sink<Integer, Future<Integer>> sinkUnderTest = Sink.head();
|
||||
final Sink<Integer, CompletionStage<Integer>> sinkUnderTest = Sink.head();
|
||||
|
||||
final Pair<TestPublisher.Probe<Integer>, Future<Integer>> probeAndFuture =
|
||||
final Pair<TestPublisher.Probe<Integer>, CompletionStage<Integer>> probeAndCompletionStage =
|
||||
TestSource.<Integer>probe(system)
|
||||
.toMat(sinkUnderTest, Keep.both())
|
||||
.run(mat);
|
||||
final TestPublisher.Probe<Integer> probe = probeAndFuture.first();
|
||||
final Future<Integer> future = probeAndFuture.second();
|
||||
final TestPublisher.Probe<Integer> probe = probeAndCompletionStage.first();
|
||||
final CompletionStage<Integer> future = probeAndCompletionStage.second();
|
||||
probe.sendError(new Exception("boom"));
|
||||
|
||||
Await.ready(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final Throwable exception = ((Failure)future.value().get()).exception();
|
||||
try {
|
||||
future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assert false;
|
||||
} catch (ExecutionException ee) {
|
||||
final Throwable exception = ee.getCause();
|
||||
assertEquals(exception.getMessage(), "boom");
|
||||
}
|
||||
//#injecting-failure
|
||||
}
|
||||
|
||||
|
|
@ -200,11 +204,11 @@ public class StreamTestKitDocTest {
|
|||
public void testSourceAndTestSink() throws Exception {
|
||||
//#test-source-and-sink
|
||||
final Flow<Integer, Integer, NotUsed> flowUnderTest = Flow.of(Integer.class)
|
||||
.mapAsyncUnordered(2, sleep -> akka.pattern.Patterns.after(
|
||||
.mapAsyncUnordered(2, sleep -> akka.pattern.PatternsCS.after(
|
||||
Duration.create(10, TimeUnit.MILLISECONDS),
|
||||
system.scheduler(),
|
||||
system.dispatcher(),
|
||||
Futures.successful(sleep)
|
||||
CompletableFuture.completedFuture(sleep)
|
||||
));
|
||||
|
||||
final Pair<TestPublisher.Probe<Integer>, TestSubscriber.Probe<Integer>> pubAndSub =
|
||||
|
|
|
|||
|
|
@ -26,6 +26,8 @@ import scala.concurrent.duration.FiniteDuration;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.stream.Collectors;
|
||||
|
|
@ -199,14 +201,14 @@ public class TwitterStreamQuickstartDocTest {
|
|||
}
|
||||
|
||||
static class Example2 {
|
||||
public void run(final Materializer mat) throws TimeoutException, InterruptedException {
|
||||
public void run(final Materializer mat) throws TimeoutException, InterruptedException, ExecutionException {
|
||||
//#backpressure-by-readline
|
||||
final Future<?> completion =
|
||||
final CompletionStage<Done> completion =
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
|
||||
.map(i -> { System.out.println("map => " + i); return i; })
|
||||
.runForeach(i -> System.console().readLine("Element = %s continue reading? [press enter]\n", i), mat);
|
||||
|
||||
Await.ready(completion, FiniteDuration.create(1, TimeUnit.MINUTES));
|
||||
completion.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
//#backpressure-by-readline
|
||||
}
|
||||
}
|
||||
|
|
@ -276,8 +278,8 @@ public class TwitterStreamQuickstartDocTest {
|
|||
|
||||
@Test
|
||||
public void demonstrateBroadcast() {
|
||||
final Sink<Author, Future<Done>> writeAuthors = Sink.ignore();
|
||||
final Sink<Hashtag, Future<Done>> writeHashtags = Sink.ignore();
|
||||
final Sink<Author, CompletionStage<Done>> writeAuthors = Sink.ignore();
|
||||
final Sink<Hashtag, CompletionStage<Done>> writeHashtags = Sink.ignore();
|
||||
|
||||
//#flow-graph-broadcast
|
||||
RunnableGraph.fromGraph(GraphDSL.create(b -> {
|
||||
|
|
@ -317,24 +319,21 @@ public class TwitterStreamQuickstartDocTest {
|
|||
@Test
|
||||
public void demonstrateCountOnFiniteStream() {
|
||||
//#tweets-fold-count
|
||||
final Sink<Integer, Future<Integer>> sumSink =
|
||||
final Sink<Integer, CompletionStage<Integer>> sumSink =
|
||||
Sink.<Integer, Integer>fold(0, (acc, elem) -> acc + elem);
|
||||
|
||||
final RunnableGraph<Future<Integer>> counter =
|
||||
final RunnableGraph<CompletionStage<Integer>> counter =
|
||||
tweets.map(t -> 1).toMat(sumSink, Keep.right());
|
||||
|
||||
final Future<Integer> sum = counter.run(mat);
|
||||
final CompletionStage<Integer> sum = counter.run(mat);
|
||||
|
||||
sum.foreach(new Foreach<Integer>() {
|
||||
public void each(Integer c) {
|
||||
System.out.println("Total tweets processed: " + c);
|
||||
}
|
||||
}, system.dispatcher());
|
||||
sum.thenAcceptAsync(c -> System.out.println("Total tweets processed: " + c),
|
||||
system.dispatcher());
|
||||
//#tweets-fold-count
|
||||
|
||||
new Object() {
|
||||
//#tweets-fold-count-oneline
|
||||
final Future<Integer> sum = tweets.map(t -> 1).runWith(sumSink, mat);
|
||||
final CompletionStage<Integer> sum = tweets.map(t -> 1).runWith(sumSink, mat);
|
||||
//#tweets-fold-count-oneline
|
||||
};
|
||||
}
|
||||
|
|
@ -344,18 +343,18 @@ public class TwitterStreamQuickstartDocTest {
|
|||
final Source<Tweet, NotUsed> tweetsInMinuteFromNow = tweets; // not really in second, just acting as if
|
||||
|
||||
//#tweets-runnable-flow-materialized-twice
|
||||
final Sink<Integer, Future<Integer>> sumSink =
|
||||
final Sink<Integer, CompletionStage<Integer>> sumSink =
|
||||
Sink.<Integer, Integer>fold(0, (acc, elem) -> acc + elem);
|
||||
final RunnableGraph<Future<Integer>> counterRunnableGraph =
|
||||
final RunnableGraph<CompletionStage<Integer>> counterRunnableGraph =
|
||||
tweetsInMinuteFromNow
|
||||
.filter(t -> t.hashtags().contains(AKKA))
|
||||
.map(t -> 1)
|
||||
.toMat(sumSink, Keep.right());
|
||||
|
||||
// materialize the stream once in the morning
|
||||
final Future<Integer> morningTweetsCount = counterRunnableGraph.run(mat);
|
||||
final CompletionStage<Integer> morningTweetsCount = counterRunnableGraph.run(mat);
|
||||
// and once in the evening, reusing the blueprint
|
||||
final Future<Integer> eveningTweetsCount = counterRunnableGraph.run(mat);
|
||||
final CompletionStage<Integer> eveningTweetsCount = counterRunnableGraph.run(mat);
|
||||
//#tweets-runnable-flow-materialized-twice
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,10 +3,9 @@
|
|||
*/
|
||||
package docs.stream.io;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import akka.Done;
|
||||
import akka.actor.ActorSystem;
|
||||
|
|
@ -15,7 +14,6 @@ import akka.stream.io.IOResult;
|
|||
import akka.stream.javadsl.Sink;
|
||||
import akka.stream.javadsl.FileIO;
|
||||
import docs.stream.SilenceSystemOut;
|
||||
import docs.stream.cookbook.RecipeParseLines;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
|
@ -56,10 +54,10 @@ public class StreamFileDocTest {
|
|||
|
||||
try {
|
||||
//#file-source
|
||||
Sink<ByteString, Future<Done>> printlnSink =
|
||||
Sink.foreach(chunk -> System.out.println(chunk.utf8String()));
|
||||
Sink<ByteString, CompletionStage<Done>> printlnSink =
|
||||
Sink.<ByteString> foreach(chunk -> System.out.println(chunk.utf8String()));
|
||||
|
||||
Future<IOResult> ioResult =
|
||||
CompletionStage<IOResult> ioResult =
|
||||
FileIO.fromFile(file)
|
||||
.to(printlnSink)
|
||||
.run(mat);
|
||||
|
|
@ -74,7 +72,7 @@ public class StreamFileDocTest {
|
|||
final File file = File.createTempFile(getClass().getName(), ".tmp");
|
||||
|
||||
try {
|
||||
Sink<ByteString, Future<IOResult>> fileSink =
|
||||
Sink<ByteString, CompletionStage<IOResult>> fileSink =
|
||||
//#custom-dispatcher-code
|
||||
FileIO.toFile(file)
|
||||
.withAttributes(ActorAttributes.dispatcher("custom-blocking-io-dispatcher"));
|
||||
|
|
|
|||
|
|
@ -3,6 +3,7 @@
|
|||
*/
|
||||
package docs.stream.io;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
|
||||
import akka.NotUsed;
|
||||
|
|
@ -60,14 +61,14 @@ public class StreamTcpDocTest {
|
|||
{
|
||||
//#echo-server-simple-bind
|
||||
// IncomingConnection and ServerBinding imported from Tcp
|
||||
final Source<IncomingConnection, Future<ServerBinding>> connections =
|
||||
final Source<IncomingConnection, CompletionStage<ServerBinding>> connections =
|
||||
Tcp.get(system).bind("127.0.0.1", 8889);
|
||||
//#echo-server-simple-bind
|
||||
}
|
||||
{
|
||||
|
||||
final InetSocketAddress localhost = SocketUtils.temporaryServerAddress();
|
||||
final Source<IncomingConnection, Future<ServerBinding>> connections =
|
||||
final Source<IncomingConnection, CompletionStage<ServerBinding>> connections =
|
||||
Tcp.get(system).bind(localhost.getHostName(), localhost.getPort()); // TODO getHostString in Java7
|
||||
|
||||
//#echo-server-simple-handle
|
||||
|
|
@ -93,7 +94,7 @@ public class StreamTcpDocTest {
|
|||
|
||||
final TestProbe serverProbe = new TestProbe(system);
|
||||
|
||||
final Source<IncomingConnection,Future<ServerBinding>> connections =
|
||||
final Source<IncomingConnection,CompletionStage<ServerBinding>> connections =
|
||||
Tcp.get(system).bind(localhost.getHostName(), localhost.getPort()); // TODO getHostString in Java7
|
||||
//#welcome-banner-chat-server
|
||||
connections.runForeach(connection -> {
|
||||
|
|
@ -146,14 +147,14 @@ public class StreamTcpDocTest {
|
|||
|
||||
{
|
||||
//#repl-client
|
||||
final Flow<ByteString, ByteString, Future<OutgoingConnection>> connection =
|
||||
final Flow<ByteString, ByteString, CompletionStage<OutgoingConnection>> connection =
|
||||
Tcp.get(system).outgoingConnection("127.0.0.1", 8889);
|
||||
//#repl-client
|
||||
}
|
||||
|
||||
{
|
||||
final Flow<ByteString, ByteString, Future<OutgoingConnection>> connection =
|
||||
Tcp.get(system).outgoingConnection(localhost.getHostName(), localhost.getPort()); // TODO getHostString in Java7
|
||||
final Flow<ByteString, ByteString, CompletionStage<OutgoingConnection>> connection =
|
||||
Tcp.get(system).outgoingConnection(localhost.getHostString(), localhost.getPort());
|
||||
//#repl-client
|
||||
|
||||
final PushStage<String, ByteString> replParser = new PushStage<String, ByteString>() {
|
||||
|
|
|
|||
|
|
@ -20,12 +20,10 @@ import org.junit.AfterClass;
|
|||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.Tuple2;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Future;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
|
@ -97,9 +95,9 @@ public class RecipeByteStrings extends RecipeTest {
|
|||
rawBytes.transform(() -> new Chunker(CHUNK_LIMIT));
|
||||
//#bytestring-chunker2
|
||||
|
||||
Future<List<ByteString>> chunksFuture = chunksStream.grouped(10).runWith(Sink.head(), mat);
|
||||
CompletionStage<List<ByteString>> chunksFuture = chunksStream.grouped(10).runWith(Sink.head(), mat);
|
||||
|
||||
List<ByteString> chunks = Await.result(chunksFuture, FiniteDuration.create(3, TimeUnit.SECONDS));
|
||||
List<ByteString> chunks = chunksFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
for (ByteString chunk : chunks) {
|
||||
assertTrue(chunk.size() <= 2);
|
||||
|
|
@ -159,9 +157,7 @@ public class RecipeByteStrings extends RecipeTest {
|
|||
ByteString.fromArray(new byte[] { 4, 5, 6 }),
|
||||
ByteString.fromArray(new byte[] { 7, 8, 9, 10 })));
|
||||
|
||||
FiniteDuration threeSeconds = FiniteDuration.create(3, TimeUnit.SECONDS);
|
||||
|
||||
List<ByteString> got = Await.result(bytes1.via(limiter).grouped(10).runWith(Sink.head(), mat), threeSeconds);
|
||||
List<ByteString> got = bytes1.via(limiter).grouped(10).runWith(Sink.head(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
ByteString acc = ByteString.empty();
|
||||
for (ByteString b : got) {
|
||||
acc = acc.concat(b);
|
||||
|
|
@ -170,7 +166,7 @@ public class RecipeByteStrings extends RecipeTest {
|
|||
|
||||
boolean thrown = false;
|
||||
try {
|
||||
Await.result(bytes2.via(limiter).grouped(10).runWith(Sink.head(), mat), threeSeconds);
|
||||
bytes2.via(limiter).grouped(10).runWith(Sink.head(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
} catch (IllegalStateException ex) {
|
||||
thrown = true;
|
||||
}
|
||||
|
|
@ -194,8 +190,7 @@ public class RecipeByteStrings extends RecipeTest {
|
|||
Source<ByteString, NotUsed> compacted = rawBytes.map(bs -> bs.compact());
|
||||
//#compacting-bytestrings
|
||||
|
||||
FiniteDuration timeout = FiniteDuration.create(3, TimeUnit.SECONDS);
|
||||
List<ByteString> got = Await.result(compacted.grouped(10).runWith(Sink.head(), mat), timeout);
|
||||
List<ByteString> got = compacted.grouped(10).runWith(Sink.head(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
for (ByteString byteString : got) {
|
||||
assertTrue(byteString.isCompact());
|
||||
|
|
|
|||
|
|
@ -18,8 +18,6 @@ import akka.util.ByteString;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.duration.Duration;
|
||||
|
||||
import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
|
|
@ -89,7 +87,7 @@ public class RecipeDigest extends RecipeTest {
|
|||
.transform(() -> digestCalculator("SHA-256"));
|
||||
//#calculating-digest2
|
||||
|
||||
ByteString got = Await.result(digest.runWith(Sink.head(), mat), Duration.create(3, TimeUnit.SECONDS));
|
||||
ByteString got = digest.runWith(Sink.head(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(ByteString.fromInts(
|
||||
0x24, 0x8d, 0x6a, 0x61,
|
||||
0xd2, 0x06, 0x38, 0xb8,
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import scala.concurrent.Future;
|
|||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class RecipeDroppyBroadcast extends RecipeTest {
|
||||
static ActorSystem system;
|
||||
|
|
@ -38,7 +39,7 @@ public class RecipeDroppyBroadcast extends RecipeTest {
|
|||
new JavaTestKit(system) {
|
||||
//#droppy-bcast
|
||||
// Makes a sink drop elements if too slow
|
||||
public <T> Sink<T, Future<Done>> droppySink(Sink<T, Future<Done>> sink, int size) {
|
||||
public <T> Sink<T, CompletionStage<Done>> droppySink(Sink<T, CompletionStage<Done>> sink, int size) {
|
||||
return Flow.<T> create()
|
||||
.buffer(size, OverflowStrategy.dropHead())
|
||||
.toMat(sink, Keep.right());
|
||||
|
|
@ -51,9 +52,9 @@ public class RecipeDroppyBroadcast extends RecipeTest {
|
|||
nums.add(i + 1);
|
||||
}
|
||||
|
||||
final Sink<Integer, Future<Done>> mySink1 = Sink.ignore();
|
||||
final Sink<Integer, Future<Done>> mySink2 = Sink.ignore();
|
||||
final Sink<Integer, Future<Done>> mySink3 = Sink.ignore();
|
||||
final Sink<Integer, CompletionStage<Done>> mySink1 = Sink.ignore();
|
||||
final Sink<Integer, CompletionStage<Done>> mySink2 = Sink.ignore();
|
||||
final Sink<Integer, CompletionStage<Done>> mySink3 = Sink.ignore();
|
||||
|
||||
final Source<Integer, NotUsed> myData = Source.from(nums);
|
||||
|
||||
|
|
|
|||
|
|
@ -13,8 +13,6 @@ import akka.testkit.JavaTestKit;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
|
@ -50,8 +48,7 @@ public class RecipeFlattenList extends RecipeTest {
|
|||
Source<Message, NotUsed> flattened = myData.mapConcat(i -> i);
|
||||
//#flattening-lists
|
||||
|
||||
List<Message> got = Await.result(flattened.grouped(10).runWith(Sink.head(), mat),
|
||||
new FiniteDuration(1, TimeUnit.SECONDS));
|
||||
List<Message> got = flattened.grouped(10).runWith(Sink.head(), mat).toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertEquals(got.get(0), new Message("1"));
|
||||
assertEquals(got.get(1), new Message("2"));
|
||||
assertEquals(got.get(2), new Message("3"));
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ import akka.NotUsed;
|
|||
import akka.actor.*;
|
||||
import akka.dispatch.Mapper;
|
||||
import akka.japi.pf.ReceiveBuilder;
|
||||
import akka.pattern.Patterns;
|
||||
import akka.pattern.PatternsCS;
|
||||
import akka.stream.*;
|
||||
import akka.stream.javadsl.*;
|
||||
import akka.stream.testkit.TestSubscriber;
|
||||
|
|
@ -25,6 +25,7 @@ import scala.runtime.BoxedUnit;
|
|||
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static junit.framework.TestCase.assertTrue;
|
||||
|
|
@ -150,14 +151,9 @@ public class RecipeGlobalRateLimit extends RecipeTest {
|
|||
|
||||
return f.mapAsync(parallelism, element -> {
|
||||
final Timeout triggerTimeout = new Timeout(maxAllowedWait);
|
||||
final Future<Object> limiterTriggerFuture =
|
||||
Patterns.ask(limiter, Limiter.WANT_TO_PASS, triggerTimeout);
|
||||
return limiterTriggerFuture.map(new Mapper<Object, T>() {
|
||||
@Override
|
||||
public T apply(Object parameter) {
|
||||
return element;
|
||||
}
|
||||
}, system.dispatcher());
|
||||
final CompletionStage<Object> limiterTriggerFuture =
|
||||
PatternsCS.ask(limiter, Limiter.WANT_TO_PASS, triggerTimeout);
|
||||
return limiterTriggerFuture.thenApplyAsync(response -> element, system.dispatcher());
|
||||
});
|
||||
}
|
||||
//#global-limiter-flow
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ import scala.concurrent.duration.FiniteDuration;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
|
@ -125,7 +126,7 @@ public class RecipeMultiGroupByTest extends RecipeTest {
|
|||
});
|
||||
//#multi-groupby
|
||||
|
||||
Future<List<String>> result = multiGroups
|
||||
CompletionStage<List<String>> result = multiGroups
|
||||
.grouped(10)
|
||||
.mergeSubstreams()
|
||||
.map(pair -> {
|
||||
|
|
@ -135,7 +136,7 @@ public class RecipeMultiGroupByTest extends RecipeTest {
|
|||
.grouped(10)
|
||||
.runWith(Sink.head(), mat);
|
||||
|
||||
List<String> got = Await.result(result, FiniteDuration.create(3, TimeUnit.SECONDS));
|
||||
List<String> got = result.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertTrue(got.contains("1[1: a, 1: b, all: c, all: d, 1: e]"));
|
||||
assertTrue(got.contains("2[all: c, all: d]"));
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,8 +15,6 @@ import akka.util.ByteString;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
|
@ -53,7 +51,7 @@ public class RecipeParseLines extends RecipeTest {
|
|||
.map(b -> b.utf8String());
|
||||
//#parse-lines
|
||||
|
||||
Await.result(lines.grouped(10).runWith(Sink.head(), mat), new FiniteDuration(1, TimeUnit.SECONDS));
|
||||
lines.grouped(10).runWith(Sink.head(), mat).toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -25,6 +25,8 @@ import java.util.Arrays;
|
|||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class RecipeReduceByKeyTest extends RecipeTest {
|
||||
|
|
@ -63,8 +65,8 @@ public class RecipeReduceByKeyTest extends RecipeTest {
|
|||
.mergeSubstreams();
|
||||
//#word-count
|
||||
|
||||
final Future<List<Pair<String, Integer>>> f = counts.grouped(10).runWith(Sink.head(), mat);
|
||||
final Set<Pair<String, Integer>> result = Await.result(f, getRemainingTime()).stream().collect(Collectors.toSet());
|
||||
final CompletionStage<List<Pair<String, Integer>>> f = counts.grouped(10).runWith(Sink.head(), mat);
|
||||
final Set<Pair<String, Integer>> result = f.toCompletableFuture().get(3, TimeUnit.SECONDS).stream().collect(Collectors.toSet());
|
||||
final Set<Pair<String, Integer>> expected = new HashSet<>();
|
||||
expected.add(new Pair<>("hello", 2));
|
||||
expected.add(new Pair<>("world", 1));
|
||||
|
|
@ -106,8 +108,8 @@ public class RecipeReduceByKeyTest extends RecipeTest {
|
|||
(left, right) -> left + right));
|
||||
|
||||
//#reduce-by-key-general2
|
||||
final Future<List<Pair<String, Integer>>> f = counts.grouped(10).runWith(Sink.head(), mat);
|
||||
final Set<Pair<String, Integer>> result = Await.result(f, getRemainingTime()).stream().collect(Collectors.toSet());
|
||||
final CompletionStage<List<Pair<String, Integer>>> f = counts.grouped(10).runWith(Sink.head(), mat);
|
||||
final Set<Pair<String, Integer>> result = f.toCompletableFuture().get(3, TimeUnit.SECONDS).stream().collect(Collectors.toSet());
|
||||
final Set<Pair<String, Integer>> expected = new HashSet<>();
|
||||
expected.add(new Pair<>("hello", 2));
|
||||
expected.add(new Pair<>("world", 1));
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ import scala.concurrent.duration.FiniteDuration;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class RecipeToStrict extends RecipeTest {
|
||||
|
|
@ -45,11 +46,11 @@ public class RecipeToStrict extends RecipeTest {
|
|||
final int MAX_ALLOWED_SIZE = 100;
|
||||
|
||||
//#draining-to-list
|
||||
final Future<List<String>> strings = myData
|
||||
final CompletionStage<List<String>> strings = myData
|
||||
.grouped(MAX_ALLOWED_SIZE).runWith(Sink.head(), mat);
|
||||
//#draining-to-list
|
||||
|
||||
Await.result(strings, new FiniteDuration(1, TimeUnit.SECONDS));
|
||||
strings.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@ import scala.concurrent.duration.FiniteDuration;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
|
@ -72,9 +73,8 @@ public class RecipeWorkerPool extends RecipeTest {
|
|||
Source<Message, NotUsed> processedJobs = data.via(balancer);
|
||||
//#worker-pool2
|
||||
|
||||
FiniteDuration timeout = FiniteDuration.create(200, TimeUnit.MILLISECONDS);
|
||||
Future<List<String>> future = processedJobs.map(m -> m.msg).grouped(10).runWith(Sink.head(), mat);
|
||||
List<String> got = Await.result(future, timeout);
|
||||
CompletionStage<List<String>> future = processedJobs.map(m -> m.msg).grouped(10).runWith(Sink.head(), mat);
|
||||
List<String> got = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertTrue(got.contains("1 done"));
|
||||
assertTrue(got.contains("2 done"));
|
||||
assertTrue(got.contains("3 done"));
|
||||
|
|
|
|||
|
|
@ -137,11 +137,11 @@ re-materialized the respective pool is automatically and transparently restarted
|
|||
|
||||
In addition to the automatic shutdown via the configured idle timeouts it's also possible to trigger the immediate
|
||||
shutdown of a specific pool by calling ``shutdown()`` on the :class:`HostConnectionPool` instance that the pool client
|
||||
flow materializes into. This ``shutdown()`` call produces a ``Future[Unit]`` which is fulfilled when the pool
|
||||
flow materializes into. This ``shutdown()`` call produces a ``CompletionStage<Done>`` which is fulfilled when the pool
|
||||
termination has been completed.
|
||||
|
||||
It's also possible to trigger the immediate termination of *all* connection pools in the ``ActorSystem`` at the same
|
||||
time by calling ``Http.get(system).shutdownAllConnectionPools()``. This call too produces a ``Future[Unit]`` which is fulfilled when
|
||||
time by calling ``Http.get(system).shutdownAllConnectionPools()``. This call too produces a ``CompletionStage<Done>`` which is fulfilled when
|
||||
all pools have terminated.
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -38,7 +38,7 @@ Sometimes your HTTP client needs are very basic. You simply need the HTTP respon
|
|||
want to bother with setting up a full-blown streaming infrastructure.
|
||||
|
||||
For these cases Akka HTTP offers the ``Http().singleRequest(...)`` method, which simply turns an ``HttpRequest`` instance
|
||||
into ``Future<HttpResponse>``. Internally the request is dispatched across the (cached) host connection pool for the
|
||||
into ``CompletionStage<HttpResponse>``. Internally the request is dispatched across the (cached) host connection pool for the
|
||||
request's effective URI.
|
||||
|
||||
Just like in the case of the super-pool flow described above the request must have either an absolute URI or a valid
|
||||
|
|
@ -48,7 +48,7 @@ Just like in the case of the super-pool flow described above the request must ha
|
|||
|
||||
Using the Future-Based API in Actors
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
When using the ``Future`` based API from inside an ``Actor``, all the usual caveats apply to how one should deal
|
||||
When using the ``CompletionStage`` based API from inside an ``Actor``, all the usual caveats apply to how one should deal
|
||||
with the futures completion. For example you should not access the Actors state from within the Future's callbacks
|
||||
(such as ``map``, ``onComplete``, ...) and instead you should use the ``pipe`` pattern to pipe the result back
|
||||
to the Actor as a message:
|
||||
|
|
|
|||
|
|
@ -129,7 +129,7 @@ Entity types ``HttpEntityStrict``, ``HttpEntityDefault``, and ``HttpEntityChunke
|
|||
which allows to use them for requests and responses. In contrast, ``HttpEntityCloseDelimited`` can only be used for responses.
|
||||
|
||||
Streaming entity types (i.e. all but ``HttpEntityStrict``) cannot be shared or serialized. To create a strict, sharable copy of an
|
||||
entity or message use ``HttpEntity.toStrict`` or ``HttpMessage.toStrict`` which returns a ``Future`` of the object with
|
||||
entity or message use ``HttpEntity.toStrict`` or ``HttpMessage.toStrict`` which returns a ``CompletionStage`` of the object with
|
||||
the body data collected into a ``ByteString``.
|
||||
|
||||
The class ``HttpEntities`` contains static methods to create entities from common types easily.
|
||||
|
|
|
|||
|
|
@ -104,8 +104,8 @@ Deferring Result Creation
|
|||
Sometimes a handler cannot directly complete the request but needs to do some processing asynchronously. In this case
|
||||
the completion of a request needs to be deferred until the result has been generated. This is supported by the routing
|
||||
DSL in two ways: either you can use one of the ``handleWithAsyncN`` methods passing an ``AsyncHandlerN`` which
|
||||
returns a ``Future<RouteResult>``, i.e. an eventual ``RouteResult``, or you can also use a regular handler as shown
|
||||
above and use ``RequestContext.completeWith`` for completion which takes an ``Future<RouteResult>`` as an argument.
|
||||
returns a ``CompletionStage<RouteResult>``, i.e. an eventual ``RouteResult``, or you can also use a regular handler as shown
|
||||
above and use ``RequestContext.completeWith`` for completion which takes an ``CompletionStage<RouteResult>`` as an argument.
|
||||
|
||||
This is demonstrated in the following example. Consider a asynchronous service defined like this
|
||||
(making use of Java 8 lambdas):
|
||||
|
|
@ -117,16 +117,17 @@ Here the calculator runs the actual calculation in the background and only event
|
|||
service should provide a front-end to that service without having to block while waiting for the results. As explained
|
||||
above this can be done in two ways.
|
||||
|
||||
First, you can use ``handleWithAsyncN`` to be able to return a ``Future<RouteResult>``:
|
||||
First, you can use ``handleWithAsyncN`` to be able to return a ``CompletionStage<RouteResult>``:
|
||||
|
||||
.. includecode:: /../../akka-http-tests/src/test/java/docs/http/javadsl/server/HandlerExampleDocTest.java
|
||||
:include: async-handler-1
|
||||
|
||||
The handler invokes the service and then maps the calculation result to a ``RouteResult`` using ``Future.map`` and
|
||||
returns the resulting ``Future<RouteResult>``.
|
||||
The handler invokes the service and then maps the calculation result to a ``RouteResult`` using ``CompletionStage.thenApplyAsync`` and
|
||||
returns the resulting ``CompletionStage<RouteResult>``. Note that you should always explicitly provide an executor that designates
|
||||
where the future transformation task is executed, using the JDK’s global ForkJoinPool is not recommended.
|
||||
|
||||
Otherwise, you can also still use ``handleWithN`` and use ``RequestContext.completeWith`` to "convert" a
|
||||
``Future<RouteResult>`` into a ``RouteResult`` as shown here:
|
||||
``CompletionStage<RouteResult>`` into a ``RouteResult`` as shown here:
|
||||
|
||||
.. includecode:: /../../akka-http-tests/src/test/java/docs/http/javadsl/server/HandlerExampleDocTest.java
|
||||
:include: async-handler-2
|
||||
|
|
|
|||
|
|
@ -88,7 +88,7 @@ Bind failures
|
|||
^^^^^^^^^^^^^
|
||||
For example the server might be unable to bind to the given port. For example when the port
|
||||
is already taken by another application, or if the port is privileged (i.e. only usable by ``root``).
|
||||
In this case the "binding future" will fail immediatly, and we can react to if by listening on the Future's completion:
|
||||
In this case the "binding future" will fail immediatly, and we can react to if by listening on the CompletionStage's completion:
|
||||
|
||||
.. includecode:: ../../code/docs/http/javadsl/server/HighLevelServerBindFailureExample.java
|
||||
:include: binding-failure-high-level-example
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ Http basic auth allows for protection of one or more routes with a username and
|
|||
To use it you subclass ``HttpBasicAuthenticator`` and provide your authentication logic.
|
||||
There are two factory methods to create the authentication results to return from the authentication logic:
|
||||
``authenticateAs(T)`` and ``refuseAccess()``. If the authentication is not very quick in memory, for example
|
||||
calls a database, make sure you do not block the web server thread by executing that in a separate ``Future``
|
||||
calls a database, make sure you do not block the web server thread by executing that in a separate ``CompletionStage``
|
||||
and then ``flatMap`` the result into the authentication result.
|
||||
|
||||
When you use the authenticator in your routes you must reference the concrete authenticator twice,
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ the request can either be refused by returning the return value of ``refuseAcces
|
|||
with an object that is application specific by returning the return value of ``authenticateAs(T)``.
|
||||
|
||||
If the authentication is not very quick in memory, for example calls a separate authentication server
|
||||
to verify the token, make sure you do not block the web server thread by executing that in a separate ``Future``
|
||||
to verify the token, make sure you do not block the web server thread by executing that in a separate ``CompletionStage``
|
||||
and then ``flatMap`` the result into the authentication result.
|
||||
|
||||
.. note:: OAuth2 Bearer Token sends the token as clear text and should ONLY EVER be used over
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ RouteResult
|
|||
|
||||
The ``RouteResult`` is an opaque structure that represents possible results of evaluating a route. A ``RouteResult``
|
||||
can only be created by using one of the methods of the ``RequestContext``. A result can either be a response, if
|
||||
it was generated by one of the ``completeX`` methods, it can be an eventual result, i.e. a ``Future<RouteResult`` if
|
||||
it was generated by one of the ``completeX`` methods, it can be an eventual result, i.e. a ``CompletionStage<RouteResult`` if
|
||||
``completeWith`` was used or a rejection that contains information about why the route could not handle the request.
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -95,7 +95,7 @@ Requests are handled by calling one of the ``handleWithXXX`` methods with a hand
|
|||
|
||||
- a ``Flow<HttpRequest, HttpResponse, ?>`` for ``handleWith``,
|
||||
- a function ``Function<HttpRequest, HttpResponse>`` for ``handleWithSyncHandler``,
|
||||
- a function ``Function<HttpRequest, Future<HttpResponse>>`` for ``handleWithAsyncHandler``.
|
||||
- a function ``Function<HttpRequest, CompletionStage<HttpResponse>>`` for ``handleWithAsyncHandler``.
|
||||
|
||||
Here is a complete example:
|
||||
|
||||
|
|
@ -192,7 +192,7 @@ Bind failures
|
|||
|
||||
The first type of failure is when the server is unable to bind to the given port. For example when the port
|
||||
is already taken by another application, or if the port is privileged (i.e. only usable by ``root``).
|
||||
In this case the "binding future" will fail immediatly, and we can react to if by listening on the Future's completion:
|
||||
In this case the "binding future" will fail immediatly, and we can react to if by listening on the CompletionStage’s completion:
|
||||
|
||||
.. includecode:: ../../code/docs/http/javadsl/server/HttpServerExampleDocTest.java
|
||||
:include: binding-failure-handling
|
||||
|
|
|
|||
|
|
@ -7,6 +7,25 @@ Migration Guide 2.0.x to 2.4.x
|
|||
General notes
|
||||
=============
|
||||
|
||||
Java DSL now uses Java 8 types: CompletionStage and Optional
|
||||
------------------------------------------------------------
|
||||
|
||||
In order to provide a top-notch Java API we switched from Scala’s Future and Akka’s
|
||||
``akka.japi.Option`` interim solutions to the JDK’s own types for deferred computation
|
||||
and optional results. This has been done throughout Streams & HTTP, most notably changing most
|
||||
materialized types, but also the signature of the ``mapAsync`` combinator and the
|
||||
asynchronous route result combinators in the HTTP DSL.
|
||||
|
||||
The ``akka.pattern`` package has been updated with a new set of implementations within
|
||||
the ``PatternCS`` class that provide the ability to interact between Actors and Futures
|
||||
(or streams) for ``CompletionStage``.
|
||||
|
||||
Should you have the need to use Scala Futures with these new Java APIs please use
|
||||
the ``scala-java8-compat`` library that comes as a dependency of Akka. For more
|
||||
information see `the documentation``_.
|
||||
|
||||
.. _`the documentation`:: https://github.com/scala/scala-java8-compat
|
||||
|
||||
akka.Done and akka.NotUsed replacing Unit and BoxedUnit
|
||||
-------------------------------------------------------
|
||||
|
||||
|
|
|
|||
|
|
@ -253,7 +253,7 @@ type is of the nested module (indicated by the color *red* on the diagram):
|
|||
.. includecode:: ../code/docs/stream/CompositionDocTest.java#mat-combine-1
|
||||
|
||||
Next, we create a composite :class:`Flow` from two smaller components. Here, the second enclosed :class:`Flow` has a
|
||||
materialized type of :class:`Future<OutgoingConnection>`, and we propagate this to the parent by using ``Keep.right()``
|
||||
materialized type of :class:`CompletionStage<OutgoingConnection>`, and we propagate this to the parent by using ``Keep.right()``
|
||||
as the combiner function (indicated by the color *yellow* on the diagram):
|
||||
|
||||
.. includecode:: ../code/docs/stream/CompositionDocTest.java#mat-combine-2
|
||||
|
|
@ -267,7 +267,7 @@ we use ``Keep.both()`` to get a :class:`Pair` of them as the materialized type o
|
|||
|
||||
As the last example, we wire together ``nestedSource`` and ``nestedSink`` and we use a custom combiner function to
|
||||
create a yet another materialized type of the resulting :class:`RunnableGraph`. This combiner function just ignores
|
||||
the :class:`Future<Sink>` part, and wraps the other two values in a custom case class :class:`MyClass`
|
||||
the :class:`CompletionStage<Sink>` part, and wraps the other two values in a custom case class :class:`MyClass`
|
||||
(indicated by color *purple* on the diagram):
|
||||
|
||||
.. includecode:: ../code/docs/stream/CompositionDocTest.java#mat-combine-4a
|
||||
|
|
|
|||
|
|
@ -60,7 +60,7 @@ In this recipe we will use the ``grouped`` stream operation that groups incoming
|
|||
size collections (it can be seen as the almost opposite version of the "Flattening a stream of sequences" recipe
|
||||
we showed before). By using a ``grouped(MAX_ALLOWED_SIZE)`` we create a stream of groups
|
||||
with maximum size of ``MaxAllowedSeqSize`` and then we take the first element of this stream by attaching a ``Sink.head()``. What we get is a
|
||||
:class:`Future` containing a sequence with all the elements of the original up to ``MAX_ALLOWED_SIZE`` size (further
|
||||
:class:`CompletionStage` containing a sequence with all the elements of the original up to ``MAX_ALLOWED_SIZE`` size (further
|
||||
elements are dropped).
|
||||
|
||||
.. includecode:: ../code/docs/stream/javadsl/cookbook/RecipeToStrict.java#draining-to-list
|
||||
|
|
|
|||
|
|
@ -67,7 +67,7 @@ Assume that we can lookup their email address using:
|
|||
|
||||
.. includecode:: ../code/docs/stream/IntegrationDocTest.java#email-address-lookup2
|
||||
|
||||
The ``Future`` is completed with ``Failure`` if the email is not found.
|
||||
The ``CompletionStage`` is completed normally if the email is not found.
|
||||
|
||||
Transforming the stream of authors to a stream of email addresses by using the ``lookupEmail``
|
||||
service can be done with ``mapAsync`` and we use ``Supervision.getResumingDecider`` to drop
|
||||
|
|
@ -76,4 +76,4 @@ unknown email addresses:
|
|||
.. includecode:: ../code/docs/stream/IntegrationDocTest.java#email-addresses-mapAsync-supervision
|
||||
|
||||
If we would not use ``Resume`` the default stopping strategy would complete the stream
|
||||
with failure on the first ``Future`` that was completed with ``Failure``.
|
||||
with failure on the first ``CompletionStage`` that was completed exceptionally.
|
||||
|
|
|
|||
|
|
@ -80,7 +80,7 @@ one actor prepare the work, and then have it be materialized at some completely
|
|||
|
||||
After running (materializing) the ``RunnableGraph`` we get a special container object, the ``MaterializedMap``. Both
|
||||
sources and sinks are able to put specific objects into this map. Whether they put something in or not is implementation
|
||||
dependent. For example a ``FoldSink`` will make a ``Future`` available in this map which will represent the result
|
||||
dependent. For example a ``FoldSink`` will make a ``CompletionStage`` available in this map which will represent the result
|
||||
of the folding process over the stream. In general, a stream can expose multiple materialized values,
|
||||
but it is quite common to be interested in only the value of the Source or the Sink in the stream. For this reason
|
||||
there is a convenience method called ``runWith()`` available for ``Sink``, ``Source`` or ``Flow`` requiring, respectively,
|
||||
|
|
@ -105,7 +105,7 @@ of the given sink or source.
|
|||
|
||||
Since a stream can be materialized multiple times, the ``MaterializedMap`` returned is different for each materialization.
|
||||
In the example below we create two running materialized instance of the stream that we described in the ``runnable``
|
||||
variable, and both materializations give us a different ``Future`` from the map even though we used the same ``sink``
|
||||
variable, and both materializations give us a different ``CompletionStage`` from the map even though we used the same ``sink``
|
||||
to refer to the future:
|
||||
|
||||
.. includecode:: ../code/docs/stream/FlowDocTest.java#stream-reuse
|
||||
|
|
|
|||
|
|
@ -222,7 +222,7 @@ times to acquire the necessary number of outlets.
|
|||
.. includecode:: ../code/docs/stream/FlowGraphDocTest.java#flow-graph-matvalue
|
||||
|
||||
Be careful not to introduce a cycle where the materialized value actually contributes to the materialized value.
|
||||
The following example demonstrates a case where the materialized ``Future`` of a fold is fed back to the fold itself.
|
||||
The following example demonstrates a case where the materialized ``CompletionStage`` of a fold is fed back to the fold itself.
|
||||
|
||||
.. includecode:: ../code/docs/stream/FlowGraphDocTest.java#flow-graph-matvalue-cycle
|
||||
|
||||
|
|
|
|||
|
|
@ -169,7 +169,7 @@ Finally, sending the emails:
|
|||
.. includecode:: ../code/docs/stream/IntegrationDocTest.java#send-emails
|
||||
|
||||
``mapAsync`` is applying the given function that is calling out to the external service to
|
||||
each of the elements as they pass through this processing step. The function returns a :class:`Future`
|
||||
each of the elements as they pass through this processing step. The function returns a :class:`CompletionStage`
|
||||
and the value of that future will be emitted downstreams. The number of Futures
|
||||
that shall run in parallel is given as the first argument to ``mapAsync``.
|
||||
These Futures may complete in any order, but the elements that are emitted
|
||||
|
|
@ -190,8 +190,8 @@ is not important and then we can use the more efficient ``mapAsyncUnordered``:
|
|||
|
||||
.. includecode:: ../code/docs/stream/IntegrationDocTest.java#external-service-mapAsyncUnordered
|
||||
|
||||
In the above example the services conveniently returned a :class:`Future` of the result.
|
||||
If that is not the case you need to wrap the call in a :class:`Future`. If the service call
|
||||
In the above example the services conveniently returned a :class:`CompletionStage` of the result.
|
||||
If that is not the case you need to wrap the call in a :class:`CompletionStage`. If the service call
|
||||
involves blocking you must also make sure that you run it on a dedicated execution context, to
|
||||
avoid starvation and disturbance of other tasks in the system.
|
||||
|
||||
|
|
@ -215,7 +215,7 @@ external service, you can use ``ask``:
|
|||
.. includecode:: ../code/docs/stream/IntegrationDocTest.java#save-tweets
|
||||
|
||||
Note that if the ``ask`` is not completed within the given timeout the stream is completed with failure.
|
||||
If that is not desired outcome you can use ``recover`` on the ``ask`` :class:`Future`.
|
||||
If that is not desired outcome you can use ``recover`` on the ``ask`` :class:`CompletionStage`.
|
||||
|
||||
Illustrating ordering and parallelism
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ Streaming TCP
|
|||
|
||||
Accepting connections: Echo Server
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
In order to implement a simple EchoServer we ``bind`` to a given address, which returns a ``Source<IncomingConnection, Future<ServerBinding>>``,
|
||||
In order to implement a simple EchoServer we ``bind`` to a given address, which returns a ``Source<IncomingConnection, CompletionStage<ServerBinding>>``,
|
||||
which will emit an :class:`IncomingConnection` element for each new connection that the Server should handle:
|
||||
|
||||
.. includecode:: ../code/docs/stream/io/StreamTcpDocTest.java#echo-server-simple-bind
|
||||
|
|
|
|||
|
|
@ -163,21 +163,21 @@ First, let's write such an element counter using ``Flow.of(Class)`` and ``Sink.f
|
|||
|
||||
First we prepare a reusable ``Flow`` that will change each incoming tweet into an integer of value ``1``. We'll use this in
|
||||
order to combine those with a ``Sink.fold`` that will sum all ``Integer`` elements of the stream and make its result available as
|
||||
a ``Future<Integer>``. Next we connect the ``tweets`` stream to ``count`` with ``via``. Finally we connect the Flow to the previously
|
||||
a ``CompletionStage<Integer>``. Next we connect the ``tweets`` stream to ``count`` with ``via``. Finally we connect the Flow to the previously
|
||||
prepared Sink using ``toMat``.
|
||||
|
||||
Remember those mysterious ``Mat`` type parameters on ``Source<Out, Mat>``, ``Flow<In, Out, Mat>`` and ``Sink<In, Mat>``?
|
||||
They represent the type of values these processing parts return when materialized. When you chain these together,
|
||||
you can explicitly combine their materialized values: in our example we used the ``Keep.right`` predefined function,
|
||||
which tells the implementation to only care about the materialized type of the stage currently appended to the right.
|
||||
The materialized type of ``sumSink`` is ``Future<Integer>`` and because of using ``Keep.right``, the resulting :class:`RunnableGraph`
|
||||
has also a type parameter of ``Future<Integer>``.
|
||||
The materialized type of ``sumSink`` is ``CompletionStage<Integer>`` and because of using ``Keep.right``, the resulting :class:`RunnableGraph`
|
||||
has also a type parameter of ``CompletionStage<Integer>``.
|
||||
|
||||
This step does *not* yet materialize the
|
||||
processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can
|
||||
be ``run()``, as indicated by its type: ``RunnableGraph<Future<Integer>>``. Next we call ``run()`` which uses the :class:`ActorMaterializer`
|
||||
be ``run()``, as indicated by its type: ``RunnableGraph<CompletionStage<Integer>>``. Next we call ``run()`` which uses the :class:`ActorMaterializer`
|
||||
to materialize and run the Flow. The value returned by calling ``run()`` on a ``RunnableGraph<T>`` is of type ``T``.
|
||||
In our case this type is ``Future<Integer>`` which, when completed, will contain the total length of our tweets stream.
|
||||
In our case this type is ``CompletionStage<Integer>`` which, when completed, will contain the total length of our tweets stream.
|
||||
In case of the stream failing, this future would complete with a Failure.
|
||||
|
||||
A :class:`RunnableGraph` may be reused
|
||||
|
|
|
|||
|
|
@ -48,7 +48,7 @@ used for writing stream tests that use familiar :class:`TestProbe` from the
|
|||
:mod:`akka-testkit` API.
|
||||
|
||||
One of the more straightforward tests would be to materialize stream to a
|
||||
:class:`Future` and then use ``pipe`` pattern to pipe the result of that future
|
||||
:class:`CompletionStage` and then use ``PatternsCS.pipe`` pattern to pipe the result of that future
|
||||
to the probe.
|
||||
|
||||
.. includecode:: ../code/docs/stream/StreamTestKitDocTest.java#pipeto-testprobe
|
||||
|
|
|
|||
|
|
@ -105,9 +105,9 @@ class BasicDirectivesExamplesSpec extends RoutingSpec {
|
|||
|
||||
def sample() =
|
||||
path("sample") {
|
||||
extractExecutionContext { implicit ec =>
|
||||
extractExecutionContext { implicit executor =>
|
||||
complete {
|
||||
Future(s"Run on ${ec.##}!") // uses the `ec` ExecutionContext
|
||||
Future(s"Run on ${executor.##}!") // uses the `executor` ExecutionContext
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -132,9 +132,9 @@ class BasicDirectivesExamplesSpec extends RoutingSpec {
|
|||
//#extractExecutionContext-0
|
||||
def sample() =
|
||||
path("sample") {
|
||||
extractExecutionContext { implicit ec =>
|
||||
extractExecutionContext { implicit executor =>
|
||||
complete {
|
||||
Future(s"Run on ${ec.##}!") // uses the `ec` ExecutionContext
|
||||
Future(s"Run on ${executor.##}!") // uses the `executor` ExecutionContext
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,9 +9,9 @@ import akka.http.scaladsl.model.HttpEntity$;
|
|||
import akka.stream.Materializer;
|
||||
import akka.stream.javadsl.Source;
|
||||
import akka.util.ByteString;
|
||||
import scala.concurrent.Future;
|
||||
|
||||
import java.util.OptionalLong;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
/**
|
||||
* Represents the entity of an Http message. An entity consists of the content-type of the data
|
||||
|
|
@ -133,7 +133,7 @@ public interface HttpEntity {
|
|||
* Use getDataBytes and stream processing instead if the expected data is big or
|
||||
* is likely to take a long time.
|
||||
*/
|
||||
Future<HttpEntity.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<HttpEntity.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
/**
|
||||
* The entity type which consists of a predefined fixed ByteString of data.
|
||||
|
|
|
|||
|
|
@ -6,8 +6,8 @@ package akka.http.javadsl.model;
|
|||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import scala.concurrent.Future;
|
||||
import akka.http.javadsl.model.headers.ContentDisposition;
|
||||
import akka.http.javadsl.model.headers.ContentDispositionType;
|
||||
import akka.http.javadsl.model.headers.RangeUnit;
|
||||
|
|
@ -31,9 +31,9 @@ public interface Multipart {
|
|||
/**
|
||||
* Converts this content into its strict counterpart.
|
||||
* The given `timeout` denotes the max time that an individual part must be read in.
|
||||
* The Future is failed with an TimeoutException if one part isn't read completely after the given timeout.
|
||||
* The CompletionStage is failed with an TimeoutException if one part isn't read completely after the given timeout.
|
||||
*/
|
||||
Future<? extends Multipart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<? extends Multipart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
/**
|
||||
* Creates an entity from this multipart object.
|
||||
|
|
@ -59,7 +59,7 @@ public interface Multipart {
|
|||
|
||||
Optional<ContentDispositionType> getDispositionType();
|
||||
|
||||
Future<? extends Multipart.BodyPart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<? extends Multipart.BodyPart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
interface Strict extends Multipart.BodyPart {
|
||||
HttpEntity.Strict getEntity();
|
||||
|
|
@ -72,7 +72,7 @@ public interface Multipart {
|
|||
interface General extends Multipart {
|
||||
Source<? extends Multipart.General.BodyPart, Object> getParts();
|
||||
|
||||
Future<Multipart.General.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<Multipart.General.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
interface Strict extends Multipart.General, Multipart.Strict {
|
||||
Source<Multipart.General.BodyPart.Strict, Object> getParts();
|
||||
|
|
@ -81,7 +81,7 @@ public interface Multipart {
|
|||
}
|
||||
|
||||
interface BodyPart extends Multipart.BodyPart {
|
||||
Future<Multipart.General.BodyPart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<Multipart.General.BodyPart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
interface Strict extends Multipart.General.BodyPart, Multipart.BodyPart.Strict {
|
||||
}
|
||||
|
|
@ -95,7 +95,7 @@ public interface Multipart {
|
|||
interface FormData extends Multipart {
|
||||
Source<? extends Multipart.FormData.BodyPart, Object> getParts();
|
||||
|
||||
Future<Multipart.FormData.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<Multipart.FormData.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
interface Strict extends Multipart.FormData, Multipart.Strict {
|
||||
Source<Multipart.FormData.BodyPart.Strict, Object> getParts();
|
||||
|
|
@ -109,7 +109,7 @@ public interface Multipart {
|
|||
Iterable<HttpHeader> getAdditionalHeaders();
|
||||
Optional<String> getFilename();
|
||||
|
||||
Future<Multipart.FormData.BodyPart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<Multipart.FormData.BodyPart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
interface Strict extends Multipart.FormData.BodyPart, Multipart.BodyPart.Strict {
|
||||
}
|
||||
|
|
@ -123,7 +123,7 @@ public interface Multipart {
|
|||
interface ByteRanges extends Multipart {
|
||||
Source<? extends Multipart.ByteRanges.BodyPart, Object> getParts();
|
||||
|
||||
Future<Multipart.ByteRanges.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<Multipart.ByteRanges.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
interface Strict extends Multipart.ByteRanges, Multipart.Strict {
|
||||
Source<Multipart.ByteRanges.BodyPart.Strict, Object> getParts();
|
||||
|
|
@ -137,7 +137,7 @@ public interface Multipart {
|
|||
Iterable<HttpHeader> getAdditionalHeaders();
|
||||
akka.http.javadsl.model.headers.ContentRange getContentRangeHeader();
|
||||
|
||||
Future<Multipart.ByteRanges.BodyPart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
CompletionStage<Multipart.ByteRanges.BodyPart.Strict> toStrict(long timeoutMillis, Materializer materializer);
|
||||
|
||||
interface Strict extends Multipart.ByteRanges.BodyPart, Multipart.BodyPart.Strict {
|
||||
}
|
||||
|
|
|
|||
|
|
@ -274,7 +274,7 @@ private[http] object HttpServerBluePrint {
|
|||
}
|
||||
}
|
||||
private def schedule(delay: FiniteDuration, handler: HttpRequest ⇒ HttpResponse): Cancellable =
|
||||
materializer.scheduleOnce(delay, new Runnable { def run() = trigger.invoke(self, handler(request)) })
|
||||
materializer.scheduleOnce(delay, new Runnable { def run() = trigger.invoke((self, handler(request))) })
|
||||
|
||||
import akka.http.impl.util.JavaMapping.Implicits._
|
||||
/** JAVA API **/
|
||||
|
|
|
|||
|
|
@ -11,7 +11,6 @@ import akka.http.impl.util.JavaMapping.HttpsConnectionContext
|
|||
import akka.http.javadsl.model.ws._
|
||||
import akka.{ stream, NotUsed }
|
||||
import akka.stream.io.{ SslTlsInbound, SslTlsOutbound }
|
||||
|
||||
import scala.language.implicitConversions
|
||||
import scala.concurrent.Future
|
||||
import scala.util.Try
|
||||
|
|
@ -21,13 +20,13 @@ import akka.actor.{ ExtendedActorSystem, ActorSystem, ExtensionIdProvider, Exten
|
|||
import akka.event.LoggingAdapter
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.javadsl.{ BidiFlow, Flow, Source }
|
||||
|
||||
import akka.http.impl.util.JavaMapping.Implicits._
|
||||
import akka.http.scaladsl.{ model ⇒ sm }
|
||||
import akka.http.javadsl.model._
|
||||
import akka.http._
|
||||
|
||||
import scala.compat.java8.OptionConverters._
|
||||
import scala.compat.java8.FutureConverters._
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
object Http extends ExtensionId[Http] with ExtensionIdProvider {
|
||||
override def get(system: ActorSystem): Http = super.get(system)
|
||||
|
|
@ -38,6 +37,10 @@ object Http extends ExtensionId[Http] with ExtensionIdProvider {
|
|||
class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
||||
import akka.dispatch.ExecutionContexts.{ sameThreadExecutionContext ⇒ ec }
|
||||
|
||||
import language.implicitConversions
|
||||
private implicit def completionStageCovariant[T, U >: T](in: CompletionStage[T]): CompletionStage[U] = in.asInstanceOf[CompletionStage[U]]
|
||||
private implicit def javaModelIsScalaModel[J <: AnyRef, S <: J](in: Future[J])(implicit ev: JavaMapping.Inherited[J, S]): Future[S] = in.asInstanceOf[Future[S]]
|
||||
|
||||
private lazy val delegate = akka.http.scaladsl.Http(system)
|
||||
|
||||
/**
|
||||
|
|
@ -86,10 +89,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* fail, unless the first materialization has already been unbound. Unbinding can be triggered via the materialized
|
||||
* [[ServerBinding]].
|
||||
*/
|
||||
def bind(interface: String, port: Int, materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
|
||||
def bind(interface: String, port: Int, materializer: Materializer): Source[IncomingConnection, CompletionStage[ServerBinding]] =
|
||||
new Source(delegate.bind(interface, port)(materializer)
|
||||
.map(new IncomingConnection(_))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec).toJava))
|
||||
|
||||
/**
|
||||
* Creates a [[Source]] of [[IncomingConnection]] instances which represents a prospective HTTP server binding
|
||||
|
|
@ -106,10 +109,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
def bind(interface: String, port: Int,
|
||||
connectionContext: ConnectionContext,
|
||||
settings: ServerSettings,
|
||||
materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
|
||||
materializer: Materializer): Source[IncomingConnection, CompletionStage[ServerBinding]] =
|
||||
new Source(delegate.bind(interface, port, settings = settings, connectionContext = ConnectionContext.noEncryption().asScala)(materializer)
|
||||
.map(new IncomingConnection(_))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec).toJava))
|
||||
|
||||
/**
|
||||
* Creates a [[Source]] of [[IncomingConnection]] instances which represents a prospective HTTP server binding
|
||||
|
|
@ -125,10 +128,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
*/
|
||||
def bind(interface: String, port: Int,
|
||||
connectionContext: ConnectionContext,
|
||||
materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
|
||||
materializer: Materializer): Source[IncomingConnection, CompletionStage[ServerBinding]] =
|
||||
new Source(delegate.bind(interface, port, connectionContext = connectionContext.asScala)(materializer)
|
||||
.map(new IncomingConnection(_))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec).toJava))
|
||||
|
||||
/**
|
||||
* Creates a [[Source]] of [[IncomingConnection]] instances which represents a prospective HTTP server binding
|
||||
|
|
@ -146,10 +149,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
connectionContext: ConnectionContext,
|
||||
settings: ServerSettings,
|
||||
log: LoggingAdapter,
|
||||
materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
|
||||
materializer: Materializer): Source[IncomingConnection, CompletionStage[ServerBinding]] =
|
||||
new Source(delegate.bind(interface, port, ConnectionContext.noEncryption().asScala, settings, log)(materializer)
|
||||
.map(new IncomingConnection(_))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec).toJava))
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -160,10 +163,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
*/
|
||||
def bindAndHandle(handler: Flow[HttpRequest, HttpResponse, _],
|
||||
interface: String, port: Int,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandle(handler.asInstanceOf[Flow[sm.HttpRequest, sm.HttpResponse, _]].asScala,
|
||||
interface, port)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -175,10 +178,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
def bindAndHandle(handler: Flow[HttpRequest, HttpResponse, _],
|
||||
interface: String, port: Int,
|
||||
connectionContext: ConnectionContext,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandle(handler.asInstanceOf[Flow[sm.HttpRequest, sm.HttpResponse, _]].asScala,
|
||||
interface, port, connectionContext.asScala)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -192,10 +195,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
settings: ServerSettings,
|
||||
connectionContext: ConnectionContext,
|
||||
log: LoggingAdapter,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandle(handler.asInstanceOf[Flow[sm.HttpRequest, sm.HttpResponse, _]].asScala,
|
||||
interface, port, connectionContext.asScala, settings, log)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -206,9 +209,9 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
*/
|
||||
def bindAndHandleSync(handler: Function[HttpRequest, HttpResponse],
|
||||
interface: String, port: Int,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandleSync(handler.apply(_).asScala, interface, port)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -220,9 +223,9 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
def bindAndHandleSync(handler: Function[HttpRequest, HttpResponse],
|
||||
interface: String, port: Int,
|
||||
connectionContext: ConnectionContext,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandleSync(handler.apply(_).asScala, interface, port, connectionContext.asScala)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -236,10 +239,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
settings: ServerSettings,
|
||||
connectionContext: ConnectionContext,
|
||||
log: LoggingAdapter,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandleSync(handler.apply(_).asScala,
|
||||
interface, port, connectionContext.asScala, settings, log)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -248,11 +251,11 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* The number of concurrently accepted connections can be configured by overriding
|
||||
* the `akka.http.server.max-connections` setting.
|
||||
*/
|
||||
def bindAndHandleAsync(handler: Function[HttpRequest, Future[HttpResponse]],
|
||||
def bindAndHandleAsync(handler: Function[HttpRequest, CompletionStage[HttpResponse]],
|
||||
interface: String, port: Int,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
delegate.bindAndHandleAsync(handler.apply(_).asInstanceOf[Future[sm.HttpResponse]], interface, port)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandleAsync(handler.apply(_).toScala, interface, port)(materializer)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -261,12 +264,12 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* The number of concurrently accepted connections can be configured by overriding
|
||||
* the `akka.http.server.max-connections` setting.
|
||||
*/
|
||||
def bindAndHandleAsync(handler: Function[HttpRequest, Future[HttpResponse]],
|
||||
def bindAndHandleAsync(handler: Function[HttpRequest, CompletionStage[HttpResponse]],
|
||||
interface: String, port: Int,
|
||||
connectionContext: ConnectionContext,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
delegate.bindAndHandleAsync(handler.apply(_).asInstanceOf[Future[sm.HttpResponse]], interface, port, connectionContext.asScala)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandleAsync(handler.apply(_).toScala, interface, port, connectionContext.asScala)(materializer)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Convenience method which starts a new HTTP server at the given endpoint and uses the given `handler`
|
||||
|
|
@ -275,14 +278,14 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* The number of concurrently accepted connections can be configured by overriding
|
||||
* the `akka.http.server.max-connections` setting.
|
||||
*/
|
||||
def bindAndHandleAsync(handler: Function[HttpRequest, Future[HttpResponse]],
|
||||
def bindAndHandleAsync(handler: Function[HttpRequest, CompletionStage[HttpResponse]],
|
||||
interface: String, port: Int,
|
||||
settings: ServerSettings, connectionContext: ConnectionContext,
|
||||
parallelism: Int, log: LoggingAdapter,
|
||||
materializer: Materializer): Future[ServerBinding] =
|
||||
delegate.bindAndHandleAsync(handler.apply(_).asInstanceOf[Future[sm.HttpResponse]],
|
||||
materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
delegate.bindAndHandleAsync(handler.apply(_).toScala,
|
||||
interface, port, connectionContext.asScala, settings, parallelism, log)(materializer)
|
||||
.map(new ServerBinding(_))(ec)
|
||||
.map(new ServerBinding(_))(ec).toJava
|
||||
|
||||
/**
|
||||
* Constructs a client layer stage using the configured default [[ClientConnectionSettings]].
|
||||
|
|
@ -311,7 +314,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
*
|
||||
* If the hostname is given with an `https://` prefix, the default [[HttpsConnectionContext]] will be used.
|
||||
*/
|
||||
def outgoingConnection(host: String): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
def outgoingConnection(host: String): Flow[HttpRequest, HttpResponse, CompletionStage[OutgoingConnection]] =
|
||||
outgoingConnection(ConnectHttp.toHost(host))
|
||||
|
||||
/**
|
||||
|
|
@ -320,7 +323,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
*
|
||||
* Use the [[ConnectHttp]] DSL to configure target host and whether HTTPS should be used.
|
||||
*/
|
||||
def outgoingConnection(to: ConnectHttp): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
def outgoingConnection(to: ConnectHttp): Flow[HttpRequest, HttpResponse, CompletionStage[OutgoingConnection]] =
|
||||
adaptOutgoingFlow {
|
||||
if (to.isHttps) delegate.outgoingConnectionHttps(to.host, to.port, to.effectiveConnectionContext(defaultClientHttpsContext).asScala)
|
||||
else delegate.outgoingConnection(to.host, to.port)
|
||||
|
|
@ -334,7 +337,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
connectionContext: ConnectionContext,
|
||||
localAddress: Optional[InetSocketAddress],
|
||||
settings: ClientConnectionSettings,
|
||||
log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
log: LoggingAdapter): Flow[HttpRequest, HttpResponse, CompletionStage[OutgoingConnection]] =
|
||||
adaptOutgoingFlow {
|
||||
connectionContext match {
|
||||
case https: HttpsConnectionContext ⇒ delegate.outgoingConnectionHttps(host, port, https.asScala, localAddress.asScala, settings, log)
|
||||
|
|
@ -504,8 +507,8 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* Note that the request must have either an absolute URI or a valid `Host` header, otherwise
|
||||
* the future will be completed with an error.
|
||||
*/
|
||||
def singleRequest(request: HttpRequest, materializer: Materializer): Future[HttpResponse] =
|
||||
delegate.singleRequest(request.asScala)(materializer)
|
||||
def singleRequest(request: HttpRequest, materializer: Materializer): CompletionStage[HttpResponse] =
|
||||
delegate.singleRequest(request.asScala)(materializer).toJava
|
||||
|
||||
/**
|
||||
* Fires a single [[HttpRequest]] across the (cached) host connection pool for the request's
|
||||
|
|
@ -516,8 +519,8 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* Note that the request must have either an absolute URI or a valid `Host` header, otherwise
|
||||
* the future will be completed with an error.
|
||||
*/
|
||||
def singleRequest(request: HttpRequest, connectionContext: HttpsConnectionContext, materializer: Materializer): Future[HttpResponse] =
|
||||
delegate.singleRequest(request.asScala, connectionContext.asScala)(materializer)
|
||||
def singleRequest(request: HttpRequest, connectionContext: HttpsConnectionContext, materializer: Materializer): CompletionStage[HttpResponse] =
|
||||
delegate.singleRequest(request.asScala, connectionContext.asScala)(materializer).toJava
|
||||
|
||||
/**
|
||||
* Fires a single [[HttpRequest]] across the (cached) host connection pool for the request's
|
||||
|
|
@ -531,15 +534,15 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
def singleRequest(request: HttpRequest,
|
||||
connectionContext: HttpsConnectionContext,
|
||||
settings: ConnectionPoolSettings,
|
||||
log: LoggingAdapter, materializer: Materializer): Future[HttpResponse] =
|
||||
delegate.singleRequest(request.asScala, connectionContext.asScala, settings, log)(materializer)
|
||||
log: LoggingAdapter, materializer: Materializer): CompletionStage[HttpResponse] =
|
||||
delegate.singleRequest(request.asScala, connectionContext.asScala, settings, log)(materializer).toJava
|
||||
|
||||
/**
|
||||
* Constructs a WebSocket [[BidiFlow]].
|
||||
*
|
||||
* The layer is not reusable and must only be materialized once.
|
||||
*/
|
||||
def webSocketClientLayer(request: WebSocketRequest): BidiFlow[Message, SslTlsOutbound, SslTlsInbound, Message, Future[WebSocketUpgradeResponse]] =
|
||||
def webSocketClientLayer(request: WebSocketRequest): BidiFlow[Message, SslTlsOutbound, SslTlsInbound, Message, CompletionStage[WebSocketUpgradeResponse]] =
|
||||
adaptWsBidiFlow(delegate.webSocketClientLayer(request.asScala))
|
||||
|
||||
/**
|
||||
|
|
@ -549,7 +552,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* The layer is not reusable and must only be materialized once.
|
||||
*/
|
||||
def webSocketClientLayer(request: WebSocketRequest,
|
||||
settings: ClientConnectionSettings): BidiFlow[Message, SslTlsOutbound, SslTlsInbound, Message, Future[WebSocketUpgradeResponse]] =
|
||||
settings: ClientConnectionSettings): BidiFlow[Message, SslTlsOutbound, SslTlsInbound, Message, CompletionStage[WebSocketUpgradeResponse]] =
|
||||
adaptWsBidiFlow(delegate.webSocketClientLayer(request.asScala, settings))
|
||||
|
||||
/**
|
||||
|
|
@ -560,7 +563,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
*/
|
||||
def webSocketClientLayer(request: WebSocketRequest,
|
||||
settings: ClientConnectionSettings,
|
||||
log: LoggingAdapter): BidiFlow[Message, SslTlsOutbound, SslTlsInbound, Message, Future[WebSocketUpgradeResponse]] =
|
||||
log: LoggingAdapter): BidiFlow[Message, SslTlsOutbound, SslTlsInbound, Message, CompletionStage[WebSocketUpgradeResponse]] =
|
||||
adaptWsBidiFlow(delegate.webSocketClientLayer(request.asScala, settings, log))
|
||||
|
||||
/**
|
||||
|
|
@ -568,7 +571,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
*
|
||||
* The layer is not reusable and must only be materialized once.
|
||||
*/
|
||||
def webSocketClientFlow(request: WebSocketRequest): Flow[Message, Message, Future[WebSocketUpgradeResponse]] =
|
||||
def webSocketClientFlow(request: WebSocketRequest): Flow[Message, Message, CompletionStage[WebSocketUpgradeResponse]] =
|
||||
adaptWsFlow {
|
||||
delegate.webSocketClientFlow(request.asScala)
|
||||
}
|
||||
|
|
@ -582,7 +585,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
connectionContext: ConnectionContext,
|
||||
localAddress: Optional[InetSocketAddress],
|
||||
settings: ClientConnectionSettings,
|
||||
log: LoggingAdapter): Flow[Message, Message, Future[WebSocketUpgradeResponse]] =
|
||||
log: LoggingAdapter): Flow[Message, Message, CompletionStage[WebSocketUpgradeResponse]] =
|
||||
adaptWsFlow {
|
||||
delegate.webSocketClientFlow(request.asScala, connectionContext.asScala, localAddress.asScala, settings, log)
|
||||
}
|
||||
|
|
@ -595,7 +598,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
*/
|
||||
def singleWebSocketRequest[T](request: WebSocketRequest,
|
||||
clientFlow: Flow[Message, Message, T],
|
||||
materializer: Materializer): Pair[Future[WebSocketUpgradeResponse], T] =
|
||||
materializer: Materializer): Pair[CompletionStage[WebSocketUpgradeResponse], T] =
|
||||
adaptWsResultTuple {
|
||||
delegate.singleWebSocketRequest(
|
||||
request.asScala,
|
||||
|
|
@ -611,7 +614,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
def singleWebSocketRequest[T](request: WebSocketRequest,
|
||||
clientFlow: Flow[Message, Message, T],
|
||||
connectionContext: ConnectionContext,
|
||||
materializer: Materializer): Pair[Future[WebSocketUpgradeResponse], T] =
|
||||
materializer: Materializer): Pair[CompletionStage[WebSocketUpgradeResponse], T] =
|
||||
adaptWsResultTuple {
|
||||
delegate.singleWebSocketRequest(
|
||||
request.asScala,
|
||||
|
|
@ -629,7 +632,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
localAddress: Optional[InetSocketAddress],
|
||||
settings: ClientConnectionSettings,
|
||||
log: LoggingAdapter,
|
||||
materializer: Materializer): Pair[Future[WebSocketUpgradeResponse], T] =
|
||||
materializer: Materializer): Pair[CompletionStage[WebSocketUpgradeResponse], T] =
|
||||
adaptWsResultTuple {
|
||||
delegate.singleWebSocketRequest(
|
||||
request.asScala,
|
||||
|
|
@ -648,7 +651,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* If existing pool client flows are re-used or new ones materialized concurrently with or after this
|
||||
* method call the respective connection pools will be restarted and not contribute to the returned future.
|
||||
*/
|
||||
def shutdownAllConnectionPools(): Future[Unit] = delegate.shutdownAllConnectionPools()
|
||||
def shutdownAllConnectionPools(): CompletionStage[Unit] = delegate.shutdownAllConnectionPools().toJava
|
||||
|
||||
/**
|
||||
* Gets the default
|
||||
|
|
@ -673,11 +676,11 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
JavaMapping.toJava(scalaFlow)(JavaMapping.flowMapping[Pair[HttpRequest, T], (scaladsl.model.HttpRequest, T), Pair[Try[HttpResponse], T], (Try[scaladsl.model.HttpResponse], T), Mat])
|
||||
}
|
||||
|
||||
private def adaptOutgoingFlow[T, Mat](scalaFlow: stream.scaladsl.Flow[scaladsl.model.HttpRequest, scaladsl.model.HttpResponse, Future[scaladsl.Http.OutgoingConnection]]): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
private def adaptOutgoingFlow[T, Mat](scalaFlow: stream.scaladsl.Flow[scaladsl.model.HttpRequest, scaladsl.model.HttpResponse, Future[scaladsl.Http.OutgoingConnection]]): Flow[HttpRequest, HttpResponse, CompletionStage[OutgoingConnection]] =
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(scalaFlow)(Keep.right)
|
||||
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
|
||||
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec).toJava)
|
||||
}
|
||||
|
||||
private def adaptServerLayer(serverLayer: scaladsl.Http.ServerLayer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, NotUsed] =
|
||||
|
|
@ -690,12 +693,12 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
JavaMapping.adapterBidiFlow[HttpRequest, sm.HttpRequest, sm.HttpResponse, HttpResponse]
|
||||
.atop(clientLayer))
|
||||
|
||||
private def adaptWsBidiFlow(wsLayer: scaladsl.Http.WebSocketClientLayer): BidiFlow[Message, SslTlsOutbound, SslTlsInbound, Message, Future[WebSocketUpgradeResponse]] =
|
||||
private def adaptWsBidiFlow(wsLayer: scaladsl.Http.WebSocketClientLayer): BidiFlow[Message, SslTlsOutbound, SslTlsInbound, Message, CompletionStage[WebSocketUpgradeResponse]] =
|
||||
new BidiFlow(
|
||||
JavaMapping.adapterBidiFlow[Message, sm.ws.Message, sm.ws.Message, Message]
|
||||
.atopMat(wsLayer)((_, s) ⇒ adaptWsUpgradeResponse(s)))
|
||||
|
||||
private def adaptWsFlow(wsLayer: stream.scaladsl.Flow[sm.ws.Message, sm.ws.Message, Future[scaladsl.model.ws.WebSocketUpgradeResponse]]): Flow[Message, Message, Future[WebSocketUpgradeResponse]] =
|
||||
private def adaptWsFlow(wsLayer: stream.scaladsl.Flow[sm.ws.Message, sm.ws.Message, Future[scaladsl.model.ws.WebSocketUpgradeResponse]]): Flow[Message, Message, CompletionStage[WebSocketUpgradeResponse]] =
|
||||
Flow.fromGraph(JavaMapping.adapterBidiFlow[Message, sm.ws.Message, sm.ws.Message, Message].joinMat(wsLayer)(Keep.right).mapMaterializedValue(adaptWsUpgradeResponse _))
|
||||
|
||||
private def adaptWsFlow[Mat](javaFlow: Flow[Message, Message, Mat]): stream.scaladsl.Flow[scaladsl.model.ws.Message, scaladsl.model.ws.Message, Mat] =
|
||||
|
|
@ -704,10 +707,10 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
.viaMat(javaFlow.asScala)(Keep.right)
|
||||
.map(_.asScala)
|
||||
|
||||
private def adaptWsResultTuple[T](result: (Future[scaladsl.model.ws.WebSocketUpgradeResponse], T)): Pair[Future[WebSocketUpgradeResponse], T] =
|
||||
private def adaptWsResultTuple[T](result: (Future[scaladsl.model.ws.WebSocketUpgradeResponse], T)): Pair[CompletionStage[WebSocketUpgradeResponse], T] =
|
||||
result match {
|
||||
case (fut, tMat) ⇒ Pair(adaptWsUpgradeResponse(fut), tMat)
|
||||
}
|
||||
private def adaptWsUpgradeResponse(responseFuture: Future[scaladsl.model.ws.WebSocketUpgradeResponse]): Future[WebSocketUpgradeResponse] =
|
||||
responseFuture.map(WebSocketUpgradeResponse.adapt)(system.dispatcher)
|
||||
private def adaptWsUpgradeResponse(responseFuture: Future[scaladsl.model.ws.WebSocketUpgradeResponse]): CompletionStage[WebSocketUpgradeResponse] =
|
||||
responseFuture.map(WebSocketUpgradeResponse.adapt)(system.dispatcher).toJava
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,12 +6,14 @@ package akka.http.javadsl
|
|||
|
||||
import java.net.InetSocketAddress
|
||||
import akka.NotUsed
|
||||
import akka.japi.function.Function;
|
||||
import scala.concurrent.Future
|
||||
import akka.japi.function.Function
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.javadsl.Flow
|
||||
import akka.http.javadsl.model._
|
||||
import akka.http.scaladsl.{ model ⇒ sm }
|
||||
import java.util.concurrent.CompletionStage
|
||||
import scala.concurrent.Future
|
||||
import scala.compat.java8.FutureConverters._
|
||||
|
||||
/**
|
||||
* Represents one accepted incoming HTTP connection.
|
||||
|
|
@ -50,12 +52,12 @@ class IncomingConnection private[http] (delegate: akka.http.scaladsl.Http.Incomi
|
|||
/**
|
||||
* Handles the connection with the given handler function.
|
||||
*/
|
||||
def handleWithAsyncHandler(handler: Function[HttpRequest, Future[HttpResponse]], materializer: Materializer): Unit =
|
||||
delegate.handleWithAsyncHandler(handler.apply(_).asInstanceOf[Future[sm.HttpResponse]])(materializer)
|
||||
def handleWithAsyncHandler(handler: Function[HttpRequest, CompletionStage[HttpResponse]], materializer: Materializer): Unit =
|
||||
delegate.handleWithAsyncHandler(handler.apply(_).toScala.asInstanceOf[Future[sm.HttpResponse]])(materializer)
|
||||
|
||||
/**
|
||||
* Handles the connection with the given handler function.
|
||||
*/
|
||||
def handleWithAsyncHandler(handler: Function[HttpRequest, Future[HttpResponse]], parallelism: Int, materializer: Materializer): Unit =
|
||||
delegate.handleWithAsyncHandler(handler.apply(_).asInstanceOf[Future[sm.HttpResponse]], parallelism)(materializer)
|
||||
def handleWithAsyncHandler(handler: Function[HttpRequest, CompletionStage[HttpResponse]], parallelism: Int, materializer: Materializer): Unit =
|
||||
delegate.handleWithAsyncHandler(handler.apply(_).toScala.asInstanceOf[Future[sm.HttpResponse]], parallelism)(materializer)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,7 +5,8 @@
|
|||
package akka.http.javadsl
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import scala.concurrent.Future
|
||||
import java.util.concurrent.CompletionStage
|
||||
import scala.compat.java8.FutureConverters._
|
||||
|
||||
/**
|
||||
* Represents a prospective HTTP server binding.
|
||||
|
|
@ -20,7 +21,7 @@ class ServerBinding private[http] (delegate: akka.http.scaladsl.Http.ServerBindi
|
|||
* Asynchronously triggers the unbinding of the port that was bound by the materialization of the `connections`
|
||||
* [[Source]]
|
||||
*
|
||||
* The produced [[Future]] is fulfilled when the unbinding has been completed.
|
||||
* The produced [[java.util.concurrent.CompletionStage]] is fulfilled when the unbinding has been completed.
|
||||
*/
|
||||
def unbind(): Future[Unit] = delegate.unbind()
|
||||
def unbind(): CompletionStage[Unit] = delegate.unbind().toJava
|
||||
}
|
||||
|
|
|
|||
|
|
@ -25,11 +25,16 @@ import akka.http.impl.util.StreamUtils
|
|||
import akka.http.impl.util.JavaMapping.Implicits._
|
||||
|
||||
import scala.compat.java8.OptionConverters._
|
||||
import scala.compat.java8.FutureConverters._
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
/**
|
||||
* Models the entity (aka "body" or "content) of an HTTP message.
|
||||
*/
|
||||
sealed trait HttpEntity extends jm.HttpEntity {
|
||||
import language.implicitConversions
|
||||
private implicit def completionStageCovariant[T, U >: T](in: CompletionStage[T]): CompletionStage[U] = in.asInstanceOf[CompletionStage[U]]
|
||||
|
||||
/**
|
||||
* Determines whether this entity is known to be empty.
|
||||
*/
|
||||
|
|
@ -96,8 +101,8 @@ sealed trait HttpEntity extends jm.HttpEntity {
|
|||
override def isChunked: Boolean = false
|
||||
|
||||
/** Java API */
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): Future[jm.HttpEntity.Strict] =
|
||||
toStrict(timeoutMillis.millis)(materializer)
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[jm.HttpEntity.Strict] =
|
||||
toStrict(timeoutMillis.millis)(materializer).toJava
|
||||
}
|
||||
|
||||
/* An entity that can be used for body parts */
|
||||
|
|
|
|||
|
|
@ -7,7 +7,6 @@ package akka.http.scaladsl.model
|
|||
import java.io.File
|
||||
import java.util.Optional
|
||||
import akka.http.impl.util.Util
|
||||
|
||||
import scala.collection.immutable.VectorBuilder
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.Future
|
||||
|
|
@ -24,8 +23,9 @@ import akka.http.scaladsl.model.headers._
|
|||
import akka.http.impl.engine.rendering.BodyPartRenderer
|
||||
import akka.http.javadsl.{ model ⇒ jm }
|
||||
import FastFuture._
|
||||
|
||||
import scala.compat.java8.OptionConverters._
|
||||
import scala.compat.java8.FutureConverters._
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
/**
|
||||
* The model of multipart content for media-types `multipart/\*` (general multipart content),
|
||||
|
|
@ -74,8 +74,8 @@ sealed trait Multipart extends jm.Multipart {
|
|||
JSource.fromGraph(parts.asInstanceOf[Source[Multipart.BodyPart, AnyRef]])
|
||||
|
||||
/** Java API */
|
||||
def toStrict(timeoutMillis: Long, materializer: Materializer): Future[_ <: jm.Multipart.Strict] =
|
||||
toStrict(FiniteDuration(timeoutMillis, concurrent.duration.MILLISECONDS))(materializer)
|
||||
def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[_ <: jm.Multipart.Strict] =
|
||||
toStrict(FiniteDuration(timeoutMillis, concurrent.duration.MILLISECONDS))(materializer).toJava
|
||||
|
||||
/** Java API */
|
||||
def toEntity(charset: jm.HttpCharset, boundary: String): jm.RequestEntity =
|
||||
|
|
@ -170,8 +170,8 @@ object Multipart {
|
|||
def getDispositionType: Optional[jm.headers.ContentDispositionType] = Util.convertOption(dispositionType)
|
||||
|
||||
/** Java API */
|
||||
def toStrict(timeoutMillis: Long, materializer: Materializer): Future[_ <: jm.Multipart.BodyPart.Strict] =
|
||||
toStrict(FiniteDuration(timeoutMillis, concurrent.duration.MILLISECONDS))(materializer)
|
||||
def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[_ <: jm.Multipart.BodyPart.Strict] =
|
||||
toStrict(FiniteDuration(timeoutMillis, concurrent.duration.MILLISECONDS))(materializer).toJava
|
||||
}
|
||||
|
||||
object BodyPart {
|
||||
|
|
@ -211,8 +211,8 @@ object Multipart {
|
|||
super.getParts.asInstanceOf[JSource[_ <: jm.Multipart.General.BodyPart, AnyRef]]
|
||||
|
||||
/** Java API */
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): Future[jm.Multipart.General.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.General.Strict]]
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[jm.Multipart.General.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.General.Strict]].toJava
|
||||
}
|
||||
object General {
|
||||
def apply(mediaType: MediaType.Multipart, parts: BodyPart.Strict*): Strict = Strict(mediaType, parts.toVector)
|
||||
|
|
@ -257,8 +257,8 @@ object Multipart {
|
|||
def toByteRangesBodyPart: Try[Multipart.ByteRanges.BodyPart]
|
||||
|
||||
/** Java API */
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): Future[jm.Multipart.General.BodyPart.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.General.BodyPart.Strict]]
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[jm.Multipart.General.BodyPart.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.General.BodyPart.Strict]].toJava
|
||||
|
||||
private[BodyPart] def tryCreateFormDataBodyPart[T](f: (String, Map[String, String], immutable.Seq[HttpHeader]) ⇒ T): Try[T] = {
|
||||
val params = dispositionParams
|
||||
|
|
@ -322,8 +322,8 @@ object Multipart {
|
|||
super.getParts.asInstanceOf[JSource[_ <: jm.Multipart.FormData.BodyPart, AnyRef]]
|
||||
|
||||
/** Java API */
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): Future[jm.Multipart.FormData.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.FormData.Strict]]
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[jm.Multipart.FormData.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.FormData.Strict]].toJava
|
||||
}
|
||||
object FormData {
|
||||
def apply(parts: Multipart.FormData.BodyPart.Strict*): Multipart.FormData.Strict = Strict(parts.toVector)
|
||||
|
|
@ -415,8 +415,8 @@ object Multipart {
|
|||
def getFilename: Optional[String] = filename.asJava
|
||||
|
||||
/** Java API */
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): Future[jm.Multipart.FormData.BodyPart.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.FormData.BodyPart.Strict]]
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[jm.Multipart.FormData.BodyPart.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.FormData.BodyPart.Strict]].toJava
|
||||
}
|
||||
object BodyPart {
|
||||
def apply(_name: String, _entity: BodyPartEntity,
|
||||
|
|
@ -470,8 +470,8 @@ object Multipart {
|
|||
super.getParts.asInstanceOf[JSource[_ <: jm.Multipart.ByteRanges.BodyPart, AnyRef]]
|
||||
|
||||
/** Java API */
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): Future[jm.Multipart.ByteRanges.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.ByteRanges.Strict]]
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[jm.Multipart.ByteRanges.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.ByteRanges.Strict]].toJava
|
||||
}
|
||||
object ByteRanges {
|
||||
def apply(parts: Multipart.ByteRanges.BodyPart.Strict*): Strict = Strict(parts.toVector)
|
||||
|
|
@ -545,8 +545,8 @@ object Multipart {
|
|||
def getContentRangeHeader: jm.headers.ContentRange = contentRangeHeader
|
||||
|
||||
/** Java API */
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): Future[jm.Multipart.ByteRanges.BodyPart.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.ByteRanges.BodyPart.Strict]]
|
||||
override def toStrict(timeoutMillis: Long, materializer: Materializer): CompletionStage[jm.Multipart.ByteRanges.BodyPart.Strict] =
|
||||
super.toStrict(timeoutMillis, materializer).asInstanceOf[Future[jm.Multipart.ByteRanges.BodyPart.Strict]].toJava
|
||||
}
|
||||
object BodyPart {
|
||||
def apply(_contentRange: ContentRange, _entity: BodyPartEntity, _rangeUnit: RangeUnit = RangeUnits.Bytes,
|
||||
|
|
|
|||
|
|
@ -17,12 +17,13 @@ import akka.stream.javadsl.Flow;
|
|||
import akka.stream.javadsl.Keep;
|
||||
import akka.stream.javadsl.Sink;
|
||||
import akka.stream.javadsl.Source;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Future;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class WSEchoTestClientApp {
|
||||
private static final Function<Message, String> messageStringifier = new Function<Message, String>() {
|
||||
|
|
@ -57,23 +58,23 @@ public class WSEchoTestClientApp {
|
|||
TextMessage.create("ghi")
|
||||
)).concat(Source.fromFuture(delayedCompletion).drop(1));
|
||||
|
||||
Sink<Message, Future<List<String>>> echoSink =
|
||||
Sink<Message, CompletionStage<List<String>>> echoSink =
|
||||
Flow.of(Message.class)
|
||||
.map(messageStringifier)
|
||||
.grouped(1000)
|
||||
.toMat(Sink.<List<String>>head(), Keep.<NotUsed, Future<List<String>>>right());
|
||||
.toMat(Sink.<List<String>>head(), Keep.right());
|
||||
|
||||
Flow<Message, Message, Future<List<String>>> echoClient =
|
||||
Flow.fromSinkAndSourceMat(echoSink, echoSource, Keep.<Future<List<String>>, NotUsed>left());
|
||||
Flow<Message, Message, CompletionStage<List<String>>> echoClient =
|
||||
Flow.fromSinkAndSourceMat(echoSink, echoSource, Keep.left());
|
||||
|
||||
Future<List<String>> result =
|
||||
CompletionStage<List<String>> result =
|
||||
Http.get(system).singleWebSocketRequest(
|
||||
WebSocketRequest.create("ws://echo.websocket.org"),
|
||||
echoClient,
|
||||
materializer
|
||||
).second();
|
||||
|
||||
List<String> messages = Await.result(result, FiniteDuration.apply(10, "second"));
|
||||
List<String> messages = result.toCompletableFuture().get(10, TimeUnit.SECONDS);
|
||||
System.out.println("Collected " + messages.size() + " messages:");
|
||||
for (String msg: messages)
|
||||
System.out.println(msg);
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ import scala.concurrent.duration.FiniteDuration;
|
|||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class JavaTestServer {
|
||||
|
|
@ -32,7 +33,7 @@ public class JavaTestServer {
|
|||
try {
|
||||
final Materializer materializer = ActorMaterializer.create(system);
|
||||
|
||||
Future<ServerBinding> serverBindingFuture =
|
||||
CompletionStage<ServerBinding> serverBindingFuture =
|
||||
Http.get(system).bindAndHandleSync(
|
||||
new Function<HttpRequest, HttpResponse>() {
|
||||
public HttpResponse apply(HttpRequest request) throws Exception {
|
||||
|
|
@ -47,7 +48,7 @@ public class JavaTestServer {
|
|||
}
|
||||
}, "localhost", 8080, materializer);
|
||||
|
||||
Await.result(serverBindingFuture, new FiniteDuration(1, TimeUnit.SECONDS)); // will throw if binding fails
|
||||
serverBindingFuture.toCompletableFuture().get(1, TimeUnit.SECONDS); // will throw if binding fails
|
||||
System.out.println("Press ENTER to stop.");
|
||||
new BufferedReader(new InputStreamReader(System.in)).readLine();
|
||||
} finally {
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.http.javadsl.testkit
|
||||
|
||||
import scala.annotation.varargs
|
||||
import scala.concurrent.ExecutionContext
|
||||
import scala.concurrent.ExecutionContextExecutor
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.Materializer
|
||||
import akka.http.scaladsl.server
|
||||
|
|
@ -31,7 +31,7 @@ import akka.http.impl.util._
|
|||
abstract class RouteTest extends AllDirectives {
|
||||
implicit def system: ActorSystem
|
||||
implicit def materializer: Materializer
|
||||
implicit def executionContext: ExecutionContext = system.dispatcher
|
||||
implicit def executionContext: ExecutionContextExecutor = system.dispatcher
|
||||
|
||||
protected def awaitDuration: FiniteDuration = 500.millis
|
||||
|
||||
|
|
|
|||
|
|
@ -5,16 +5,16 @@
|
|||
package akka.http.javadsl.server.examples.simple;
|
||||
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.dispatch.Futures;
|
||||
import akka.http.javadsl.server.*;
|
||||
import akka.http.javadsl.server.values.Parameter;
|
||||
import akka.http.javadsl.server.values.Parameters;
|
||||
import akka.http.javadsl.server.values.PathMatcher;
|
||||
import akka.http.javadsl.server.values.PathMatchers;
|
||||
import scala.concurrent.Future;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class SimpleServerApp extends HttpApp {
|
||||
static Parameter<Integer> x = Parameters.intValue("x");
|
||||
|
|
@ -29,12 +29,8 @@ public class SimpleServerApp extends HttpApp {
|
|||
int result = x * y;
|
||||
return ctx.complete(String.format("%d * %d = %d", x, y, result));
|
||||
}
|
||||
public static Future<RouteResult> multiplyAsync(final RequestContext ctx, final int x, final int y) {
|
||||
return Futures.future(new Callable<RouteResult>() {
|
||||
public RouteResult call() throws Exception {
|
||||
return multiply(ctx, x, y);
|
||||
}
|
||||
}, ctx.executionContext());
|
||||
public static CompletionStage<RouteResult> multiplyAsync(final RequestContext ctx, final int x, final int y) {
|
||||
return CompletableFuture.supplyAsync(() -> multiply(ctx, x, y), ctx.executionContext());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -19,6 +19,8 @@ import javax.net.ssl.SSLContext;
|
|||
import static akka.http.javadsl.ConnectHttp.*;
|
||||
import static akka.http.javadsl.ConnectHttp.toHostHttps;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
@SuppressWarnings("ConstantConditions")
|
||||
public class HttpAPIsTest extends JUnitRouteTest {
|
||||
|
||||
|
|
@ -43,7 +45,7 @@ public class HttpAPIsTest extends JUnitRouteTest {
|
|||
http.bindAndHandle(handler, "127.0.0.1", 8080, materializer());
|
||||
http.bindAndHandle(handler, "127.0.0.1", 8080, httpsContext, materializer());
|
||||
|
||||
final Function<HttpRequest, Future<HttpResponse>> handler1 = null;
|
||||
final Function<HttpRequest, CompletionStage<HttpResponse>> handler1 = null;
|
||||
http.bindAndHandleAsync(handler1, "127.0.0.1", 8080, materializer());
|
||||
http.bindAndHandleAsync(handler1, "127.0.0.1", 8080, httpsContext, materializer());
|
||||
|
||||
|
|
|
|||
|
|
@ -6,8 +6,8 @@ package akka.http.javadsl.server;
|
|||
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.Callable;
|
||||
import akka.dispatch.Futures;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
import akka.http.javadsl.testkit.*;
|
||||
|
||||
import akka.http.javadsl.marshallers.jackson.Jackson;
|
||||
|
|
@ -52,13 +52,10 @@ public class CompleteTest extends JUnitRouteTest {
|
|||
Handler2<Integer, Integer> slowCalc = new Handler2<Integer, Integer>() {
|
||||
@Override
|
||||
public RouteResult apply(final RequestContext ctx, final Integer x, final Integer y) {
|
||||
return ctx.completeWith(Futures.future(new Callable<RouteResult>() {
|
||||
@Override
|
||||
public RouteResult call() throws Exception {
|
||||
return ctx.completeWith(CompletableFuture.supplyAsync(() -> {
|
||||
int result = x + y;
|
||||
return ctx.complete(String.format("%d + %d = %d",x, y, result));
|
||||
}
|
||||
}, executionContext()));
|
||||
}, ctx.executionContext()));
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
|||
|
|
@ -67,7 +67,7 @@ public class CodingDirectivesTest extends JUnitRouteTest {
|
|||
.assertHeaderExists(ContentEncoding.create(HttpEncodings.DEFLATE));
|
||||
|
||||
ByteString decompressed =
|
||||
Await.result(Coder.Deflate.decode(response.entityBytes(), mat), Duration.apply(3, TimeUnit.SECONDS));
|
||||
Coder.Deflate.decode(response.entityBytes(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
Assert.assertEquals("tester", decompressed.utf8String());
|
||||
}
|
||||
@Test
|
||||
|
|
|
|||
|
|
@ -52,32 +52,19 @@ public class RouteDirectivesTest extends JUnitRouteTest {
|
|||
.withoutSizeLimit()
|
||||
.getDataBytes()
|
||||
.runWith(Sink.<ByteString>head(), ctx.materializer())
|
||||
.map(new Mapper<ByteString, RouteResult>() {
|
||||
@Override
|
||||
public RouteResult apply(ByteString s) {
|
||||
return ctx.complete(s.utf8String());
|
||||
}
|
||||
}, ctx.executionContext()));
|
||||
.thenApplyAsync(s -> ctx.complete(s.utf8String()), ctx.executionContext()));
|
||||
}
|
||||
})),
|
||||
path("limit-5")
|
||||
.route(
|
||||
handleWith(new Function<RequestContext, RouteResult>() {
|
||||
@Override
|
||||
public RouteResult apply(final RequestContext ctx) throws Exception {
|
||||
handleWith(ctx -> {
|
||||
final RequestEntity entity = ctx.request().entity();
|
||||
return ctx.completeWith(
|
||||
entity
|
||||
.withSizeLimit(5)
|
||||
.getDataBytes()
|
||||
.runWith(Sink.<ByteString>head(), ctx.materializer())
|
||||
.map(new Mapper<ByteString, RouteResult>() {
|
||||
@Override
|
||||
public RouteResult apply(ByteString s) {
|
||||
return ctx.complete(s.utf8String());
|
||||
}
|
||||
}, ctx.executionContext()));
|
||||
}
|
||||
.thenApplyAsync(s -> ctx.complete(s.utf8String()), ctx.executionContext()));
|
||||
}))
|
||||
);
|
||||
|
||||
|
|
|
|||
|
|
@ -4,9 +4,10 @@
|
|||
|
||||
package akka.http.javadsl.server.values;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import org.junit.Test;
|
||||
import scala.Option;
|
||||
import scala.concurrent.Future;
|
||||
|
||||
import akka.http.javadsl.server.*;
|
||||
import akka.http.javadsl.model.HttpRequest;
|
||||
|
|
@ -17,7 +18,7 @@ public class HttpBasicAuthenticationTest extends JUnitRouteTest {
|
|||
HttpBasicAuthenticator<String> authenticatedUser =
|
||||
new HttpBasicAuthenticator<String>("test-realm") {
|
||||
@Override
|
||||
public Future<Option<String>> authenticate(BasicCredentials credentials) {
|
||||
public CompletionStage<Optional<String>> authenticate(BasicCredentials credentials) {
|
||||
if (credentials.available() && // no anonymous access
|
||||
credentials.identifier().equals("sina") &&
|
||||
credentials.verify("1234"))
|
||||
|
|
@ -29,7 +30,7 @@ public class HttpBasicAuthenticationTest extends JUnitRouteTest {
|
|||
OAuth2Authenticator<String> authenticatedToken =
|
||||
new OAuth2Authenticator<String>("test-realm") {
|
||||
@Override
|
||||
public Future<Option<String>> authenticate(OAuth2Credentials credentials) {
|
||||
public CompletionStage<Optional<String>> authenticate(OAuth2Credentials credentials) {
|
||||
if (credentials.available() && // no anonymous access
|
||||
credentials.identifier().equals("myToken") &&
|
||||
credentials.verify("myToken"))
|
||||
|
|
|
|||
|
|
@ -13,9 +13,11 @@ import akka.http.javadsl.server.values.Parameters;
|
|||
import akka.http.javadsl.server.values.PathMatchers;
|
||||
import akka.http.javadsl.testkit.JUnitRouteTest;
|
||||
import akka.http.javadsl.testkit.TestRoute;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.ExecutionContext;
|
||||
import scala.concurrent.Future;
|
||||
|
||||
public class HandlerExampleDocTest extends JUnitRouteTest {
|
||||
@Test
|
||||
|
|
@ -236,12 +238,12 @@ public class HandlerExampleDocTest extends JUnitRouteTest {
|
|||
//#async-example-full
|
||||
//#async-service-definition
|
||||
class CalculatorService {
|
||||
public Future<Integer> multiply(final int x, final int y, ExecutionContext ec) {
|
||||
return akka.dispatch.Futures.future(() -> x * y, ec);
|
||||
public CompletionStage<Integer> multiply(final int x, final int y) {
|
||||
return CompletableFuture.supplyAsync(() -> x * y);
|
||||
}
|
||||
|
||||
public Future<Integer> add(final int x, final int y, ExecutionContext ec) {
|
||||
return akka.dispatch.Futures.future(() -> x + y, ec);
|
||||
public CompletionStage<Integer> add(final int x, final int y) {
|
||||
return CompletableFuture.supplyAsync(() -> x + y);
|
||||
}
|
||||
}
|
||||
//#async-service-definition
|
||||
|
|
@ -253,15 +255,10 @@ public class HandlerExampleDocTest extends JUnitRouteTest {
|
|||
//#async-handler-1
|
||||
// would probably be injected or passed at construction time in real code
|
||||
CalculatorService calculatorService = new CalculatorService();
|
||||
public Future<RouteResult> multiplyAsync(final RequestContext ctx, int x, int y) {
|
||||
Future<Integer> result = calculatorService.multiply(x, y, ctx.executionContext());
|
||||
Mapper<Integer, RouteResult> func = new Mapper<Integer, RouteResult>() {
|
||||
@Override
|
||||
public RouteResult apply(Integer product) {
|
||||
return ctx.complete("x * y = " + product);
|
||||
}
|
||||
}; // cannot be written as lambda, unfortunately
|
||||
return result.map(func, ctx.executionContext());
|
||||
public CompletionStage<RouteResult> multiplyAsync(final RequestContext ctx, int x, int y) {
|
||||
CompletionStage<Integer> result = calculatorService.multiply(x, y);
|
||||
return result.thenApplyAsync(product -> ctx.complete("x * y = " + product),
|
||||
ctx.executionContext());
|
||||
}
|
||||
Route multiplyAsyncRoute =
|
||||
path("multiply").route(
|
||||
|
|
@ -271,14 +268,9 @@ public class HandlerExampleDocTest extends JUnitRouteTest {
|
|||
|
||||
//#async-handler-2
|
||||
public RouteResult addAsync(final RequestContext ctx, int x, int y) {
|
||||
Future<Integer> result = calculatorService.add(x, y, ctx.executionContext());
|
||||
Mapper<Integer, RouteResult> func = new Mapper<Integer, RouteResult>() {
|
||||
@Override
|
||||
public RouteResult apply(Integer sum) {
|
||||
return ctx.complete("x + y = " + sum);
|
||||
}
|
||||
}; // cannot be written as lambda, unfortunately
|
||||
return ctx.completeWith(result.map(func, ctx.executionContext()));
|
||||
CompletionStage<Integer> result = calculatorService.add(x, y);
|
||||
return ctx.completeWith(result.thenApplyAsync(sum -> ctx.complete("x + y = " + sum),
|
||||
ctx.executionContext()));
|
||||
}
|
||||
Route addAsyncRoute =
|
||||
path("add").route(
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.http.javadsl.server
|
||||
|
||||
import scala.concurrent.Future
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
[..21#/**
|
||||
* A route Handler that handles a request (that is encapsulated in a [[RequestContext]])
|
||||
|
|
@ -21,7 +21,7 @@ trait Handler1[[#T1#]] extends akka.japi.function.Function2[RequestContext, [#T1
|
|||
}
|
||||
/**
|
||||
* A route Handler that handles a request (that is encapsulated in a [[RequestContext]])
|
||||
* and returns a [[scala.concurrent.Future]] of [[RouteResult]] with the response (or the rejection).
|
||||
* and returns a [[java.util.concurrent.CompletionStage]] of [[RouteResult]] with the response (or the rejection).
|
||||
*
|
||||
* A route `Handler1` is a convenience class that extends Function of arity `N+1`,
|
||||
* since it needs to pass along the [[RequestContext]] as well, yet for readability
|
||||
|
|
@ -30,8 +30,8 @@ trait Handler1[[#T1#]] extends akka.japi.function.Function2[RequestContext, [#T1
|
|||
* Use the methods in [[RequestContext]] to create a [[RouteResult]].
|
||||
* A handler MUST NOT return `null` as the result.
|
||||
*/
|
||||
trait AsyncHandler1[[#T1#]] extends akka.japi.function.Function2[RequestContext, [#T1#], Future[RouteResult]] {
|
||||
override def apply(ctx: RequestContext, [#t1: T1#]): Future[RouteResult]
|
||||
trait AsyncHandler1[[#T1#]] extends akka.japi.function.Function2[RequestContext, [#T1#], CompletionStage[RouteResult]] {
|
||||
override def apply(ctx: RequestContext, [#t1: T1#]): CompletionStage[RouteResult]
|
||||
}#
|
||||
|
||||
]
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import akka.http.javadsl.server.RouteResult
|
|||
import akka.http.javadsl.server.RequestVal
|
||||
import akka.http.javadsl.server.RequestContext
|
||||
import scala.annotation.varargs
|
||||
import scala.concurrent.Future
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
abstract class BasicDirectivesBase {
|
||||
/** INTERNAL API */
|
||||
|
|
@ -24,12 +24,12 @@ abstract class BasicDirectivesBase {
|
|||
handle(extractions: _*)(ctx => handler(ctx))
|
||||
|
||||
/**
|
||||
* Handles the route using the given function, completing the route once the returned [[scala.concurrent.Future]] completes.
|
||||
* Handles the route using the given function, completing the route once the returned [[java.util.concurrent.CompletionStage]] completes.
|
||||
* The function MUST NOT return `null`.
|
||||
*
|
||||
* If the `handler` is accessing request values these must be passed to this method in order for extraction to be performed.
|
||||
*/
|
||||
@varargs def handleWithAsync(handler: akka.japi.function.Function[RequestContext, Future[RouteResult]], extractions: RequestVal[_]*): Route =
|
||||
@varargs def handleWithAsync(handler: akka.japi.function.Function[RequestContext, CompletionStage[RouteResult]], extractions: RequestVal[_]*): Route =
|
||||
handle(extractions: _*)(ctx => ctx.completeWith(handler(ctx)))
|
||||
|
||||
|
||||
|
|
@ -45,7 +45,7 @@ abstract class BasicDirectivesBase {
|
|||
handle([#v1#])(ctx => handler(ctx, [#v1.get(ctx)#]))
|
||||
|
||||
/**
|
||||
* Handles the route using the given function, completing the route once the returned [[scala.concurrent.Future]] completes.
|
||||
* Handles the route using the given function, completing the route once the returned [[java.util.concurrent.CompletionStage]] completes.
|
||||
* The function MUST NOT return `null`.
|
||||
*
|
||||
* For convenience, using Java 8 lambda expressions as the `handler` function is recommended.
|
||||
|
|
@ -53,7 +53,7 @@ abstract class BasicDirectivesBase {
|
|||
* [[akka.japi.function.Function2]] should prove to be useful, as it matches naming-wise with the number of
|
||||
* handled request values.
|
||||
*/
|
||||
def handleWithAsync1[[#T1#]]([#v1: RequestVal[T1]#], handler: akka.japi.function.Function2[RequestContext, [#T1#], Future[RouteResult]]): Route =
|
||||
def handleWithAsync1[[#T1#]]([#v1: RequestVal[T1]#], handler: akka.japi.function.Function2[RequestContext, [#T1#], CompletionStage[RouteResult]]): Route =
|
||||
handle([#v1#])(ctx => ctx.completeWith(handler(ctx, [#v1.get(ctx)#])))#
|
||||
|
||||
]
|
||||
|
|
|
|||
|
|
@ -4,12 +4,14 @@
|
|||
|
||||
package akka.http.javadsl.server;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import akka.http.scaladsl.coding.Deflate$;
|
||||
import akka.http.scaladsl.coding.Gzip$;
|
||||
import akka.http.scaladsl.coding.NoCoding$;
|
||||
import akka.stream.Materializer;
|
||||
import akka.util.ByteString;
|
||||
import scala.concurrent.Future;
|
||||
import scala.compat.java8.FutureConverters;
|
||||
|
||||
/**
|
||||
* A coder is an implementation of the predefined encoders/decoders defined for HTTP.
|
||||
|
|
@ -26,8 +28,8 @@ public enum Coder {
|
|||
public ByteString encode(ByteString input) {
|
||||
return underlying.encode(input);
|
||||
}
|
||||
public Future<ByteString> decode(ByteString input, Materializer mat) {
|
||||
return underlying.decode(input, mat);
|
||||
public CompletionStage<ByteString> decode(ByteString input, Materializer mat) {
|
||||
return FutureConverters.toJava(underlying.decode(input, mat));
|
||||
}
|
||||
public akka.http.scaladsl.coding.Coder _underlyingScalaCoder() {
|
||||
return underlying;
|
||||
|
|
|
|||
|
|
@ -7,12 +7,13 @@ package akka.http.impl.server
|
|||
import akka.http.javadsl.model.ContentType
|
||||
import akka.http.scaladsl.model.HttpEntity
|
||||
import akka.stream.Materializer
|
||||
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
import scala.concurrent.{ ExecutionContextExecutor, Future }
|
||||
import akka.http.javadsl.{ model ⇒ jm }
|
||||
import akka.http.impl.util.JavaMapping.Implicits._
|
||||
import akka.http.scaladsl.server.{ RequestContext ⇒ ScalaRequestContext }
|
||||
import akka.http.javadsl.server._
|
||||
import java.util.concurrent.CompletionStage
|
||||
import scala.compat.java8.FutureConverters._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -28,6 +29,7 @@ private[http] final case class RequestContextImpl(underlying: ScalaRequestContex
|
|||
futureResult.flatMap {
|
||||
case r: RouteResultImpl ⇒ r.underlying
|
||||
}(executionContext())
|
||||
def completeWith(futureResult: CompletionStage[RouteResult]): RouteResult = completeWith(futureResult.toScala)
|
||||
def complete(text: String): RouteResult = underlying.complete(text)
|
||||
def complete(contentType: ContentType.NonBinary, text: String): RouteResult =
|
||||
underlying.complete(HttpEntity(contentType.asScala, text))
|
||||
|
|
@ -48,6 +50,6 @@ private[http] final case class RequestContextImpl(underlying: ScalaRequestContex
|
|||
|
||||
def reject(customRejection: CustomRejection): RouteResult = underlying.reject(CustomRejectionWrapper(customRejection))
|
||||
|
||||
def executionContext(): ExecutionContext = underlying.executionContext
|
||||
def executionContext(): ExecutionContextExecutor = underlying.executionContext
|
||||
def materializer(): Materializer = underlying.materializer
|
||||
}
|
||||
|
|
|
|||
|
|
@ -23,6 +23,10 @@ import akka.http.scaladsl.server
|
|||
import akka.http.javadsl.server._
|
||||
import RouteStructure._
|
||||
|
||||
import scala.compat.java8.FutureConverters._
|
||||
import scala.compat.java8.OptionConverters._
|
||||
import akka.dispatch.ExecutionContexts.sameThreadExecutionContext
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
|
|
@ -94,7 +98,7 @@ private[http] object RouteImplementation extends Directives with server.RouteCon
|
|||
}
|
||||
}
|
||||
|
||||
authenticator.authenticate(javaCreds)
|
||||
authenticator.authenticate(javaCreds).toScala.map(_.asScala)(sameThreadExecutionContext)
|
||||
}).flatMap { user ⇒
|
||||
addExtraction(authenticator.asInstanceOf[RequestVal[Any]], user)
|
||||
}
|
||||
|
|
@ -117,7 +121,7 @@ private[http] object RouteImplementation extends Directives with server.RouteCon
|
|||
}
|
||||
}
|
||||
|
||||
authenticator.authenticate(javaCreds)
|
||||
authenticator.authenticate(javaCreds).toScala.map(_.asScala)(sameThreadExecutionContext)
|
||||
}).flatMap { user ⇒
|
||||
addExtraction(authenticator.asInstanceOf[RequestVal[Any]], user)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.http.javadsl.server
|
||||
|
||||
import scala.concurrent.Future
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
/**
|
||||
* A route Handler that handles a request (that is encapsulated in a [[RequestContext]])
|
||||
|
|
@ -22,12 +22,11 @@ trait Handler extends akka.japi.function.Function[RequestContext, RouteResult] {
|
|||
|
||||
/**
|
||||
* A route Handler that handles a request (that is encapsulated in a [[RequestContext]])
|
||||
* and returns a [[scala.concurrent.Future]] of [[RouteResult]] with the response (or the rejection).
|
||||
* and returns a [[java.util.concurrent.CompletionStage]] of [[RouteResult]] with the response (or the rejection).
|
||||
*
|
||||
* Use the methods in [[RequestContext]] to create a [[RouteResult]].
|
||||
* A handler MUST NOT return `null` as the result.
|
||||
*/
|
||||
trait AsyncHandler extends akka.japi.function.Function[RequestContext, Future[RouteResult]] {
|
||||
override def apply(ctx: RequestContext): Future[RouteResult]
|
||||
trait AsyncHandler extends akka.japi.function.Function[RequestContext, CompletionStage[RouteResult]] {
|
||||
override def apply(ctx: RequestContext): CompletionStage[RouteResult]
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,9 +4,9 @@
|
|||
|
||||
package akka.http.javadsl.server
|
||||
|
||||
import scala.concurrent.Future
|
||||
import akka.actor.ActorSystem
|
||||
import akka.http.scaladsl.Http.ServerBinding
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
/**
|
||||
* A convenience class to derive from to get everything from HttpService and Directives into scope.
|
||||
|
|
@ -22,6 +22,6 @@ abstract class HttpApp
|
|||
* Starts an HTTP server on the given interface and port. Creates the route by calling the
|
||||
* user-implemented [[createRoute]] method and uses the route to handle requests of the server.
|
||||
*/
|
||||
def bindRoute(interface: String, port: Int, system: ActorSystem): Future[ServerBinding] =
|
||||
def bindRoute(interface: String, port: Int, system: ActorSystem): CompletionStage[ServerBinding] =
|
||||
bindRoute(interface, port, createRoute(), system)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,19 +4,20 @@
|
|||
|
||||
package akka.http.javadsl.server
|
||||
|
||||
import scala.concurrent.Future
|
||||
import akka.actor.ActorSystem
|
||||
import akka.http.scaladsl.{ server, Http }
|
||||
import akka.http.scaladsl.Http.ServerBinding
|
||||
import akka.http.impl.server.RouteImplementation
|
||||
import akka.stream.{ ActorMaterializer, Materializer }
|
||||
import akka.stream.scaladsl.{ Keep, Sink }
|
||||
import java.util.concurrent.CompletionStage
|
||||
import scala.compat.java8.FutureConverters._
|
||||
|
||||
trait HttpServiceBase {
|
||||
/**
|
||||
* Starts a server on the given interface and port and uses the route to handle incoming requests.
|
||||
*/
|
||||
def bindRoute(interface: String, port: Int, route: Route, system: ActorSystem): Future[ServerBinding] = {
|
||||
def bindRoute(interface: String, port: Int, route: Route, system: ActorSystem): CompletionStage[ServerBinding] = {
|
||||
implicit val sys = system
|
||||
implicit val materializer = ActorMaterializer()
|
||||
handleConnectionsWithRoute(interface, port, route, system, materializer)
|
||||
|
|
@ -25,19 +26,19 @@ trait HttpServiceBase {
|
|||
/**
|
||||
* Starts a server on the given interface and port and uses the route to handle incoming requests.
|
||||
*/
|
||||
def bindRoute(interface: String, port: Int, route: Route, system: ActorSystem, materializer: Materializer): Future[ServerBinding] =
|
||||
def bindRoute(interface: String, port: Int, route: Route, system: ActorSystem, materializer: Materializer): CompletionStage[ServerBinding] =
|
||||
handleConnectionsWithRoute(interface, port, route, system, materializer)
|
||||
|
||||
/**
|
||||
* Uses the route to handle incoming connections and requests for the ServerBinding.
|
||||
*/
|
||||
def handleConnectionsWithRoute(interface: String, port: Int, route: Route, system: ActorSystem, materializer: Materializer): Future[ServerBinding] = {
|
||||
def handleConnectionsWithRoute(interface: String, port: Int, route: Route, system: ActorSystem, materializer: Materializer): CompletionStage[ServerBinding] = {
|
||||
implicit val s = system
|
||||
implicit val m = materializer
|
||||
|
||||
import system.dispatcher
|
||||
val r: server.Route = RouteImplementation(route)
|
||||
Http(system).bind(interface, port).toMat(Sink.foreach(_.handleWith(r)))(Keep.left).run()(materializer)
|
||||
Http(system).bind(interface, port).toMat(Sink.foreach(_.handleWith(r)))(Keep.left).run()(materializer).toJava
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,9 +4,10 @@
|
|||
|
||||
package akka.http.javadsl.server
|
||||
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
import scala.concurrent.ExecutionContextExecutor
|
||||
import akka.http.javadsl.model._
|
||||
import akka.stream.Materializer
|
||||
import java.util.concurrent.CompletionStage
|
||||
|
||||
/**
|
||||
* The RequestContext represents the state of the request while it is routed through
|
||||
|
|
@ -24,7 +25,7 @@ trait RequestContext {
|
|||
def unmatchedPath: String
|
||||
|
||||
/** Returns the ExecutionContext of this RequestContext */
|
||||
def executionContext(): ExecutionContext
|
||||
def executionContext(): ExecutionContextExecutor
|
||||
|
||||
/** Returns the Materializer of this RequestContext */
|
||||
def materializer(): Materializer
|
||||
|
|
@ -63,7 +64,7 @@ trait RequestContext {
|
|||
/**
|
||||
* Defers completion of the request
|
||||
*/
|
||||
def completeWith(futureResult: Future[RouteResult]): RouteResult
|
||||
def completeWith(futureResult: CompletionStage[RouteResult]): RouteResult
|
||||
|
||||
/**
|
||||
* Explicitly rejects the request as not found. Other route alternatives
|
||||
|
|
|
|||
|
|
@ -13,6 +13,8 @@ import akka.http.javadsl.server._
|
|||
|
||||
import scala.annotation.varargs
|
||||
import scala.concurrent.Future
|
||||
import java.util.concurrent.CompletionStage
|
||||
import scala.compat.java8.FutureConverters._
|
||||
|
||||
abstract class BasicDirectives extends BasicDirectivesBase {
|
||||
/**
|
||||
|
|
@ -152,16 +154,22 @@ abstract class BasicDirectives extends BasicDirectivesBase {
|
|||
res
|
||||
}
|
||||
def returnTypeMatches(method: Method): Boolean =
|
||||
method.getReturnType == classOf[RouteResult] || returnsFuture(method)
|
||||
method.getReturnType == classOf[RouteResult] || returnsFuture(method) || returnsCompletionStage(method)
|
||||
|
||||
def returnsFuture(method: Method): Boolean =
|
||||
method.getReturnType == classOf[Future[_]] &&
|
||||
method.getGenericReturnType.isInstanceOf[ParameterizedType] &&
|
||||
method.getGenericReturnType.asInstanceOf[ParameterizedType].getActualTypeArguments()(0) == classOf[RouteResult]
|
||||
|
||||
def returnsCompletionStage(method: Method): Boolean =
|
||||
method.getReturnType == classOf[CompletionStage[_]] &&
|
||||
method.getGenericReturnType.isInstanceOf[ParameterizedType] &&
|
||||
method.getGenericReturnType.asInstanceOf[ParameterizedType].getActualTypeArguments()(0) == classOf[RouteResult]
|
||||
|
||||
/** Makes sure both RouteResult and Future[RouteResult] are acceptable result types. */
|
||||
def adaptResult(method: Method): (RequestContext, AnyRef) ⇒ RouteResult =
|
||||
if (returnsFuture(method)) (ctx, v) ⇒ ctx.completeWith(v.asInstanceOf[Future[RouteResult]])
|
||||
if (returnsFuture(method)) (ctx, v) ⇒ ctx.completeWith(v.asInstanceOf[Future[RouteResult]].toJava)
|
||||
else if (returnsCompletionStage(method)) (ctx, v) => ctx.completeWith(v.asInstanceOf[CompletionStage[RouteResult]])
|
||||
else (_, v) ⇒ v.asInstanceOf[RouteResult]
|
||||
|
||||
val IdentityAdaptor: (RequestContext, Seq[Any]) ⇒ Seq[Any] = (_, ps) ⇒ ps
|
||||
|
|
|
|||
|
|
@ -7,9 +7,10 @@ package akka.http.javadsl.server.values
|
|||
import akka.http.impl.server.{ ExtractionImplBase, RouteStructure }
|
||||
import akka.http.javadsl.server.{ AbstractDirective, RequestVal, Route }
|
||||
import akka.http.scaladsl.util.FastFuture
|
||||
|
||||
import scala.concurrent.Future
|
||||
import scala.reflect.ClassTag
|
||||
import java.util.concurrent.CompletionStage
|
||||
import java.util.Optional
|
||||
import java.util.concurrent.CompletableFuture
|
||||
|
||||
/**
|
||||
* Represents existing or missing Http Basic authentication credentials.
|
||||
|
|
@ -38,18 +39,18 @@ trait BasicCredentials {
|
|||
*/
|
||||
abstract class HttpBasicAuthenticator[T](val realm: String) extends AbstractDirective with ExtractionImplBase[T] with RequestVal[T] {
|
||||
protected[http] implicit def classTag: ClassTag[T] = reflect.classTag[AnyRef].asInstanceOf[ClassTag[T]]
|
||||
def authenticate(credentials: BasicCredentials): Future[Option[T]]
|
||||
def authenticate(credentials: BasicCredentials): CompletionStage[Optional[T]]
|
||||
|
||||
/**
|
||||
* Creates a return value for use in [[authenticate]] that successfully authenticates the requests and provides
|
||||
* the given user.
|
||||
*/
|
||||
def authenticateAs(user: T): Future[Option[T]] = FastFuture.successful(Some(user))
|
||||
def authenticateAs(user: T): CompletionStage[Optional[T]] = CompletableFuture.completedFuture(Optional.of(user))
|
||||
|
||||
/**
|
||||
* Refuses access for this user.
|
||||
*/
|
||||
def refuseAccess(): Future[Option[T]] = FastFuture.successful(None)
|
||||
def refuseAccess(): CompletionStage[Optional[T]] = CompletableFuture.completedFuture(Optional.empty())
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -6,10 +6,10 @@ package akka.http.javadsl.server.values
|
|||
|
||||
import akka.http.impl.server.{ ExtractionImplBase, RouteStructure }
|
||||
import akka.http.javadsl.server.{ AbstractDirective, RequestVal, Route }
|
||||
import akka.http.scaladsl.util.FastFuture
|
||||
|
||||
import scala.concurrent.Future
|
||||
import scala.reflect.ClassTag
|
||||
import java.util.concurrent.CompletionStage
|
||||
import java.util.Optional
|
||||
import java.util.concurrent.CompletableFuture
|
||||
|
||||
/**
|
||||
* Represents existing or missing OAuth 2 authentication credentials.
|
||||
|
|
@ -38,18 +38,18 @@ trait OAuth2Credentials {
|
|||
*/
|
||||
abstract class OAuth2Authenticator[T](val realm: String) extends AbstractDirective with ExtractionImplBase[T] with RequestVal[T] {
|
||||
protected[http] implicit def classTag: ClassTag[T] = reflect.classTag[AnyRef].asInstanceOf[ClassTag[T]]
|
||||
def authenticate(credentials: OAuth2Credentials): Future[Option[T]]
|
||||
def authenticate(credentials: OAuth2Credentials): CompletionStage[Optional[T]]
|
||||
|
||||
/**
|
||||
* Creates a return value for use in [[authenticate]] that successfully authenticates the requests and provides
|
||||
* the given user.
|
||||
*/
|
||||
def authenticateAs(user: T): Future[Option[T]] = FastFuture.successful(Some(user))
|
||||
def authenticateAs(user: T): CompletionStage[Optional[T]] = CompletableFuture.completedFuture(Optional.of(user))
|
||||
|
||||
/**
|
||||
* Refuses access for this user.
|
||||
*/
|
||||
def refuseAccess(): Future[Option[T]] = FastFuture.successful(None)
|
||||
def refuseAccess(): CompletionStage[Optional[T]] = CompletableFuture.completedFuture(Optional.empty())
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
|
||||
package akka.http.scaladsl.server
|
||||
|
||||
import scala.concurrent.{ Future, ExecutionContext }
|
||||
import scala.concurrent.{ Future, ExecutionContextExecutor }
|
||||
import akka.stream.Materializer
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.http.scaladsl.marshalling.ToResponseMarshallable
|
||||
|
|
@ -25,7 +25,7 @@ trait RequestContext {
|
|||
/**
|
||||
* The default ExecutionContext to be used for scheduling asynchronous logic related to this request.
|
||||
*/
|
||||
implicit def executionContext: ExecutionContext
|
||||
implicit def executionContext: ExecutionContextExecutor
|
||||
|
||||
/**
|
||||
* The default Materializer.
|
||||
|
|
@ -46,7 +46,7 @@ trait RequestContext {
|
|||
* Returns a copy of this context with the given fields updated.
|
||||
*/
|
||||
def reconfigure(
|
||||
executionContext: ExecutionContext = executionContext,
|
||||
executionContext: ExecutionContextExecutor = executionContext,
|
||||
materializer: Materializer = materializer,
|
||||
log: LoggingAdapter = log,
|
||||
settings: RoutingSettings = settings): RequestContext
|
||||
|
|
@ -76,7 +76,7 @@ trait RequestContext {
|
|||
/**
|
||||
* Returns a copy of this context with the new HttpRequest.
|
||||
*/
|
||||
def withExecutionContext(ec: ExecutionContext): RequestContext
|
||||
def withExecutionContext(ec: ExecutionContextExecutor): RequestContext
|
||||
|
||||
/**
|
||||
* Returns a copy of this context with the new HttpRequest.
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
|
||||
package akka.http.scaladsl.server
|
||||
|
||||
import scala.concurrent.{ Future, ExecutionContext }
|
||||
import scala.concurrent.{ Future, ExecutionContextExecutor }
|
||||
import akka.stream.Materializer
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.http.scaladsl.marshalling.{ Marshal, ToResponseMarshallable }
|
||||
|
|
@ -18,15 +18,15 @@ import akka.http.scaladsl.util.FastFuture._
|
|||
private[http] class RequestContextImpl(
|
||||
val request: HttpRequest,
|
||||
val unmatchedPath: Uri.Path,
|
||||
val executionContext: ExecutionContext,
|
||||
val executionContext: ExecutionContextExecutor,
|
||||
val materializer: Materializer,
|
||||
val log: LoggingAdapter,
|
||||
val settings: RoutingSettings) extends RequestContext {
|
||||
|
||||
def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContext, materializer: Materializer) =
|
||||
def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContextExecutor, materializer: Materializer) =
|
||||
this(request, request.uri.path, ec, materializer, log, settings)
|
||||
|
||||
def reconfigure(executionContext: ExecutionContext, materializer: Materializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext =
|
||||
def reconfigure(executionContext: ExecutionContextExecutor, materializer: Materializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext =
|
||||
copy(executionContext = executionContext, materializer = materializer, log = log, settings = settings)
|
||||
|
||||
override def complete(trm: ToResponseMarshallable): Future[RouteResult] =
|
||||
|
|
@ -47,7 +47,7 @@ private[http] class RequestContextImpl(
|
|||
override def withRequest(request: HttpRequest): RequestContext =
|
||||
if (request != this.request) copy(request = request) else this
|
||||
|
||||
override def withExecutionContext(executionContext: ExecutionContext): RequestContext =
|
||||
override def withExecutionContext(executionContext: ExecutionContextExecutor): RequestContext =
|
||||
if (executionContext != this.executionContext) copy(executionContext = executionContext) else this
|
||||
|
||||
override def withMaterializer(materializer: Materializer): RequestContext =
|
||||
|
|
@ -83,7 +83,7 @@ private[http] class RequestContextImpl(
|
|||
|
||||
private def copy(request: HttpRequest = request,
|
||||
unmatchedPath: Uri.Path = unmatchedPath,
|
||||
executionContext: ExecutionContext = executionContext,
|
||||
executionContext: ExecutionContextExecutor = executionContext,
|
||||
materializer: Materializer = materializer,
|
||||
log: LoggingAdapter = log,
|
||||
settings: RoutingSettings = settings) =
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.http.scaladsl.server
|
|||
import akka.NotUsed
|
||||
import akka.stream.Materializer
|
||||
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
import scala.concurrent.{ ExecutionContextExecutor, Future }
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.http.scaladsl.model.{ HttpRequest, HttpResponse }
|
||||
import akka.http.scaladsl.util.FastFuture._
|
||||
|
|
@ -41,7 +41,7 @@ object Route {
|
|||
def handlerFlow(route: Route)(implicit routingSettings: RoutingSettings,
|
||||
materializer: Materializer,
|
||||
routingLog: RoutingLog,
|
||||
executionContext: ExecutionContext = null,
|
||||
executionContext: ExecutionContextExecutor = null,
|
||||
rejectionHandler: RejectionHandler = RejectionHandler.default,
|
||||
exceptionHandler: ExceptionHandler = null): Flow[HttpRequest, HttpResponse, NotUsed] =
|
||||
Flow[HttpRequest].mapAsync(1)(asyncHandler(route))
|
||||
|
|
@ -52,7 +52,7 @@ object Route {
|
|||
def asyncHandler(route: Route)(implicit routingSettings: RoutingSettings,
|
||||
materializer: Materializer,
|
||||
routingLog: RoutingLog,
|
||||
executionContext: ExecutionContext = null,
|
||||
executionContext: ExecutionContextExecutor = null,
|
||||
rejectionHandler: RejectionHandler = RejectionHandler.default,
|
||||
exceptionHandler: ExceptionHandler = null): HttpRequest ⇒ Future[HttpResponse] = {
|
||||
val effectiveEC = if (executionContext ne null) executionContext else materializer.executionContext
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.http.scaladsl.server
|
||||
package directives
|
||||
|
||||
import scala.concurrent.{ Future, ExecutionContext }
|
||||
import scala.concurrent.{ Future, ExecutionContextExecutor }
|
||||
import scala.collection.immutable
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.Materializer
|
||||
|
|
@ -132,13 +132,13 @@ trait BasicDirectives {
|
|||
/**
|
||||
* Runs its inner route with the given alternative [[ExecutionContext]].
|
||||
*/
|
||||
def withExecutionContext(ec: ExecutionContext): Directive0 =
|
||||
def withExecutionContext(ec: ExecutionContextExecutor): Directive0 =
|
||||
mapRequestContext(_ withExecutionContext ec)
|
||||
|
||||
/**
|
||||
* Extracts the [[ExecutionContext]] from the [[RequestContext]].
|
||||
*/
|
||||
def extractExecutionContext: Directive1[ExecutionContext] = BasicDirectives._extractExecutionContext
|
||||
def extractExecutionContext: Directive1[ExecutionContextExecutor] = BasicDirectives._extractExecutionContext
|
||||
|
||||
/**
|
||||
* Runs its inner route with the given alternative [[Materializer]].
|
||||
|
|
@ -191,7 +191,7 @@ object BasicDirectives extends BasicDirectives {
|
|||
private val _extractUnmatchedPath: Directive1[Uri.Path] = extract(_.unmatchedPath)
|
||||
private val _extractRequest: Directive1[HttpRequest] = extract(_.request)
|
||||
private val _extractUri: Directive1[Uri] = extract(_.request.uri)
|
||||
private val _extractExecutionContext: Directive1[ExecutionContext] = extract(_.executionContext)
|
||||
private val _extractExecutionContext: Directive1[ExecutionContextExecutor] = extract(_.executionContext)
|
||||
private val _extractMaterializer: Directive1[Materializer] = extract(_.materializer)
|
||||
private val _extractLog: Directive1[LoggingAdapter] = extract(_.log)
|
||||
private val _extractSettings: Directive1[RoutingSettings] = extract(_.settings)
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import scala.concurrent.Future;
|
|||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.io.OutputStream;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
|
@ -32,8 +33,6 @@ public class OutputStreamSinkTest extends StreamTest {
|
|||
@Test
|
||||
public void mustSignalFailureViaIoResult() throws Exception {
|
||||
|
||||
final FiniteDuration timeout = FiniteDuration.create(300, TimeUnit.MILLISECONDS);
|
||||
|
||||
final OutputStream os = new OutputStream() {
|
||||
volatile int left = 3;
|
||||
public void write(int data) {
|
||||
|
|
@ -43,8 +42,8 @@ public class OutputStreamSinkTest extends StreamTest {
|
|||
left -= 1;
|
||||
}
|
||||
};
|
||||
final Future<IOResult> resultFuture = Source.single(ByteString.fromString("123456")).runWith(StreamConverters.fromOutputStream(() -> os), materializer);
|
||||
final IOResult result = Await.result(resultFuture, timeout);
|
||||
final CompletionStage<IOResult> resultFuture = Source.single(ByteString.fromString("123456")).runWith(StreamConverters.fromOutputStream(() -> os), materializer);
|
||||
final IOResult result = resultFuture.toCompletableFuture().get(300, TimeUnit.MILLISECONDS);
|
||||
|
||||
assertFalse(result.wasSuccessful());
|
||||
assertTrue(result.getError().getMessage().equals("Can't accept more data."));
|
||||
|
|
|
|||
|
|
@ -6,6 +6,7 @@ package akka.stream.javadsl;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.NotUsed;
|
||||
|
|
@ -85,33 +86,20 @@ public class BidiFlowTest extends StreamTest {
|
|||
}
|
||||
}));
|
||||
|
||||
private final BidiFlow<Integer, Long, ByteString, String, Future<Integer>> bidiMat =
|
||||
private final BidiFlow<Integer, Long, ByteString, String, CompletionStage<Integer>> bidiMat =
|
||||
BidiFlow.fromGraph(
|
||||
GraphDSL.create(
|
||||
Sink.<Integer>head(),
|
||||
new Function2<GraphDSL.Builder<Future<Integer>>, SinkShape<Integer>, BidiShape<Integer, Long, ByteString, String>>() {
|
||||
@Override
|
||||
public BidiShape<Integer, Long, ByteString, String> apply(Builder<Future<Integer>> b, SinkShape<Integer> sink)
|
||||
throws Exception {
|
||||
(b, sink) -> {
|
||||
b.from(b.add(Source.single(42))).to(sink);
|
||||
final FlowShape<Integer, Long> top = b.add(Flow
|
||||
.of(Integer.class).map(new Function<Integer, Long>() {
|
||||
@Override
|
||||
public Long apply(Integer arg) {
|
||||
return (long) ((int) arg) + 2;
|
||||
}
|
||||
}));
|
||||
.of(Integer.class).map(i -> (long)(i + 2)));
|
||||
final FlowShape<ByteString, String> bottom = b.add(Flow
|
||||
.of(ByteString.class).map(new Function<ByteString, String>() {
|
||||
@Override
|
||||
public String apply(ByteString arg) {
|
||||
return arg.decodeString("UTF-8");
|
||||
}
|
||||
}));
|
||||
.of(ByteString.class).map(bytes -> bytes.decodeString("UTF-8")));
|
||||
return new BidiShape<Integer, Long, ByteString, String>(top
|
||||
.in(), top.out(), bottom.in(), bottom.out());
|
||||
}
|
||||
}));
|
||||
));
|
||||
|
||||
private final String str = "Hello World";
|
||||
private final ByteString bytes = ByteString.fromString(str);
|
||||
|
|
@ -125,14 +113,11 @@ public class BidiFlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustWorkInIsolation() throws Exception {
|
||||
final Pair<Future<Long>, Future<String>> p =
|
||||
final Pair<CompletionStage<Long>, CompletionStage<String>> p =
|
||||
RunnableGraph.fromGraph(GraphDSL
|
||||
.create(Sink.<Long> head(), Sink.<String> head(),
|
||||
Keep.<Future<Long>, Future<String>> both(),
|
||||
new Function3<Builder<Pair<Future<Long>, Future<String>>>, SinkShape<Long>, SinkShape<String>, ClosedShape>() {
|
||||
@Override
|
||||
public ClosedShape apply(Builder<Pair<Future<Long>, Future<String>>> b, SinkShape<Long> st,
|
||||
SinkShape<String> sb) throws Exception {
|
||||
Keep.both(),
|
||||
(b, st, sb) -> {
|
||||
final BidiShape<Integer, Long, ByteString, String> s =
|
||||
b.add(bidi);
|
||||
b.from(b.add(Source.single(1))).toInlet(s.in1());
|
||||
|
|
@ -140,11 +125,10 @@ public class BidiFlowTest extends StreamTest {
|
|||
b.from(b.add(Source.single(bytes))).toInlet(s.in2());
|
||||
b.from(s.out2()).to(sb);
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
})).run(materializer);
|
||||
|
||||
final Long rt = Await.result(p.first(), oneSec);
|
||||
final String rb = Await.result(p.second(), oneSec);
|
||||
final Long rt = p.first().toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
final String rb = p.second().toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals((Long) 3L, rt);
|
||||
assertEquals(str, rb);
|
||||
|
|
@ -158,8 +142,8 @@ public class BidiFlowTest extends StreamTest {
|
|||
return ByteString.fromString("Hello " + arg);
|
||||
}
|
||||
}));
|
||||
final Future<List<String>> result = Source.from(list).via(f).grouped(10).runWith(Sink.<List<String>> head(), materializer);
|
||||
assertEquals(Arrays.asList("Hello 3", "Hello 4", "Hello 5"), Await.result(result, oneSec));
|
||||
final CompletionStage<List<String>> result = Source.from(list).via(f).grouped(10).runWith(Sink.<List<String>> head(), materializer);
|
||||
assertEquals(Arrays.asList("Hello 3", "Hello 4", "Hello 5"), result.toCompletableFuture().get(1, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -171,8 +155,8 @@ public class BidiFlowTest extends StreamTest {
|
|||
}
|
||||
}).join(bidi);
|
||||
final List<ByteString> inputs = Arrays.asList(ByteString.fromString("1"), ByteString.fromString("2"));
|
||||
final Future<List<Long>> result = Source.from(inputs).via(f).grouped(10).runWith(Sink.<List<Long>> head(), materializer);
|
||||
assertEquals(Arrays.asList(3L, 4L), Await.result(result, oneSec));
|
||||
final CompletionStage<List<Long>> result = Source.from(inputs).via(f).grouped(10).runWith(Sink.<List<Long>> head(), materializer);
|
||||
assertEquals(Arrays.asList(3L, 4L), result.toCompletableFuture().get(1, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -183,8 +167,8 @@ public class BidiFlowTest extends StreamTest {
|
|||
return arg.toString();
|
||||
}
|
||||
}));
|
||||
final Future<List<String>> result = Source.from(list).via(f).grouped(10).runWith(Sink.<List<String>> head(), materializer);
|
||||
assertEquals(Arrays.asList("5", "6", "7"), Await.result(result, oneSec));
|
||||
final CompletionStage<List<String>> result = Source.from(list).via(f).grouped(10).runWith(Sink.<List<String>> head(), materializer);
|
||||
assertEquals(Arrays.asList("5", "6", "7"), result.toCompletableFuture().get(1, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -195,80 +179,49 @@ public class BidiFlowTest extends StreamTest {
|
|||
return arg.toString();
|
||||
}
|
||||
}).join(inverse.reversed()).join(bidi.reversed());
|
||||
final Future<List<String>> result = Source.from(list).via(f).grouped(10).runWith(Sink.<List<String>> head(), materializer);
|
||||
assertEquals(Arrays.asList("5", "6", "7"), Await.result(result, oneSec));
|
||||
final CompletionStage<List<String>> result = Source.from(list).via(f).grouped(10).runWith(Sink.<List<String>> head(), materializer);
|
||||
assertEquals(Arrays.asList("5", "6", "7"), result.toCompletableFuture().get(1, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustMaterializeToItsValue() throws Exception {
|
||||
final Future<Integer> f = RunnableGraph.fromGraph(
|
||||
GraphDSL.create(bidiMat,
|
||||
new Function2<Builder<Future<Integer> >, BidiShape<Integer, Long, ByteString, String>, ClosedShape>() {
|
||||
@Override
|
||||
public ClosedShape apply(Builder<Future<Integer>> b,
|
||||
BidiShape<Integer, Long, ByteString, String> shape) throws Exception {
|
||||
final FlowShape<String, Integer> left = b.add(Flow.of(String.class).map(
|
||||
new Function<String, Integer>() {
|
||||
@Override
|
||||
public Integer apply(String arg) {
|
||||
return Integer.valueOf(arg);
|
||||
}
|
||||
}));
|
||||
final FlowShape<Long, ByteString> right = b.add(Flow.of(Long.class).map(
|
||||
new Function<Long, ByteString>() {
|
||||
@Override
|
||||
public ByteString apply(Long arg) {
|
||||
return ByteString.fromString("Hello " + arg);
|
||||
}
|
||||
}));
|
||||
final CompletionStage<Integer> f = RunnableGraph.fromGraph(
|
||||
GraphDSL.create(bidiMat, (b, shape) -> {
|
||||
final FlowShape<String, Integer> left = b.add(Flow.of(String.class).map(Integer::valueOf));
|
||||
final FlowShape<Long, ByteString> right = b.add(Flow.of(Long.class).map(s -> ByteString.fromString("Hello " + s)));
|
||||
b.from(shape.out2()).via(left).toInlet(shape.in1())
|
||||
.from(shape.out1()).via(right).toInlet(shape.in2());
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
})).run(materializer);
|
||||
assertEquals((Integer) 42, Await.result(f, oneSec));
|
||||
assertEquals((Integer) 42, f.toCompletableFuture().get(1, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustCombineMaterializationValues() throws Exception {
|
||||
final Flow<String, Integer, Future<Integer>> left = Flow.fromGraph(GraphDSL.create(
|
||||
Sink.<Integer>head(), new Function2<Builder<Future<Integer>>, SinkShape<Integer>, FlowShape<String, Integer>>() {
|
||||
@Override
|
||||
public FlowShape<String, Integer> apply(Builder<Future<Integer>> b,
|
||||
SinkShape<Integer> sink) throws Exception {
|
||||
final Flow<String, Integer, CompletionStage<Integer>> left = Flow.fromGraph(GraphDSL.create(
|
||||
Sink.<Integer>head(), (b, sink) -> {
|
||||
final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.<Integer>create(2));
|
||||
final UniformFanInShape<Integer, Integer> merge = b.add(Merge.<Integer>create(2));
|
||||
final FlowShape<String, Integer> flow = b.add(Flow.of(String.class).map(
|
||||
new Function<String, Integer>() {
|
||||
@Override
|
||||
public Integer apply(String arg) {
|
||||
return Integer.valueOf(arg);
|
||||
}
|
||||
}));
|
||||
final FlowShape<String, Integer> flow = b.add(Flow.of(String.class).map(Integer::valueOf));
|
||||
b.from(bcast).to(sink)
|
||||
.from(b.add(Source.single(1))).viaFanOut(bcast).toFanIn(merge)
|
||||
.from(flow).toFanIn(merge);
|
||||
return new FlowShape<String, Integer>(flow.in(), merge.out());
|
||||
}
|
||||
}));
|
||||
final Flow<Long, ByteString, Future<List<Long>>> right = Flow.fromGraph(GraphDSL.create(
|
||||
Sink.<List<Long>>head(), new Function2<Builder<Future<List<Long>>>, SinkShape<List<Long>>, FlowShape<Long, ByteString>>() {
|
||||
@Override
|
||||
public FlowShape<Long, ByteString> apply(Builder<Future<List<Long>>> b,
|
||||
SinkShape<List<Long>> sink) throws Exception {
|
||||
final Flow<Long, ByteString, CompletionStage<List<Long>>> right = Flow.fromGraph(GraphDSL.create(
|
||||
Sink.<List<Long>>head(), (b, sink) -> {
|
||||
final FlowShape<Long, List<Long>> flow = b.add(Flow.of(Long.class).grouped(10));
|
||||
b.from(flow).to(sink);
|
||||
return new FlowShape<Long, ByteString>(flow.in(), b.add(Source.single(ByteString.fromString("10"))).out());
|
||||
}
|
||||
}));
|
||||
final Pair<Pair<Future<Integer>, Future<Integer>>, Future<List<Long>>> result =
|
||||
left.joinMat(bidiMat, Keep.<Future<Integer>, Future<Integer>> both()).joinMat(right, Keep.<Pair<Future<Integer>, Future<Integer>>, Future<List<Long>>> both()).run(materializer);
|
||||
final Future<Integer> l = result.first().first();
|
||||
final Future<Integer> m = result.first().second();
|
||||
final Future<List<Long>> r = result.second();
|
||||
assertEquals((Integer) 1, Await.result(l, oneSec));
|
||||
assertEquals((Integer) 42, Await.result(m, oneSec));
|
||||
final Long[] rr = Await.result(r, oneSec).toArray(new Long[2]);
|
||||
final Pair<Pair<CompletionStage<Integer>, CompletionStage<Integer>>, CompletionStage<List<Long>>> result =
|
||||
left.joinMat(bidiMat, Keep.both()).joinMat(right, Keep.both()).run(materializer);
|
||||
final CompletionStage<Integer> l = result.first().first();
|
||||
final CompletionStage<Integer> m = result.first().second();
|
||||
final CompletionStage<List<Long>> r = result.second();
|
||||
assertEquals((Integer) 1, l.toCompletableFuture().get(1, TimeUnit.SECONDS));
|
||||
assertEquals((Integer) 42, m.toCompletableFuture().get(1, TimeUnit.SECONDS));
|
||||
final Long[] rr = r.toCompletableFuture().get(1, TimeUnit.SECONDS).toArray(new Long[2]);
|
||||
Arrays.sort(rr);
|
||||
assertArrayEquals(new Long[] { 3L, 12L }, rr);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream.javadsl;
|
|||
|
||||
import akka.NotUsed;
|
||||
import akka.japi.Pair;
|
||||
import akka.pattern.Patterns;
|
||||
import akka.pattern.PatternsCS;
|
||||
import akka.japi.tuple.Tuple4;
|
||||
import akka.stream.*;
|
||||
import akka.stream.javadsl.GraphDSL.Builder;
|
||||
|
|
@ -23,6 +23,7 @@ import scala.concurrent.Future;
|
|||
import scala.concurrent.duration.Duration;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
|
|
@ -83,9 +84,9 @@ public class FlowGraphTest extends StreamTest {
|
|||
|
||||
// collecting
|
||||
final Publisher<String> pub = source.runWith(publisher, materializer);
|
||||
final Future<List<String>> all = Source.fromPublisher(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
|
||||
final CompletionStage<List<String>> all = Source.fromPublisher(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
|
||||
|
||||
final List<String> result = Await.result(all, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
final List<String> result = all.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
assertEquals(new HashSet<Object>(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet<String>(result));
|
||||
}
|
||||
|
||||
|
|
@ -259,19 +260,16 @@ public class FlowGraphTest extends StreamTest {
|
|||
}
|
||||
});
|
||||
|
||||
final Future<Integer> future = RunnableGraph.fromGraph(GraphDSL.create(Sink.<Integer>head(),
|
||||
new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() {
|
||||
@Override
|
||||
public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
|
||||
final CompletionStage<Integer> future = RunnableGraph.fromGraph(GraphDSL.create(Sink.<Integer>head(),
|
||||
(b, out) -> {
|
||||
final FanInShape2<Integer, Integer, Integer> zip = b.add(sumZip);
|
||||
b.from(b.add(in1)).toInlet(zip.in0());
|
||||
b.from(b.add(in2)).toInlet(zip.in1());
|
||||
b.from(zip.out()).to(out);
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
})).run(materializer);
|
||||
|
||||
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
|
||||
final Integer result = future.toCompletableFuture().get(300, TimeUnit.MILLISECONDS);
|
||||
assertEquals(11, (int) result);
|
||||
}
|
||||
|
||||
|
|
@ -289,11 +287,8 @@ public class FlowGraphTest extends StreamTest {
|
|||
}
|
||||
});
|
||||
|
||||
final Future<Integer> future = RunnableGraph.fromGraph(
|
||||
GraphDSL.create(Sink.<Integer>head(),
|
||||
new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() {
|
||||
@Override
|
||||
public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
|
||||
final CompletionStage<Integer> future = RunnableGraph.fromGraph(
|
||||
GraphDSL.create(Sink.<Integer>head(), (b, out) -> {
|
||||
final FanInShape4<Integer, Integer, Integer, Integer, Integer> zip = b.add(sumZip);
|
||||
b.from(b.add(in1)).toInlet(zip.in0());
|
||||
b.from(b.add(in2)).toInlet(zip.in1());
|
||||
|
|
@ -301,10 +296,9 @@ public class FlowGraphTest extends StreamTest {
|
|||
b.from(b.add(in4)).toInlet(zip.in3());
|
||||
b.from(zip.out()).to(out);
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
})).run(materializer);
|
||||
|
||||
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
|
||||
final Integer result = future.toCompletableFuture().get(300, TimeUnit.MILLISECONDS);
|
||||
assertEquals(1111, (int) result);
|
||||
}
|
||||
|
||||
|
|
@ -314,21 +308,14 @@ public class FlowGraphTest extends StreamTest {
|
|||
final Source<Integer, NotUsed> in1 = Source.single(1);
|
||||
final TestProbe probe = TestProbe.apply(system);
|
||||
|
||||
final Future<Integer> future = RunnableGraph.fromGraph(
|
||||
GraphDSL.create(Sink.<Integer> head(), new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() {
|
||||
@Override
|
||||
public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
|
||||
final CompletionStage<Integer> future = RunnableGraph.fromGraph(
|
||||
GraphDSL.create(Sink.<Integer> head(), (b, out) -> {
|
||||
b.from(b.add(Source.single(1))).to(out);
|
||||
b.from(b.materializedValue()).to(b.add(Sink.foreach(new Procedure<Future<Integer>>(){
|
||||
public void apply(Future<Integer> mat) throws Exception {
|
||||
Patterns.pipe(mat, system.dispatcher()).to(probe.ref());
|
||||
}
|
||||
})));
|
||||
b.from(b.materializedValue()).to(b.add(Sink.foreach(mat -> PatternsCS.pipe(mat, system.dispatcher()).to(probe.ref()))));
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
})).run(materializer);
|
||||
|
||||
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
|
||||
final Integer result = future.toCompletableFuture().get(300, TimeUnit.MILLISECONDS);
|
||||
assertEquals(1, (int) result);
|
||||
|
||||
probe.expectMsg(1);
|
||||
|
|
|
|||
|
|
@ -28,6 +28,9 @@ import scala.concurrent.duration.Duration;
|
|||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.stream.Stream;
|
||||
|
|
@ -50,8 +53,8 @@ public class FlowTest extends StreamTest {
|
|||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final String[] lookup = { "a", "b", "c", "d", "e", "f" };
|
||||
final java.lang.Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5);
|
||||
final Source<Integer, ?> ints = Source.from(input);
|
||||
final Flow<Integer, String, ?> flow1 = Flow.of(Integer.class).drop(2).take(3
|
||||
final Source<Integer, NotUsed> ints = Source.from(input);
|
||||
final Flow<Integer, String, NotUsed> flow1 = Flow.of(Integer.class).drop(2).take(3
|
||||
).takeWithin(FiniteDuration.create(10, TimeUnit.SECONDS
|
||||
)).map(new Function<Integer, String>() {
|
||||
public String apply(Integer elem) {
|
||||
|
|
@ -62,7 +65,7 @@ public class FlowTest extends StreamTest {
|
|||
return !elem.equals("c");
|
||||
}
|
||||
});
|
||||
final Flow<String, String, ?> flow2 = Flow.of(String.class).grouped(2
|
||||
final Flow<String, String, NotUsed> flow2 = Flow.of(String.class).grouped(2
|
||||
).mapConcat(new Function<java.util.List<String>, java.lang.Iterable<String>>() {
|
||||
public java.util.List<String> apply(java.util.List<String> elem) {
|
||||
return elem;
|
||||
|
|
@ -74,16 +77,9 @@ public class FlowTest extends StreamTest {
|
|||
}
|
||||
});
|
||||
|
||||
ints.via(flow1.via(flow2)).runFold("", new Function2<String, String, String>() {
|
||||
public String apply(String acc, String elem) {
|
||||
return acc + elem;
|
||||
}
|
||||
}, materializer
|
||||
).foreach(new Foreach<String>() { // Scala Future
|
||||
public void each(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}, system.dispatcher());
|
||||
ints.via(flow1.via(flow2))
|
||||
.runFold("", (acc, elem) -> acc + elem, materializer)
|
||||
.thenAccept(elem -> probe.getRef().tell(elem, ActorRef.noSender()));
|
||||
|
||||
probe.expectMsgEquals("de");
|
||||
}
|
||||
|
|
@ -91,36 +87,26 @@ public class FlowTest extends StreamTest {
|
|||
@Test
|
||||
public void mustBeAbleToUseDropWhile() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Source<Integer, ?> source = Source.from(Arrays.asList(0, 1, 2, 3));
|
||||
final Flow<Integer, Integer, ?> flow = Flow.of(Integer.class).dropWhile
|
||||
(new Predicate<Integer>() {
|
||||
public boolean test(Integer elem) {
|
||||
return elem < 2;
|
||||
}
|
||||
});
|
||||
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3));
|
||||
final Flow<Integer, Integer, NotUsed> flow =
|
||||
Flow.of(Integer.class).dropWhile(elem -> elem < 2);
|
||||
|
||||
final Future<Done> future = source.via(flow).runWith(Sink.foreach(new Procedure<Integer>() { // Scala Future
|
||||
public void apply(Integer elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future =
|
||||
source.via(flow).runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
|
||||
probe.expectMsgEquals(2);
|
||||
probe.expectMsgEquals(3);
|
||||
Await.ready(future, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseIntersperse() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Source<String, ?> source = Source.from(Arrays.asList("0", "1", "2", "3"));
|
||||
final Flow<String, String, ?> flow = Flow.of(String.class).intersperse("[", ",", "]");
|
||||
final Source<String, NotUsed> source = Source.from(Arrays.asList("0", "1", "2", "3"));
|
||||
final Flow<String, String, NotUsed> flow = Flow.of(String.class).intersperse("[", ",", "]");
|
||||
|
||||
final Future<Done> future = source.via(flow).runWith(Sink.foreach(new Procedure<String>() { // Scala Future
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future =
|
||||
source.via(flow).runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
|
||||
probe.expectMsgEquals("[");
|
||||
probe.expectMsgEquals("0");
|
||||
|
|
@ -131,20 +117,17 @@ public class FlowTest extends StreamTest {
|
|||
probe.expectMsgEquals(",");
|
||||
probe.expectMsgEquals("3");
|
||||
probe.expectMsgEquals("]");
|
||||
Await.ready(future, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseIntersperseAndConcat() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Source<String, ?> source = Source.from(Arrays.asList("0", "1", "2", "3"));
|
||||
final Flow<String, String, ?> flow = Flow.of(String.class).intersperse(",");
|
||||
final Source<String, NotUsed> source = Source.from(Arrays.asList("0", "1", "2", "3"));
|
||||
final Flow<String, String, NotUsed> flow = Flow.of(String.class).intersperse(",");
|
||||
|
||||
final Future<Done> future = Source.single(">> ").concat(source.via(flow)).runWith(Sink.foreach(new Procedure<String>() { // Scala Future
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future =
|
||||
Source.single(">> ").concat(source.via(flow)).runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
|
||||
probe.expectMsgEquals(">> ");
|
||||
probe.expectMsgEquals("0");
|
||||
|
|
@ -154,25 +137,22 @@ public class FlowTest extends StreamTest {
|
|||
probe.expectMsgEquals("2");
|
||||
probe.expectMsgEquals(",");
|
||||
probe.expectMsgEquals("3");
|
||||
Await.ready(future, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseTakeWhile() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Source<Integer, ?> source = Source.from(Arrays.asList(0, 1, 2, 3));
|
||||
final Flow<Integer, Integer, ?> flow = Flow.of(Integer.class).takeWhile
|
||||
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3));
|
||||
final Flow<Integer, Integer, NotUsed> flow = Flow.of(Integer.class).takeWhile
|
||||
(new Predicate<Integer>() {
|
||||
public boolean test(Integer elem) {
|
||||
return elem < 2;
|
||||
}
|
||||
});
|
||||
|
||||
final Future<Done> future = source.via(flow).runWith(Sink.foreach(new Procedure<Integer>() { // Scala Future
|
||||
public void apply(Integer elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future =
|
||||
source.via(flow).runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
|
||||
probe.expectMsgEquals(0);
|
||||
probe.expectMsgEquals(1);
|
||||
|
|
@ -180,7 +160,7 @@ public class FlowTest extends StreamTest {
|
|||
FiniteDuration duration = Duration.apply(200, TimeUnit.MILLISECONDS);
|
||||
|
||||
probe.expectNoMsg(duration);
|
||||
Await.ready(future, duration);
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -189,7 +169,7 @@ public class FlowTest extends StreamTest {
|
|||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7);
|
||||
// duplicate each element, stop after 4 elements, and emit sum to the end
|
||||
final Flow<Integer, Integer, ?> flow = Flow.of(Integer.class).transform(new Creator<Stage<Integer, Integer>>() {
|
||||
final Flow<Integer, Integer, NotUsed> flow = Flow.of(Integer.class).transform(new Creator<Stage<Integer, Integer>>() {
|
||||
@Override
|
||||
public PushPullStage<Integer, Integer> create() throws Exception {
|
||||
return new StatefulStage<Integer, Integer>() {
|
||||
|
|
@ -252,9 +232,9 @@ public class FlowTest extends StreamTest {
|
|||
.grouped(10)
|
||||
.mergeSubstreams();
|
||||
|
||||
final Future<List<List<String>>> future =
|
||||
final CompletionStage<List<List<String>>> future =
|
||||
Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
|
||||
final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray();
|
||||
final Object[] result = future.toCompletableFuture().get(1, TimeUnit.SECONDS).toArray();
|
||||
Arrays.sort(result, (Comparator<Object>)(Object) new Comparator<List<String>>() {
|
||||
@Override
|
||||
public int compare(List<String> o1, List<String> o2) {
|
||||
|
|
@ -278,9 +258,9 @@ public class FlowTest extends StreamTest {
|
|||
.grouped(10)
|
||||
.concatSubstreams();
|
||||
|
||||
final Future<List<List<String>>> future =
|
||||
final CompletionStage<List<List<String>>> future =
|
||||
Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
|
||||
final List<List<String>> result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final List<List<String>> result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals(Arrays.asList(Arrays.asList("A", "B", "C"), Arrays.asList(".", "D"), Arrays.asList(".", "E", "F")), result);
|
||||
}
|
||||
|
|
@ -298,9 +278,9 @@ public class FlowTest extends StreamTest {
|
|||
.grouped(10)
|
||||
.concatSubstreams();
|
||||
|
||||
final Future<List<List<String>>> future =
|
||||
final CompletionStage<List<List<String>>> future =
|
||||
Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
|
||||
final List<List<String>> result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final List<List<String>> result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals(Arrays.asList(Arrays.asList("A", "B", "C", "."), Arrays.asList("D", "."), Arrays.asList("E", "F")), result);
|
||||
}
|
||||
|
|
@ -352,9 +332,9 @@ public class FlowTest extends StreamTest {
|
|||
|
||||
// collecting
|
||||
final Publisher<String> pub = source.runWith(publisher, materializer);
|
||||
final Future<List<String>> all = Source.fromPublisher(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
|
||||
final CompletionStage<List<String>> all = Source.fromPublisher(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
|
||||
|
||||
final List<String> result = Await.result(all, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
final List<String> result = all.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
assertEquals(new HashSet<Object>(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet<String>(result));
|
||||
}
|
||||
|
||||
|
|
@ -396,9 +376,9 @@ public class FlowTest extends StreamTest {
|
|||
final Iterable<String> input1 = Arrays.asList("A", "B", "C");
|
||||
final Iterable<String> input2 = Arrays.asList("D", "E", "F");
|
||||
|
||||
final Source<String, ?> in1 = Source.from(input1);
|
||||
final Source<String, ?> in2 = Source.from(input2);
|
||||
final Flow<String, String, ?> flow = Flow.of(String.class);
|
||||
final Source<String, NotUsed> in1 = Source.from(input1);
|
||||
final Source<String, NotUsed> in2 = Source.from(input2);
|
||||
final Flow<String, String, NotUsed> flow = Flow.of(String.class);
|
||||
in1.via(flow.concat(in2)).runForeach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
|
|
@ -415,9 +395,9 @@ public class FlowTest extends StreamTest {
|
|||
final Iterable<String> input1 = Arrays.asList("A", "B", "C");
|
||||
final Iterable<String> input2 = Arrays.asList("D", "E", "F");
|
||||
|
||||
final Source<String, ?> in1 = Source.from(input1);
|
||||
final Source<String, ?> in2 = Source.from(input2);
|
||||
final Flow<String, String, ?> flow = Flow.of(String.class);
|
||||
final Source<String, NotUsed> in1 = Source.from(input1);
|
||||
final Source<String, NotUsed> in2 = Source.from(input2);
|
||||
final Flow<String, String, NotUsed> flow = Flow.of(String.class);
|
||||
in2.via(flow.prepend(in1)).runForeach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
|
|
@ -432,15 +412,14 @@ public class FlowTest extends StreamTest {
|
|||
public void mustBeAbleToUsePrefixAndTail() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<Integer> input = Arrays.asList(1, 2, 3, 4, 5, 6);
|
||||
final Flow<Integer, Pair<List<Integer>, Source<Integer, NotUsed>>, ?> flow = Flow.of(Integer.class).prefixAndTail(3);
|
||||
Future<Pair<List<Integer>, Source<Integer, NotUsed>>> future =
|
||||
final Flow<Integer, Pair<List<Integer>, Source<Integer, NotUsed>>, NotUsed> flow = Flow.of(Integer.class).prefixAndTail(3);
|
||||
CompletionStage<Pair<List<Integer>, Source<Integer, NotUsed>>> future =
|
||||
Source.from(input).via(flow).runWith(Sink.<Pair<List<Integer>, Source<Integer, NotUsed>>>head(), materializer);
|
||||
Pair<List<Integer>, Source<Integer, NotUsed>> result = Await.result(future,
|
||||
probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
Pair<List<Integer>, Source<Integer, NotUsed>> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(Arrays.asList(1, 2, 3), result.first());
|
||||
|
||||
Future<List<Integer>> tailFuture = result.second().grouped(4).runWith(Sink.<List<Integer>>head(), materializer);
|
||||
List<Integer> tailResult = Await.result(tailFuture, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<List<Integer>> tailFuture = result.second().grouped(4).runWith(Sink.<List<Integer>>head(), materializer);
|
||||
List<Integer> tailResult = tailFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(Arrays.asList(4, 5, 6), tailResult);
|
||||
}
|
||||
|
||||
|
|
@ -454,12 +433,12 @@ public class FlowTest extends StreamTest {
|
|||
mainInputs.add(Source.from(input1));
|
||||
mainInputs.add(Source.from(input2));
|
||||
|
||||
final Flow<Source<Integer, NotUsed>, List<Integer>, ?> flow = Flow.<Source<Integer, NotUsed>>create().
|
||||
final Flow<Source<Integer, NotUsed>, List<Integer>, NotUsed> flow = Flow.<Source<Integer, NotUsed>>create().
|
||||
flatMapConcat(ConstantFun.<Source<Integer, NotUsed>>javaIdentityFunction()).grouped(6);
|
||||
Future<List<Integer>> future = Source.from(mainInputs).via(flow)
|
||||
CompletionStage<List<Integer>> future = Source.from(mainInputs).via(flow)
|
||||
.runWith(Sink.<List<Integer>>head(), materializer);
|
||||
|
||||
List<Integer> result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
List<Integer> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals(Arrays.asList(1, 2, 3, 4, 5), result);
|
||||
}
|
||||
|
|
@ -478,12 +457,12 @@ public class FlowTest extends StreamTest {
|
|||
mainInputs.add(Source.from(input3));
|
||||
mainInputs.add(Source.from(input4));
|
||||
|
||||
final Flow<Source<Integer, NotUsed>, List<Integer>, ?> flow = Flow.<Source<Integer, NotUsed>>create().
|
||||
final Flow<Source<Integer, NotUsed>, List<Integer>, NotUsed> flow = Flow.<Source<Integer, NotUsed>>create().
|
||||
flatMapMerge(3, ConstantFun.<Source<Integer, NotUsed>>javaIdentityFunction()).grouped(60);
|
||||
Future<List<Integer>> future = Source.from(mainInputs).via(flow)
|
||||
CompletionStage<List<Integer>> future = Source.from(mainInputs).via(flow)
|
||||
.runWith(Sink.<List<Integer>>head(), materializer);
|
||||
|
||||
List<Integer> result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
List<Integer> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
final Set<Integer> set = new HashSet<Integer>();
|
||||
for (Integer i: result) {
|
||||
set.add(i);
|
||||
|
|
@ -501,10 +480,10 @@ public class FlowTest extends StreamTest {
|
|||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final List<String> input = Arrays.asList("A", "B", "C");
|
||||
final Flow<String, List<String>, NotUsed> flow = Flow.of(String.class).buffer(2, OverflowStrategy.backpressure()).grouped(4);
|
||||
Future<List<String>> future = Source.from(input).via(flow)
|
||||
final CompletionStage<List<String>> future = Source.from(input).via(flow)
|
||||
.runWith(Sink.<List<String>>head(), materializer);
|
||||
|
||||
List<String> result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
List<String> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(input, result);
|
||||
}
|
||||
|
||||
|
|
@ -523,13 +502,8 @@ public class FlowTest extends StreamTest {
|
|||
return aggr + in;
|
||||
}
|
||||
});
|
||||
Future <String> future = Source.from(input).via(flow).runFold("", new Function2<String, String, String>() {
|
||||
@Override
|
||||
public String apply(String aggr, String in) throws Exception {
|
||||
return aggr + in;
|
||||
}
|
||||
}, materializer);
|
||||
String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<String> future = Source.from(input).via(flow).runFold("", (aggr, in) -> aggr + in, materializer);
|
||||
String result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals("ABC", result);
|
||||
}
|
||||
|
||||
|
|
@ -548,13 +522,8 @@ public class FlowTest extends StreamTest {
|
|||
return aggr + in;
|
||||
}
|
||||
});
|
||||
Future <String> future = Source.from(input).via(flow).runFold("", new Function2<String, String, String>() {
|
||||
@Override
|
||||
public String apply(String aggr, String in) throws Exception {
|
||||
return aggr + in;
|
||||
}
|
||||
}, materializer);
|
||||
String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<String> future = Source.from(input).via(flow).runFold("", (aggr, in) -> aggr + in, materializer);
|
||||
String result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals("ABC", result);
|
||||
}
|
||||
|
||||
|
|
@ -578,13 +547,8 @@ public class FlowTest extends StreamTest {
|
|||
return aggr + in;
|
||||
}
|
||||
});
|
||||
Future <String> future = Source.from(input).via(flow).runFold("", new Function2<String, String, String>() {
|
||||
@Override
|
||||
public String apply(String aggr, String in) throws Exception {
|
||||
return aggr + in;
|
||||
}
|
||||
}, materializer);
|
||||
String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<String> future = Source.from(input).via(flow).runFold("", (aggr, in) -> aggr + in, materializer);
|
||||
String result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals("ABC", result);
|
||||
}
|
||||
|
||||
|
|
@ -593,9 +557,9 @@ public class FlowTest extends StreamTest {
|
|||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final List<String> input = Arrays.asList("A", "B", "C");
|
||||
final Flow<String, String, NotUsed> flow = Flow.of(String.class).expand(in -> Stream.iterate(in, i -> i).iterator());
|
||||
final Sink<String, Future<String>> sink = Sink.<String>head();
|
||||
Future<String> future = Source.from(input).via(flow).runWith(sink, materializer);
|
||||
String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
final Sink<String, CompletionStage<String>> sink = Sink.<String>head();
|
||||
CompletionStage<String> future = Source.from(input).via(flow).runWith(sink, materializer);
|
||||
String result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals("A", result);
|
||||
}
|
||||
|
||||
|
|
@ -603,11 +567,7 @@ public class FlowTest extends StreamTest {
|
|||
public void mustBeAbleToUseMapAsync() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<String> input = Arrays.asList("a", "b", "c");
|
||||
final Flow<String, String, NotUsed> flow = Flow.of(String.class).mapAsync(4, new Function<String, Future<String>>() {
|
||||
public Future<String> apply(String elem) {
|
||||
return Futures.successful(elem.toUpperCase());
|
||||
}
|
||||
});
|
||||
final Flow<String, String, NotUsed> flow = Flow.of(String.class).mapAsync(4, elem -> CompletableFuture.completedFuture(elem.toUpperCase()));
|
||||
Source.from(input).via(flow).runForeach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
|
|
@ -623,8 +583,8 @@ public class FlowTest extends StreamTest {
|
|||
final TestPublisher.ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system);
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
|
||||
final Source<Integer, ?> source = Source.fromPublisher(publisherProbe);
|
||||
final Flow<Integer, Integer, ?> flow = Flow.of(Integer.class).map(
|
||||
final Source<Integer, NotUsed> source = Source.fromPublisher(publisherProbe);
|
||||
final Flow<Integer, Integer, NotUsed> flow = Flow.of(Integer.class).map(
|
||||
new Function<Integer, Integer>() {
|
||||
public Integer apply(Integer elem) {
|
||||
if (elem == 2) throw new RuntimeException("ex");
|
||||
|
|
@ -638,11 +598,8 @@ public class FlowTest extends StreamTest {
|
|||
}
|
||||
});
|
||||
|
||||
final Future<Done> future = source.via(flow).runWith(Sink.foreach(new Procedure<Integer>() {
|
||||
public void apply(Integer elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future =
|
||||
source.via(flow).runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
|
||||
final PublisherProbeSubscription<Integer> s = publisherProbe.expectSubscription();
|
||||
|
||||
|
|
@ -652,7 +609,7 @@ public class FlowTest extends StreamTest {
|
|||
probe.expectMsgEquals(1);
|
||||
s.sendNext(2);
|
||||
probe.expectMsgEquals(0);
|
||||
Await.ready(future, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -660,9 +617,9 @@ public class FlowTest extends StreamTest {
|
|||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final List<String> input = Arrays.asList("A", "B", "C");
|
||||
|
||||
Flow<String,String,?> otherFlow = Flow.of(String.class);
|
||||
Flow<String,String,NotUsed> otherFlow = Flow.of(String.class);
|
||||
|
||||
Flow<String,String,?> myFlow = Flow.of(String.class).via(otherFlow);
|
||||
Flow<String,String,NotUsed> myFlow = Flow.of(String.class).via(otherFlow);
|
||||
Source.from(input).via(myFlow).runWith(Sink.foreach(new Procedure<String>() { // Scala Future
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
|
|
@ -676,7 +633,7 @@ public class FlowTest extends StreamTest {
|
|||
public void mustBeAbleToMaterializeIdentityToJavaSink() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final List<String> input = Arrays.asList("A", "B", "C");
|
||||
Flow<String,String,?> otherFlow = Flow.of(String.class);
|
||||
Flow<String,String,NotUsed> otherFlow = Flow.of(String.class);
|
||||
|
||||
Sink<String,NotUsed> sink = Flow.of(String.class).to(otherFlow.to(Sink.foreach(new Procedure<String>() { // Scala Future
|
||||
public void apply(String elem) {
|
||||
|
|
@ -767,54 +724,54 @@ public class FlowTest extends StreamTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseInitialTimeout() throws Exception {
|
||||
public void mustBeAbleToUseInitialTimeout() throws Throwable {
|
||||
try {
|
||||
Await.result(
|
||||
Source.<Integer>maybe()
|
||||
.via(Flow.of(Integer.class).initialTimeout(Duration.create(1, "second")))
|
||||
.runWith(Sink.<Integer>head(), materializer),
|
||||
Duration.create(3, "second")
|
||||
);
|
||||
try {
|
||||
Source.<Integer> maybe().via(Flow.of(Integer.class).initialTimeout(Duration.create(1, "second")))
|
||||
.runWith(Sink.<Integer> head(), materializer).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
fail("A TimeoutException was expected");
|
||||
} catch(TimeoutException e) {
|
||||
} catch (ExecutionException e) {
|
||||
throw e.getCause();
|
||||
}
|
||||
} catch (TimeoutException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseCompletionTimeout() throws Exception {
|
||||
public void mustBeAbleToUseCompletionTimeout() throws Throwable {
|
||||
try {
|
||||
Await.result(
|
||||
Source.<Integer>maybe()
|
||||
.via(Flow.of(Integer.class).completionTimeout(Duration.create(1, "second")))
|
||||
.runWith(Sink.<Integer>head(), materializer),
|
||||
Duration.create(3, "second")
|
||||
);
|
||||
try {
|
||||
Source.<Integer> maybe().via(Flow.of(Integer.class).completionTimeout(Duration.create(1, "second")))
|
||||
.runWith(Sink.<Integer> head(), materializer).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
fail("A TimeoutException was expected");
|
||||
} catch(TimeoutException e) {
|
||||
} catch (ExecutionException e) {
|
||||
throw e.getCause();
|
||||
}
|
||||
} catch (TimeoutException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseIdleTimeout() throws Exception {
|
||||
public void mustBeAbleToUseIdleTimeout() throws Throwable {
|
||||
try {
|
||||
Await.result(
|
||||
Source.<Integer>maybe()
|
||||
.via(Flow.of(Integer.class).idleTimeout(Duration.create(1, "second")))
|
||||
.runWith(Sink.<Integer>head(), materializer),
|
||||
Duration.create(3, "second")
|
||||
);
|
||||
try {
|
||||
Source.<Integer> maybe().via(Flow.of(Integer.class).idleTimeout(Duration.create(1, "second")))
|
||||
.runWith(Sink.<Integer> head(), materializer).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
fail("A TimeoutException was expected");
|
||||
} catch(TimeoutException e) {
|
||||
} catch (ExecutionException e) {
|
||||
throw e.getCause();
|
||||
}
|
||||
} catch (TimeoutException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseKeepAlive() throws Exception {
|
||||
Integer result = Await.result(
|
||||
Integer result =
|
||||
Source.<Integer>maybe()
|
||||
.via(Flow.of(Integer.class)
|
||||
.keepAlive(Duration.create(1, "second"), new Creator<Integer>() {
|
||||
|
|
@ -824,9 +781,8 @@ public class FlowTest extends StreamTest {
|
|||
})
|
||||
)
|
||||
.takeWithin(Duration.create(1500, "milliseconds"))
|
||||
.runWith(Sink.<Integer>head(), materializer),
|
||||
Duration.create(3, "second")
|
||||
);
|
||||
.runWith(Sink.<Integer>head(), materializer)
|
||||
.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals((Object) 0, result);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,6 +8,8 @@ import java.util.Arrays;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.NotUsed;
|
||||
import akka.japi.function.Function;
|
||||
|
|
@ -40,21 +42,17 @@ public class SinkTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustBeAbleToUseFuture() throws Exception {
|
||||
final Sink<Integer, Future<Integer>> futSink = Sink.head();
|
||||
final Sink<Integer, CompletionStage<Integer>> futSink = Sink.head();
|
||||
final List<Integer> list = Collections.singletonList(1);
|
||||
final Future<Integer> future = Source.from(list).runWith(futSink, materializer);
|
||||
assert Await.result(future, Duration.create("1 second")).equals(1);
|
||||
final CompletionStage<Integer> future = Source.from(list).runWith(futSink, materializer);
|
||||
assert future.toCompletableFuture().get(1, TimeUnit.SECONDS).equals(1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseFold() throws Exception {
|
||||
Sink<Integer, Future<Integer>> foldSink = Sink.fold(0, new Function2<Integer, Integer, Integer>() {
|
||||
@Override public Integer apply(Integer arg1, Integer arg2) throws Exception {
|
||||
return arg1 + arg2;
|
||||
}
|
||||
});
|
||||
Sink<Integer, CompletionStage<Integer>> foldSink = Sink.fold(0, (arg1, arg2) -> arg1 + arg2);
|
||||
@SuppressWarnings("unused")
|
||||
Future<Integer> integerFuture = Source.from(new ArrayList<Integer>()).runWith(foldSink, materializer);
|
||||
CompletionStage<Integer> integerFuture = Source.from(new ArrayList<Integer>()).runWith(foldSink, materializer);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -97,7 +95,7 @@ public class SinkTest extends StreamTest {
|
|||
|
||||
public void mustSuitablyOverrideAttributeHandlingMethods() {
|
||||
@SuppressWarnings("unused")
|
||||
final Sink<Integer, Future<Integer>> s =
|
||||
final Sink<Integer, CompletionStage<Integer>> s =
|
||||
Sink.<Integer> head().withAttributes(Attributes.name("")).addAttributes(Attributes.asyncBoundary()).named("");
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -13,6 +13,7 @@ import akka.dispatch.OnSuccess;
|
|||
import akka.japi.JavaPartialFunction;
|
||||
import akka.japi.Pair;
|
||||
import akka.japi.function.*;
|
||||
import akka.japi.pf.PFBuilder;
|
||||
import akka.stream.*;
|
||||
import akka.stream.impl.ConstantFun;
|
||||
import akka.stream.stage.*;
|
||||
|
|
@ -29,6 +30,9 @@ import scala.concurrent.duration.FiniteDuration;
|
|||
import scala.util.Try;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.stream.Stream;
|
||||
|
|
@ -54,32 +58,18 @@ public class SourceTest extends StreamTest {
|
|||
final java.lang.Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5);
|
||||
final Source<Integer, NotUsed> ints = Source.from(input);
|
||||
|
||||
ints.drop(2).take(3).takeWithin(FiniteDuration.create(10, TimeUnit.SECONDS)).map(new Function<Integer, String>() {
|
||||
public String apply(Integer elem) {
|
||||
return lookup[elem];
|
||||
}
|
||||
}).filter(new Predicate<String>() {
|
||||
public boolean test(String elem) {
|
||||
return !elem.equals("c");
|
||||
}
|
||||
}).grouped(2).mapConcat(new Function<java.util.List<String>, java.util.List<String>>() {
|
||||
public java.util.List<String> apply(java.util.List<String> elem) {
|
||||
return elem;
|
||||
}
|
||||
}).groupedWithin(100, FiniteDuration.create(50, TimeUnit.MILLISECONDS))
|
||||
.mapConcat(new Function<java.util.List<String>, java.util.List<String>>() {
|
||||
public java.util.List<String> apply(java.util.List<String> elem) {
|
||||
return elem;
|
||||
}
|
||||
}).runFold("", new Function2<String, String, String>() {
|
||||
public String apply(String acc, String elem) {
|
||||
return acc + elem;
|
||||
}
|
||||
}, materializer).foreach(new Foreach<String>() { // Scala Future
|
||||
public void each(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}, system.dispatcher());
|
||||
ints
|
||||
.drop(2)
|
||||
.take(3)
|
||||
.takeWithin(FiniteDuration.create(10, TimeUnit.SECONDS))
|
||||
.map(elem -> lookup[elem])
|
||||
.filter(elem -> !elem.equals("c"))
|
||||
.grouped(2)
|
||||
.mapConcat(elem -> elem)
|
||||
.groupedWithin(100, FiniteDuration.create(50, TimeUnit.MILLISECONDS))
|
||||
.mapConcat(elem -> elem)
|
||||
.runFold("", (acc, elem) -> acc + elem, materializer)
|
||||
.thenAccept(elem -> probe.getRef().tell(elem, ActorRef.noSender()));
|
||||
|
||||
probe.expectMsgEquals("de");
|
||||
}
|
||||
|
|
@ -88,20 +78,11 @@ public class SourceTest extends StreamTest {
|
|||
public void mustBeAbleToUseVoidTypeInForeach() {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final java.lang.Iterable<String> input = Arrays.asList("a", "b", "c");
|
||||
Source<String, ?> ints = Source.from(input);
|
||||
Source<String, NotUsed> ints = Source.from(input);
|
||||
|
||||
Future<Done> completion = ints.runForeach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}, materializer);
|
||||
final CompletionStage<Done> completion = ints.runForeach(elem -> probe.getRef().tell(elem, ActorRef.noSender()), materializer);
|
||||
|
||||
completion.onSuccess(new OnSuccess<Done>() {
|
||||
@Override
|
||||
public void onSuccess(Done elem) throws Throwable {
|
||||
probe.getRef().tell(String.valueOf(elem), ActorRef.noSender());
|
||||
}
|
||||
}, system.dispatcher());
|
||||
completion.thenAccept(elem -> probe.getRef().tell(String.valueOf(elem), ActorRef.noSender()));
|
||||
|
||||
probe.expectMsgEquals("a");
|
||||
probe.expectMsgEquals("b");
|
||||
|
|
@ -176,9 +157,9 @@ public class SourceTest extends StreamTest {
|
|||
.grouped(10)
|
||||
.mergeSubstreams();
|
||||
|
||||
final Future<List<List<String>>> future =
|
||||
final CompletionStage<List<List<String>>> future =
|
||||
source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
|
||||
final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray();
|
||||
final Object[] result = future.toCompletableFuture().get(1, TimeUnit.SECONDS).toArray();
|
||||
Arrays.sort(result, (Comparator<Object>)(Object) new Comparator<List<String>>() {
|
||||
@Override
|
||||
public int compare(List<String> o1, List<String> o2) {
|
||||
|
|
@ -202,9 +183,9 @@ public class SourceTest extends StreamTest {
|
|||
.grouped(10)
|
||||
.concatSubstreams();
|
||||
|
||||
final Future<List<List<String>>> future =
|
||||
final CompletionStage<List<List<String>>> future =
|
||||
source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
|
||||
final List<List<String>> result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final List<List<String>> result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals(Arrays.asList(Arrays.asList("A", "B", "C"), Arrays.asList(".", "D"), Arrays.asList(".", "E", "F")), result);
|
||||
}
|
||||
|
|
@ -222,9 +203,9 @@ public class SourceTest extends StreamTest {
|
|||
.grouped(10)
|
||||
.concatSubstreams();
|
||||
|
||||
final Future<List<List<String>>> future =
|
||||
final CompletionStage<List<List<String>>> future =
|
||||
source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
|
||||
final List<List<String>> result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
|
||||
final List<List<String>> result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals(Arrays.asList(Arrays.asList("A", "B", "C", "."), Arrays.asList("D", "."), Arrays.asList("E", "F")), result);
|
||||
}
|
||||
|
|
@ -235,8 +216,8 @@ public class SourceTest extends StreamTest {
|
|||
final Iterable<String> input1 = Arrays.asList("A", "B", "C");
|
||||
final Iterable<String> input2 = Arrays.asList("D", "E", "F");
|
||||
|
||||
final Source<String, ?> in1 = Source.from(input1);
|
||||
final Source<String, ?> in2 = Source.from(input2);
|
||||
final Source<String, NotUsed> in1 = Source.from(input1);
|
||||
final Source<String, NotUsed> in2 = Source.from(input2);
|
||||
|
||||
in1.concat(in2).runForeach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
|
|
@ -254,8 +235,8 @@ public class SourceTest extends StreamTest {
|
|||
final Iterable<String> input1 = Arrays.asList("A", "B", "C");
|
||||
final Iterable<String> input2 = Arrays.asList("D", "E", "F");
|
||||
|
||||
final Source<String, ?> in1 = Source.from(input1);
|
||||
final Source<String, ?> in2 = Source.from(input2);
|
||||
final Source<String, NotUsed> in1 = Source.from(input1);
|
||||
final Source<String, NotUsed> in2 = Source.from(input2);
|
||||
|
||||
in2.prepend(in1).runForeach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
|
|
@ -308,20 +289,16 @@ public class SourceTest extends StreamTest {
|
|||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<String> input = Arrays.asList("A", "B", "C");
|
||||
|
||||
Source.from(input).map(new Function<String, String>() {
|
||||
public String apply(String arg0) throws Exception {
|
||||
throw new RuntimeException("simulated err");
|
||||
}
|
||||
}).runWith(Sink.<String>head(), materializer).onComplete(new OnSuccess<Try<String>>() {
|
||||
@Override
|
||||
public void onSuccess(Try<String> e) throws Throwable {
|
||||
if (e == null) {
|
||||
Source.from(input)
|
||||
.<String> map(in -> { throw new RuntimeException("simulated err"); })
|
||||
.runWith(Sink.<String>head(), materializer)
|
||||
.whenComplete((s, ex) -> {
|
||||
if (ex == null) {
|
||||
probe.getRef().tell("done", ActorRef.noSender());
|
||||
} else {
|
||||
probe.getRef().tell(e.failed().get().getMessage(), ActorRef.noSender());
|
||||
probe.getRef().tell(ex.getMessage(), ActorRef.noSender());
|
||||
}
|
||||
}
|
||||
}, system.dispatcher());
|
||||
});
|
||||
|
||||
probe.expectMsgEquals("simulated err");
|
||||
}
|
||||
|
|
@ -330,8 +307,8 @@ public class SourceTest extends StreamTest {
|
|||
public void mustBeAbleToUseToFuture() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<String> input = Arrays.asList("A", "B", "C");
|
||||
Future<String> future = Source.from(input).runWith(Sink.<String>head(), materializer);
|
||||
String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<String> future = Source.from(input).runWith(Sink.<String>head(), materializer);
|
||||
String result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals("A", result);
|
||||
}
|
||||
|
||||
|
|
@ -339,14 +316,13 @@ public class SourceTest extends StreamTest {
|
|||
public void mustBeAbleToUsePrefixAndTail() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<Integer> input = Arrays.asList(1, 2, 3, 4, 5, 6);
|
||||
Future<Pair<List<Integer>, Source<Integer, NotUsed>>> future = Source.from(input).prefixAndTail(3)
|
||||
CompletionStage<Pair<List<Integer>, Source<Integer, NotUsed>>> future = Source.from(input).prefixAndTail(3)
|
||||
.runWith(Sink.<Pair<List<Integer>, Source<Integer, NotUsed>>>head(), materializer);
|
||||
Pair<List<Integer>, Source<Integer, NotUsed>> result = Await.result(future,
|
||||
probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
Pair<List<Integer>, Source<Integer, NotUsed>> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(Arrays.asList(1, 2, 3), result.first());
|
||||
|
||||
Future<List<Integer>> tailFuture = result.second().grouped(4).runWith(Sink.<List<Integer>>head(), materializer);
|
||||
List<Integer> tailResult = Await.result(tailFuture, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<List<Integer>> tailFuture = result.second().grouped(4).runWith(Sink.<List<Integer>>head(), materializer);
|
||||
List<Integer> tailResult = tailFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(Arrays.asList(4, 5, 6), tailResult);
|
||||
}
|
||||
|
||||
|
|
@ -360,12 +336,12 @@ public class SourceTest extends StreamTest {
|
|||
mainInputs.add(Source.from(input1));
|
||||
mainInputs.add(Source.from(input2));
|
||||
|
||||
Future<List<Integer>> future = Source.from(mainInputs)
|
||||
CompletionStage<List<Integer>> future = Source.from(mainInputs)
|
||||
.<Integer, NotUsed>flatMapConcat(ConstantFun.<Source<Integer,NotUsed>>javaIdentityFunction())
|
||||
.grouped(6)
|
||||
.runWith(Sink.<List<Integer>>head(), materializer);
|
||||
|
||||
List<Integer> result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
List<Integer> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals(Arrays.asList(1, 2, 3, 4, 5), result);
|
||||
}
|
||||
|
|
@ -384,11 +360,11 @@ public class SourceTest extends StreamTest {
|
|||
mainInputs.add(Source.from(input3));
|
||||
mainInputs.add(Source.from(input4));
|
||||
|
||||
Future<List<Integer>> future = Source.from(mainInputs)
|
||||
CompletionStage<List<Integer>> future = Source.from(mainInputs)
|
||||
.flatMapMerge(3, ConstantFun.<Source<Integer, NotUsed>>javaIdentityFunction()).grouped(60)
|
||||
.runWith(Sink.<List<Integer>>head(), materializer);
|
||||
|
||||
List<Integer> result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
List<Integer> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
final Set<Integer> set = new HashSet<Integer>();
|
||||
for (Integer i: result) {
|
||||
set.add(i);
|
||||
|
|
@ -405,10 +381,10 @@ public class SourceTest extends StreamTest {
|
|||
public void mustBeAbleToUseBuffer() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final List<String> input = Arrays.asList("A", "B", "C");
|
||||
Future<List<String>> future = Source.from(input).buffer(2, OverflowStrategy.backpressure()).grouped(4)
|
||||
final CompletionStage<List<String>> future = Source.from(input).buffer(2, OverflowStrategy.backpressure()).grouped(4)
|
||||
.runWith(Sink.<List<String>>head(), materializer);
|
||||
|
||||
List<String> result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
List<String> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(input, result);
|
||||
}
|
||||
|
||||
|
|
@ -416,23 +392,10 @@ public class SourceTest extends StreamTest {
|
|||
public void mustBeAbleToUseConflate() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final List<String> input = Arrays.asList("A", "B", "C");
|
||||
Future<String> future = Source.from(input).conflate(new Function<String, String>() {
|
||||
@Override
|
||||
public String apply(String s) throws Exception {
|
||||
return s;
|
||||
}
|
||||
}, new Function2<String, String, String>() {
|
||||
@Override
|
||||
public String apply(String aggr, String in) throws Exception {
|
||||
return aggr + in;
|
||||
}
|
||||
}).runFold("", new Function2<String, String, String>() {
|
||||
@Override
|
||||
public String apply(String aggr, String in) throws Exception {
|
||||
return aggr + in;
|
||||
}
|
||||
}, materializer);
|
||||
String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<String> future = Source.from(input)
|
||||
.conflate(s -> s, (aggr, in) -> aggr + in)
|
||||
.runFold("", (aggr, in) -> aggr + in, materializer);
|
||||
String result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals("ABC", result);
|
||||
}
|
||||
|
||||
|
|
@ -440,8 +403,8 @@ public class SourceTest extends StreamTest {
|
|||
public void mustBeAbleToUseExpand() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final List<String> input = Arrays.asList("A", "B", "C");
|
||||
Future<String> future = Source.from(input).expand(in -> Stream.iterate(in, i -> i).iterator()).runWith(Sink.<String>head(), materializer);
|
||||
String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<String> future = Source.from(input).expand(in -> Stream.iterate(in, i -> i).iterator()).runWith(Sink.<String>head(), materializer);
|
||||
String result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals("A", result);
|
||||
}
|
||||
|
||||
|
|
@ -468,15 +431,9 @@ public class SourceTest extends StreamTest {
|
|||
public void mustBeAbleToUseMapFuture() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<String> input = Arrays.asList("a", "b", "c");
|
||||
Source.from(input).mapAsync(4, new Function<String, Future<String>>() {
|
||||
public Future<String> apply(String elem) {
|
||||
return Futures.successful(elem.toUpperCase());
|
||||
}
|
||||
}).runForeach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}, materializer);
|
||||
Source.from(input)
|
||||
.mapAsync(4, elem -> CompletableFuture.completedFuture(elem.toUpperCase()))
|
||||
.runForeach(elem -> probe.getRef().tell(elem, ActorRef.noSender()), materializer);
|
||||
probe.expectMsgEquals("A");
|
||||
probe.expectMsgEquals("B");
|
||||
probe.expectMsgEquals("C");
|
||||
|
|
@ -486,16 +443,16 @@ public class SourceTest extends StreamTest {
|
|||
public void mustWorkFromFuture() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<String> input = Arrays.asList("A", "B", "C");
|
||||
Future<String> future1 = Source.from(input).runWith(Sink.<String>head(), materializer);
|
||||
Future<String> future2 = Source.fromFuture(future1).runWith(Sink.<String>head(), materializer);
|
||||
String result = Await.result(future2, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
CompletionStage<String> future1 = Source.from(input).runWith(Sink.<String>head(), materializer);
|
||||
CompletionStage<String> future2 = Source.fromCompletionStage(future1).runWith(Sink.<String>head(), materializer);
|
||||
String result = future2.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals("A", result);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustWorkFromRange() throws Exception {
|
||||
Future<List<Integer>> f = Source.range(0, 10).grouped(20).runWith(Sink.<List<Integer>> head(), materializer);
|
||||
final List<Integer> result = Await.result(f, FiniteDuration.create(3, TimeUnit.SECONDS));
|
||||
CompletionStage<List<Integer>> f = Source.range(0, 10).grouped(20).runWith(Sink.<List<Integer>> head(), materializer);
|
||||
final List<Integer> result = f.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(11, result.size());
|
||||
Integer counter = 0;
|
||||
for (Integer i: result)
|
||||
|
|
@ -504,8 +461,8 @@ public class SourceTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustWorkFromRangeWithStep() throws Exception {
|
||||
Future<List<Integer>> f = Source.range(0, 10, 2).grouped(20).runWith(Sink.<List<Integer>> head(), materializer);
|
||||
final List<Integer> result = Await.result(f, FiniteDuration.create(3, TimeUnit.SECONDS));
|
||||
CompletionStage<List<Integer>> f = Source.range(0, 10, 2).grouped(20).runWith(Sink.<List<Integer>> head(), materializer);
|
||||
final List<Integer> result = f.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(6, result.size());
|
||||
Integer counter = 0;
|
||||
for (Integer i: result) {
|
||||
|
|
@ -516,8 +473,8 @@ public class SourceTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustRepeat() throws Exception {
|
||||
final Future<List<Integer>> f = Source.repeat(42).grouped(10000).runWith(Sink.<List<Integer>> head(), materializer);
|
||||
final List<Integer> result = Await.result(f, FiniteDuration.create(3, TimeUnit.SECONDS));
|
||||
final CompletionStage<List<Integer>> f = Source.repeat(42).grouped(10000).runWith(Sink.<List<Integer>> head(), materializer);
|
||||
final List<Integer> result = f.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(result.size(), 10000);
|
||||
for (Integer i: result) assertEquals(i, (Integer) 42);
|
||||
}
|
||||
|
|
@ -540,39 +497,31 @@ public class SourceTest extends StreamTest {
|
|||
@Test
|
||||
public void mustBeAbleToUseDropWhile() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Source<Integer, ?> source = Source.from(Arrays.asList(0, 1, 2, 3)).dropWhile
|
||||
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3)).dropWhile
|
||||
(new Predicate<Integer>() {
|
||||
public boolean test(Integer elem) {
|
||||
return elem < 2;
|
||||
}
|
||||
});
|
||||
|
||||
final Future<Done> future = source.runWith(Sink.foreach(new Procedure<Integer>() { // Scala Future
|
||||
public void apply(Integer elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future = source.runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
|
||||
probe.expectMsgEquals(2);
|
||||
probe.expectMsgEquals(3);
|
||||
Await.ready(future, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseTakeWhile() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Source<Integer, ?> source = Source.from(Arrays.asList(0, 1, 2, 3)).takeWhile
|
||||
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3)).takeWhile
|
||||
(new Predicate<Integer>() {
|
||||
public boolean test(Integer elem) {
|
||||
return elem < 2;
|
||||
}
|
||||
});
|
||||
|
||||
final Future<Done> future = source.runWith(Sink.foreach(new Procedure<Integer>() { // Scala Future
|
||||
public void apply(Integer elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future = source.runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
|
||||
probe.expectMsgEquals(0);
|
||||
probe.expectMsgEquals(1);
|
||||
|
|
@ -580,7 +529,7 @@ public class SourceTest extends StreamTest {
|
|||
FiniteDuration duration = Duration.apply(200, TimeUnit.MILLISECONDS);
|
||||
|
||||
probe.expectNoMsg(duration);
|
||||
Await.ready(future, duration);
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -588,56 +537,41 @@ public class SourceTest extends StreamTest {
|
|||
final ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system);
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
|
||||
final Source<Integer, ?> source = Source.fromPublisher(publisherProbe).map(
|
||||
new Function<Integer, Integer>() {
|
||||
public Integer apply(Integer elem) {
|
||||
final Source<Integer, NotUsed> source =
|
||||
Source.fromPublisher(publisherProbe)
|
||||
.map(elem -> {
|
||||
if (elem == 1) throw new RuntimeException("ex");
|
||||
else return elem;
|
||||
}
|
||||
})
|
||||
.recover(new JavaPartialFunction<Throwable, Integer>() {
|
||||
public Integer apply(Throwable elem, boolean isCheck) {
|
||||
if (isCheck) return null;
|
||||
return 0;
|
||||
}
|
||||
});
|
||||
.recover(new PFBuilder<Throwable, Integer>()
|
||||
.matchAny(ex -> 0)
|
||||
.build());
|
||||
|
||||
final Future<Done> future = source.runWith(Sink.foreach(new Procedure<Integer>() {
|
||||
public void apply(Integer elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future = source.runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
final PublisherProbeSubscription<Integer> s = publisherProbe.expectSubscription();
|
||||
s.sendNext(0);
|
||||
probe.expectMsgEquals(0);
|
||||
s.sendNext(1);
|
||||
probe.expectMsgEquals(0);
|
||||
|
||||
Await.ready(future, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToCombine() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Source<Integer, ?> source1 = Source.from(Arrays.asList(0, 1));
|
||||
final Source<Integer, ?> source2 = Source.from(Arrays.asList(2, 3));
|
||||
final Source<Integer, NotUsed> source1 = Source.from(Arrays.asList(0, 1));
|
||||
final Source<Integer, NotUsed> source2 = Source.from(Arrays.asList(2, 3));
|
||||
|
||||
final Source<Integer, ?> source = Source.combine(source1, source2, new ArrayList<Source<Integer, ?>>(),
|
||||
new Function<Integer, Graph<UniformFanInShape<Integer, Integer>, NotUsed>>() {
|
||||
public Graph<UniformFanInShape<Integer, Integer>, NotUsed> apply(Integer elem) {
|
||||
return Merge.create(elem);
|
||||
}
|
||||
});
|
||||
final Source<Integer, NotUsed> source = Source.combine(
|
||||
source1, source2, new ArrayList<Source<Integer, ?>>(),
|
||||
width -> Merge.<Integer> create(width));
|
||||
|
||||
final Future<Done> future = source.runWith(Sink.foreach(new Procedure<Integer>() { // Scala Future
|
||||
public void apply(Integer elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}), materializer);
|
||||
final CompletionStage<Done> future = source.runWith(Sink.foreach(elem -> probe.getRef().tell(elem, ActorRef.noSender())), materializer);
|
||||
|
||||
probe.expectMsgAllOf(0, 1, 2, 3);
|
||||
|
||||
Await.ready(future, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
future.toCompletableFuture().get(200, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -710,48 +644,53 @@ public class SourceTest extends StreamTest {
|
|||
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseInitialTimeout() throws Exception {
|
||||
public void mustBeAbleToUseInitialTimeout() throws Throwable {
|
||||
try {
|
||||
Await.result(
|
||||
Source.maybe().initialTimeout(Duration.create(1, "second")).runWith(Sink.head(), materializer),
|
||||
Duration.create(3, "second")
|
||||
);
|
||||
fail("A TimeoutException was expected");
|
||||
} catch(TimeoutException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseCompletionTimeout() throws Exception {
|
||||
try {
|
||||
Await.result(
|
||||
Source.maybe().completionTimeout(Duration.create(1, "second")).runWith(Sink.head(), materializer),
|
||||
Duration.create(3, "second")
|
||||
);
|
||||
Source.maybe().initialTimeout(Duration.create(1, "second")).runWith(Sink.head(), materializer)
|
||||
.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
fail("A TimeoutException was expected");
|
||||
} catch(TimeoutException e) {
|
||||
} catch (ExecutionException e) {
|
||||
throw e.getCause();
|
||||
}
|
||||
} catch (TimeoutException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseIdleTimeout() throws Exception {
|
||||
public void mustBeAbleToUseCompletionTimeout() throws Throwable {
|
||||
try {
|
||||
Await.result(
|
||||
Source.maybe().idleTimeout(Duration.create(1, "second")).runWith(Sink.head(), materializer),
|
||||
Duration.create(3, "second")
|
||||
);
|
||||
try {
|
||||
Source.maybe().completionTimeout(Duration.create(1, "second")).runWith(Sink.head(), materializer)
|
||||
.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
fail("A TimeoutException was expected");
|
||||
} catch(TimeoutException e) {
|
||||
} catch (ExecutionException e) {
|
||||
throw e.getCause();
|
||||
}
|
||||
} catch (TimeoutException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseIdleTimeout() throws Throwable {
|
||||
try {
|
||||
try {
|
||||
Source.maybe().idleTimeout(Duration.create(1, "second")).runWith(Sink.head(), materializer)
|
||||
.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
fail("A TimeoutException was expected");
|
||||
} catch (ExecutionException e) {
|
||||
throw e.getCause();
|
||||
}
|
||||
} catch (TimeoutException e) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseIdleInject() throws Exception {
|
||||
Integer result = Await.result(
|
||||
Integer result =
|
||||
Source.maybe()
|
||||
.keepAlive(Duration.create(1, "second"), new Creator<Integer>() {
|
||||
public Integer create() {
|
||||
|
|
@ -759,9 +698,8 @@ public class SourceTest extends StreamTest {
|
|||
}
|
||||
})
|
||||
.takeWithin(Duration.create(1500, "milliseconds"))
|
||||
.runWith(Sink.<Integer>head(), materializer),
|
||||
Duration.create(3, "second")
|
||||
);
|
||||
.runWith(Sink.<Integer>head(), materializer)
|
||||
.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals((Object) 0, result);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,6 +8,8 @@ import static org.junit.Assert.assertTrue;
|
|||
import java.net.InetSocketAddress;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.net.BindException;
|
||||
|
||||
|
|
@ -37,7 +39,7 @@ public class TcpTest extends StreamTest {
|
|||
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("TcpTest",
|
||||
AkkaSpec.testConf());
|
||||
|
||||
final Sink<IncomingConnection, Future<Done>> echoHandler =
|
||||
final Sink<IncomingConnection, CompletionStage<Done>> echoHandler =
|
||||
Sink.foreach(new Procedure<IncomingConnection>() {
|
||||
public void apply(IncomingConnection conn) {
|
||||
conn.handleWith(Flow.of(ByteString.class), materializer);
|
||||
|
|
@ -54,17 +56,16 @@ public class TcpTest extends StreamTest {
|
|||
@Test
|
||||
public void mustWorkInHappyCase() throws Exception {
|
||||
final InetSocketAddress serverAddress = TestUtils.temporaryServerAddress("127.0.0.1", false);
|
||||
final Source<IncomingConnection, Future<ServerBinding>> binding = Tcp.get(system)
|
||||
final Source<IncomingConnection, CompletionStage<ServerBinding>> binding = Tcp.get(system)
|
||||
.bind(serverAddress.getHostName(), serverAddress.getPort()); // TODO getHostString in Java7
|
||||
|
||||
final Future<ServerBinding> future = binding.to(echoHandler).run(materializer);
|
||||
final ServerBinding b = Await.result(future, FiniteDuration.create(5, TimeUnit.SECONDS));
|
||||
final CompletionStage<ServerBinding> future = binding.to(echoHandler).run(materializer);
|
||||
final ServerBinding b = future.toCompletableFuture().get(5, TimeUnit.SECONDS);
|
||||
assertEquals(b.localAddress().getPort(), serverAddress.getPort());
|
||||
|
||||
final Future<ByteString> resultFuture = Source
|
||||
final CompletionStage<ByteString> resultFuture = Source
|
||||
.from(testInput)
|
||||
// TODO getHostString in Java7
|
||||
.via(Tcp.get(system).outgoingConnection(serverAddress.getHostName(), serverAddress.getPort()))
|
||||
.via(Tcp.get(system).outgoingConnection(serverAddress.getHostString(), serverAddress.getPort()))
|
||||
.runFold(ByteString.empty(),
|
||||
new Function2<ByteString, ByteString, ByteString>() {
|
||||
public ByteString apply(ByteString acc, ByteString elem) {
|
||||
|
|
@ -72,7 +73,7 @@ public class TcpTest extends StreamTest {
|
|||
}
|
||||
}, materializer);
|
||||
|
||||
final byte[] result = Await.result(resultFuture, FiniteDuration.create(5, TimeUnit.SECONDS)).toArray();
|
||||
final byte[] result = resultFuture.toCompletableFuture().get(5, TimeUnit.SECONDS).toArray();
|
||||
for (int i = 0; i < testInput.size(); i ++) {
|
||||
assertEquals(testInput.get(i).head(), result[i]);
|
||||
}
|
||||
|
|
@ -81,11 +82,11 @@ public class TcpTest extends StreamTest {
|
|||
@Test
|
||||
public void mustReportServerBindFailure() throws Exception {
|
||||
final InetSocketAddress serverAddress = TestUtils.temporaryServerAddress("127.0.0.1", false);
|
||||
final Source<IncomingConnection, Future<ServerBinding>> binding = Tcp.get(system)
|
||||
final Source<IncomingConnection, CompletionStage<ServerBinding>> binding = Tcp.get(system)
|
||||
.bind(serverAddress.getHostName(), serverAddress.getPort()); // TODO getHostString in Java7
|
||||
|
||||
final Future<ServerBinding> future = binding.to(echoHandler).run(materializer);
|
||||
final ServerBinding b = Await.result(future, FiniteDuration.create(5, TimeUnit.SECONDS));
|
||||
final CompletionStage<ServerBinding> future = binding.to(echoHandler).run(materializer);
|
||||
final ServerBinding b = future.toCompletableFuture().get(5, TimeUnit.SECONDS);
|
||||
assertEquals(b.localAddress().getPort(), serverAddress.getPort());
|
||||
|
||||
new JavaTestKit(system) {{
|
||||
|
|
@ -93,9 +94,11 @@ public class TcpTest extends StreamTest {
|
|||
@Override
|
||||
protected Void run() {
|
||||
try {
|
||||
Await.result(binding.to(echoHandler).run(materializer), FiniteDuration.create(5, TimeUnit.SECONDS));
|
||||
binding.to(echoHandler).run(materializer).toCompletableFuture().get(5, TimeUnit.SECONDS);
|
||||
assertTrue("Expected BindFailedException, but nothing was reported", false);
|
||||
} catch (BindFailedException e) {
|
||||
} catch (ExecutionException e) {
|
||||
if (e.getCause() instanceof BindFailedException) {} // all good
|
||||
else throw new AssertionError("failed", e);
|
||||
// expected
|
||||
} catch (Exception e) {
|
||||
throw new AssertionError("failed", e);
|
||||
|
|
@ -107,19 +110,19 @@ public class TcpTest extends StreamTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void mustReportClientConnectFailure() throws Exception {
|
||||
public void mustReportClientConnectFailure() throws Throwable {
|
||||
final InetSocketAddress serverAddress = TestUtils.temporaryServerAddress(
|
||||
"127.0.0.1", false);
|
||||
try {
|
||||
Await.result(
|
||||
try {
|
||||
Source.from(testInput)
|
||||
// TODO getHostString in Java7
|
||||
.viaMat(Tcp.get(system).outgoingConnection(serverAddress.getHostName(), serverAddress.getPort()),
|
||||
Keep.<NotUsed, Future<OutgoingConnection>> right())
|
||||
.to(Sink.<ByteString> ignore())
|
||||
.run(materializer),
|
||||
FiniteDuration.create(5, TimeUnit.SECONDS));
|
||||
.viaMat(Tcp.get(system).outgoingConnection(serverAddress.getHostString(), serverAddress.getPort()),
|
||||
Keep.right())
|
||||
.to(Sink.<ByteString> ignore()).run(materializer).toCompletableFuture().get(5, TimeUnit.SECONDS);
|
||||
assertTrue("Expected StreamTcpException, but nothing was reported", false);
|
||||
} catch (ExecutionException e) {
|
||||
throw e.getCause();
|
||||
}
|
||||
} catch (StreamTcpException e) {
|
||||
// expected
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,6 +3,7 @@
|
|||
*/
|
||||
package akka.stream.stage;
|
||||
|
||||
import akka.NotUsed;
|
||||
import akka.stream.StreamTest;
|
||||
import akka.stream.javadsl.AkkaJUnitActorSystemResource;
|
||||
import akka.stream.javadsl.Sink;
|
||||
|
|
@ -19,6 +20,8 @@ import scala.concurrent.duration.Duration;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class StageTest extends StreamTest {
|
||||
public StageTest() {
|
||||
|
|
@ -32,16 +35,16 @@ public class StageTest extends StreamTest {
|
|||
@Test
|
||||
public void javaStageUsage() throws Exception {
|
||||
final java.lang.Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5);
|
||||
final Source<Integer, ?> ints = Source.from(input);
|
||||
final Source<Integer, NotUsed> ints = Source.from(input);
|
||||
final JavaIdentityStage<Integer> identity = new JavaIdentityStage<Integer>();
|
||||
|
||||
final Future<List<Integer>> result =
|
||||
final CompletionStage<List<Integer>> result =
|
||||
ints
|
||||
.via(identity)
|
||||
.via(identity)
|
||||
.grouped(1000)
|
||||
.runWith(Sink.<List<Integer>>head(), materializer);
|
||||
|
||||
assertEquals(Arrays.asList(0, 1, 2, 3, 4, 5), Await.result(result, Duration.create(3, "seconds")));
|
||||
assertEquals(Arrays.asList(0, 1, 2, 3, 4, 5), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,9 +11,9 @@ import akka.stream.stage.OutHandler
|
|||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit.{ AkkaSpec, TestSubscriber }
|
||||
import akka.testkit.TestProbe
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.{ Future, _ }
|
||||
import akka.Done
|
||||
|
||||
class QueueSourceSpec extends AkkaSpec {
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
|
@ -70,7 +70,7 @@ class QueueSourceSpec extends AkkaSpec {
|
|||
expectNoMsg(pause)
|
||||
|
||||
sub.cancel()
|
||||
expectMsg(())
|
||||
expectMsg(Done)
|
||||
}
|
||||
|
||||
"buffer when needed" in {
|
||||
|
|
@ -123,7 +123,7 @@ class QueueSourceSpec extends AkkaSpec {
|
|||
|
||||
sub.cancel()
|
||||
|
||||
expectMsgAllOf(QueueOfferResult.QueueClosed, ())
|
||||
expectMsgAllOf(QueueOfferResult.QueueClosed, Done)
|
||||
}
|
||||
|
||||
"fail stream on buffer overflow in fail mode" in assertAllStagesStopped {
|
||||
|
|
@ -218,7 +218,7 @@ class QueueSourceSpec extends AkkaSpec {
|
|||
val sub = s.expectSubscription
|
||||
queue.watchCompletion().pipeTo(testActor)
|
||||
sub.cancel()
|
||||
expectMsg(())
|
||||
expectMsg(Done)
|
||||
|
||||
queue.offer(1).onFailure { case e ⇒ e.isInstanceOf[IllegalStateException] should ===(true) }
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,33 @@
|
|||
/**
|
||||
* Copyright (C) 2016 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream
|
||||
|
||||
sealed abstract class QueueOfferResult
|
||||
|
||||
/**
|
||||
* Contains types that is used as return types for async callbacks to streams
|
||||
*/
|
||||
object QueueOfferResult {
|
||||
|
||||
/**
|
||||
* Type is used to indicate that stream is successfully enqueued an element
|
||||
*/
|
||||
final case object Enqueued extends QueueOfferResult
|
||||
|
||||
/**
|
||||
* Type is used to indicate that stream is dropped an element
|
||||
*/
|
||||
final case object Dropped extends QueueOfferResult
|
||||
|
||||
/**
|
||||
* Type is used to indicate that stream is failed before or during call to the stream
|
||||
* @param cause - exception that stream failed with
|
||||
*/
|
||||
final case class Failure(cause: Throwable) extends QueueOfferResult
|
||||
|
||||
/**
|
||||
* Type is used to indicate that stream is completed before call
|
||||
*/
|
||||
case object QueueClosed extends QueueOfferResult
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue