Deprecates status message based api #27503 (#27519)

* Deprecates status message based api #27503
* Deprecates actorRefWithAck for actorRefWithBackpressure
This commit is contained in:
Nicolas Vollmar 2019-09-10 11:59:19 +02:00 committed by Patrik Nordwall
parent aee0152da2
commit 751918e84c
34 changed files with 618 additions and 195 deletions

View file

@ -6,33 +6,27 @@ package akka.stream.javadsl
import java.util
import java.util.Optional
import java.util.concurrent.{ CompletableFuture, CompletionStage }
import java.util.function.{ BiFunction, Supplier }
import akka.actor.{ ActorRef, Cancellable }
import akka.actor.{ ActorRef, Cancellable, ClassicActorSystemProvider }
import akka.event.LoggingAdapter
import akka.japi.{ function, Pair, Util }
import akka.japi.{ function, JavaPartialFunction, Pair, Util }
import akka.stream._
import akka.stream.impl.LinearTraversalBuilder
import akka.util.{ ConstantFun, Timeout }
import akka.util.JavaDurationConverters._
import akka.util.ccompat.JavaConverters._
import akka.util.{ unused, _ }
import akka.{ Done, NotUsed }
import com.github.ghik.silencer.silent
import org.reactivestreams.{ Publisher, Subscriber }
import scala.annotation.unchecked.uncheckedVariance
import akka.util.ccompat.JavaConverters._
import scala.collection.immutable
import scala.compat.java8.FutureConverters._
import scala.compat.java8.OptionConverters._
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{ Future, Promise }
import scala.compat.java8.OptionConverters._
import java.util.concurrent.CompletionStage
import java.util.concurrent.CompletableFuture
import java.util.function.{ BiFunction, Supplier }
import akka.actor.ClassicActorSystemProvider
import akka.util.unused
import com.github.ghik.silencer.silent
import scala.compat.java8.FutureConverters._
import scala.reflect.ClassTag
/** Java API */
@ -287,6 +281,64 @@ object Source {
def asSubscriber[T](): Source[T, Subscriber[T]] =
new Source(scaladsl.Source.asSubscriber)
/**
* Creates a `Source` that is materialized as an [[akka.actor.ActorRef]].
* Messages sent to this actor will be emitted to the stream if there is demand from downstream,
* otherwise they will be buffered until request for demand is received.
*
* Depending on the defined [[akka.stream.OverflowStrategy]] it might drop elements if
* there is no space available in the buffer.
*
* The strategy [[akka.stream.OverflowStrategy.backpressure]] is not supported, and an
* IllegalArgument("Backpressure overflowStrategy not supported") will be thrown if it is passed as argument.
*
* The buffer can be disabled by using `bufferSize` of 0 and then received messages are dropped if there is no demand
* from downstream. When `bufferSize` is 0 the `overflowStrategy` does not matter. An async boundary is added after
* this Source; as such, it is never safe to assume the downstream will always generate demand.
*
* The stream can be completed successfully by sending the actor reference a message that is matched by
* `completionMatcher` in which case already buffered elements will be signaled before signaling
* completion.
*
* The stream can be completed with failure by sending a message that is matched by `failureMatcher`. The extracted
* [[Throwable]] will be used to fail the stream. In case the Actor is still draining its internal buffer (after having received
* a message matched by `completionMatcher`) before signaling completion and it receives a message matched by `failureMatcher`,
* the failure will be signaled downstream immediately (instead of the completion signal).
*
* Note that terminating the actor without first completing it, either with a success or a
* failure, will prevent the actor triggering downstream completion and the stream will continue
* to run even though the source actor is dead. Therefore you should **not** attempt to
* manually terminate the actor such as with a [[akka.actor.PoisonPill]].
*
* The actor will be stopped when the stream is completed, failed or canceled from downstream,
* i.e. you can watch it to get notified when that happens.
*
* See also [[akka.stream.scaladsl.Source.queue]].
*
* @param completionMatcher catches the completion message to end the stream
* @param failureMatcher catches the failure message to fail the stream
* @param bufferSize The size of the buffer in element count
* @param overflowStrategy Strategy that is used when incoming elements cannot fit inside the buffer
*/
def actorRef[T](
completionMatcher: akka.japi.function.Function[Any, java.util.Optional[CompletionStrategy]],
failureMatcher: akka.japi.function.Function[Any, java.util.Optional[Throwable]],
bufferSize: Int,
overflowStrategy: OverflowStrategy): Source[T, ActorRef] =
new Source(scaladsl.Source.actorRef(new JavaPartialFunction[Any, CompletionStrategy] {
override def apply(x: Any, isCheck: Boolean): CompletionStrategy = {
val result = completionMatcher(x)
if (!result.isPresent) throw JavaPartialFunction.noMatch()
else result.get()
}
}, new JavaPartialFunction[Any, Throwable] {
override def apply(x: Any, isCheck: Boolean): Throwable = {
val result = failureMatcher(x)
if (!result.isPresent) throw JavaPartialFunction.noMatch()
else result.get()
}
}, bufferSize, overflowStrategy))
/**
* Creates a `Source` that is materialized as an [[akka.actor.ActorRef]].
* Messages sent to this actor will be emitted to the stream if there is demand from downstream,
@ -330,8 +382,82 @@ object Source {
* @param bufferSize The size of the buffer in element count
* @param overflowStrategy Strategy that is used when incoming elements cannot fit inside the buffer
*/
@Deprecated
@deprecated("Use variant accepting completion and failure matchers", "2.6.0")
def actorRef[T](bufferSize: Int, overflowStrategy: OverflowStrategy): Source[T, ActorRef] =
new Source(scaladsl.Source.actorRef(bufferSize, overflowStrategy))
new Source(scaladsl.Source.actorRef({
case akka.actor.Status.Success(s: CompletionStrategy) => s
case akka.actor.Status.Success(_) => CompletionStrategy.Draining
case akka.actor.Status.Success => CompletionStrategy.Draining
}, { case akka.actor.Status.Failure(cause) => cause }, bufferSize, overflowStrategy))
/**
* Creates a `Source` that is materialized as an [[akka.actor.ActorRef]].
* Messages sent to this actor will be emitted to the stream if there is demand from downstream,
* and a new message will only be accepted after the previous messages has been consumed and acknowledged back.
* The stream will complete with failure if a message is sent before the acknowledgement has been replied back.
*
* The stream can be completed with failure by sending a message that is matched by `failureMatcher`. The extracted
* [[Throwable]] will be used to fail the stream. In case the Actor is still draining its internal buffer (after having received
* a message matched by `completionMatcher`) before signaling completion and it receives a message matched by `failureMatcher`,
* the failure will be signaled downstream immediately (instead of the completion signal).
*
* The actor will be stopped when the stream is completed, failed or canceled from downstream,
* i.e. you can watch it to get notified when that happens.
*/
def actorRefWithBackpressure[T](
ackMessage: Any,
completionMatcher: akka.japi.function.Function[Any, java.util.Optional[CompletionStrategy]],
failureMatcher: akka.japi.function.Function[Any, java.util.Optional[Throwable]]): Source[T, ActorRef] =
new Source(scaladsl.Source.actorRefWithBackpressure(ackMessage, new JavaPartialFunction[Any, CompletionStrategy] {
override def apply(x: Any, isCheck: Boolean): CompletionStrategy = {
val result = completionMatcher(x)
if (!result.isPresent) throw JavaPartialFunction.noMatch()
else result.get()
}
}, new JavaPartialFunction[Any, Throwable] {
override def apply(x: Any, isCheck: Boolean): Throwable = {
val result = failureMatcher(x)
if (!result.isPresent) throw JavaPartialFunction.noMatch()
else result.get()
}
}))
/**
* Creates a `Source` that is materialized as an [[akka.actor.ActorRef]].
* Messages sent to this actor will be emitted to the stream if there is demand from downstream,
* and a new message will only be accepted after the previous messages has been consumed and acknowledged back.
* The stream will complete with failure if a message is sent before the acknowledgement has been replied back.
*
* The stream can be completed with failure by sending a message that is matched by `failureMatcher`. The extracted
* [[Throwable]] will be used to fail the stream. In case the Actor is still draining its internal buffer (after having received
* a message matched by `completionMatcher`) before signaling completion and it receives a message matched by `failureMatcher`,
* the failure will be signaled downstream immediately (instead of the completion signal).
*
* The actor will be stopped when the stream is completed, failed or canceled from downstream,
* i.e. you can watch it to get notified when that happens.
*
* @deprecated Use actorRefWithBackpressure instead
*/
@Deprecated
@deprecated("Use actorRefWithBackpressure instead", "2.6.0")
def actorRefWithAck[T](
ackMessage: Any,
completionMatcher: akka.japi.function.Function[Any, java.util.Optional[CompletionStrategy]],
failureMatcher: akka.japi.function.Function[Any, java.util.Optional[Throwable]]): Source[T, ActorRef] =
new Source(scaladsl.Source.actorRefWithBackpressure(ackMessage, new JavaPartialFunction[Any, CompletionStrategy] {
override def apply(x: Any, isCheck: Boolean): CompletionStrategy = {
val result = completionMatcher(x)
if (!result.isPresent) throw JavaPartialFunction.noMatch()
else result.get()
}
}, new JavaPartialFunction[Any, Throwable] {
override def apply(x: Any, isCheck: Boolean): Throwable = {
val result = failureMatcher(x)
if (!result.isPresent) throw JavaPartialFunction.noMatch()
else result.get()
}
}))
/**
* Creates a `Source` that is materialized as an [[akka.actor.ActorRef]].
@ -352,8 +478,14 @@ object Source {
* The actor will be stopped when the stream is completed, failed or canceled from downstream,
* i.e. you can watch it to get notified when that happens.
*/
@Deprecated
@deprecated("Use actorRefWithBackpressure accepting completion and failure matchers", "2.6.0")
def actorRefWithAck[T](ackMessage: Any): Source[T, ActorRef] =
new Source(scaladsl.Source.actorRefWithAck(ackMessage))
new Source(scaladsl.Source.actorRefWithBackpressure(ackMessage, {
case akka.actor.Status.Success(s: CompletionStrategy) => s
case akka.actor.Status.Success(_) => CompletionStrategy.Draining
case akka.actor.Status.Success => CompletionStrategy.Draining
}, { case akka.actor.Status.Failure(cause) => cause }))
/**
* A graph with the shape of a source logically is a source, this method makes