remove more deprecated pekko-stream methods (#1996)
* remove deprecated setup methods * remove deprecated lazyInit methods * more deprecated methods * Update SubFlow.scala * scalafmt * Update SubFlow.scala * subflow/subsource * Update DslFactoriesConsistencySpec.scala * remove docs for removed operators
This commit is contained in:
parent
c8960e2783
commit
0dd219fab4
24 changed files with 132 additions and 2718 deletions
|
|
@ -1,34 +0,0 @@
|
|||
# Flow.lazyInitAsync
|
||||
|
||||
Deprecated by @ref:[`Flow.lazyFutureFlow`](lazyFutureFlow.md) in combination with @ref:[`prefixAndTail`](../Source-or-Flow/prefixAndTail.md).
|
||||
|
||||
@ref[Simple operators](../index.md#simple-operators)
|
||||
|
||||
## Signature
|
||||
|
||||
@apidoc[Flow.lazyInitAsync](Flow$) { scala="#lazyInitAsync[I,O,M](flowFactory:()=>scala.concurrent.Future[org.apache.pekko.stream.scaladsl.Flow[I,O,M]]):org.apache.pekko.stream.scaladsl.Flow[I,O,scala.concurrent.Future[Option[M]]]" java="#lazyInitAsync(org.apache.pekko.japi.function.Creator)" }
|
||||
|
||||
## Description
|
||||
|
||||
`fromCompletionStage` is deprecated, please use @ref:[lazyFutureFlow](lazyFutureFlow.md) in combination with @ref:[`prefixAndTail`](../Source-or-Flow/prefixAndTail.md)) instead.
|
||||
|
||||
Defers creation until a first element arrives.
|
||||
|
||||
## Reactive Streams semantics
|
||||
|
||||
@@@div { .callout }
|
||||
|
||||
**emits** when the internal flow is successfully created and it emits
|
||||
|
||||
**backpressures** when the internal flow is successfully created and it backpressures
|
||||
|
||||
**completes** when upstream completes and all elements have been emitted from the internal flow
|
||||
|
||||
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
||||
|
||||
**cancels** when downstream cancels (keep reading)
|
||||
The operator's default behavior in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||
This behavior can be controlled by setting the [[org.apache.pekko.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||
this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||
@@@
|
||||
|
||||
|
|
@ -1,35 +0,0 @@
|
|||
# Sink.lazyInitAsync
|
||||
|
||||
Deprecated by @ref:[`Sink.lazyFutureSink`](lazyFutureSink.md).
|
||||
|
||||
@ref[Sink operators](../index.md#sink-operators)
|
||||
|
||||
## Signature
|
||||
|
||||
@apidoc[Flow.lazyInitAsync](Flow$) { scala="#lazyInitAsync[I,O,M](flowFactory:()=>scala.concurrent.Future[org.apache.pekko.stream.scaladsl.Flow[I,O,M]]):org.apache.pekko.stream.scaladsl.Flow[I,O,scala.concurrent.Future[Option[M]]]" java="#lazyInitAsync(org.apache.pekko.japi.function.Creator)" }
|
||||
@apidoc[Sink.lazyInitAsync](Sink$) { scala="#lazyInitAsync[T,M](sinkFactory:()=>scala.concurrent.Future[org.apache.pekko.stream.scaladsl.Sink[T,M]]):org.apache.pekko.stream.scaladsl.Sink[T,scala.concurrent.Future[Option[M]]]" java="#lazyInitAsync(org.apache.pekko.japi.function.Creator)" }
|
||||
|
||||
|
||||
## Description
|
||||
|
||||
`lazyInitAsync` is deprecated, please use @ref:[lazyFutureSink](lazyFutureSink.md) instead.
|
||||
|
||||
Creates a real `Sink` upon receiving the first element. Internal `Sink` will not be created if there are no elements,
|
||||
because of completion or error.
|
||||
|
||||
- If upstream completes before an element was received then the @scala[`Future`]@java[`CompletionStage`] is completed with @scala[`None`]@java[an empty `Optional`].
|
||||
- If upstream fails before an element was received, `sinkFactory` throws an exception, or materialization of the internal
|
||||
sink fails then the @scala[`Future`]@java[`CompletionStage`] is completed with the exception.
|
||||
- Otherwise the @scala[`Future`]@java[`CompletionStage`] is completed with the materialized value of the internal sink.
|
||||
|
||||
## Reactive Streams semantics
|
||||
|
||||
@@@div { .callout }
|
||||
|
||||
**cancels** never
|
||||
|
||||
**backpressures** when initialized and when created sink backpressures
|
||||
|
||||
@@@
|
||||
|
||||
|
||||
|
|
@ -1,20 +0,0 @@
|
|||
# Sink.setup
|
||||
|
||||
Defer the creation of a `Sink` until materialization and access `ActorMaterializer` and `Attributes`
|
||||
|
||||
@ref[Sink operators](../index.md#sink-operators)
|
||||
|
||||
@@@ warning
|
||||
|
||||
The `setup` operator has been deprecated, use @ref:[fromMaterializer](./fromMaterializer.md) instead.
|
||||
|
||||
@@@
|
||||
|
||||
## Signature
|
||||
|
||||
@apidoc[Sink.setup](Sink$) { scala="#setup[T,M](factory:(org.apache.pekko.stream.ActorMaterializer,org.apache.pekko.stream.Attributes)=>org.apache.pekko.stream.scaladsl.Sink[T,M]):org.apache.pekko.stream.scaladsl.Sink[T,scala.concurrent.Future[M]]" java="#setup(java.util.function.BiFunction)" }
|
||||
|
||||
## Description
|
||||
|
||||
Typically used when access to materializer is needed to run a different stream during the construction of a sink.
|
||||
Can also be used to access the underlying `ActorSystem` from `ActorMaterializer`.
|
||||
|
|
@ -1,21 +0,0 @@
|
|||
# setup
|
||||
|
||||
Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`
|
||||
|
||||
@ref[Simple operators](../index.md#simple-operators)
|
||||
|
||||
@@@ warning
|
||||
|
||||
The `setup` operator has been deprecated, use @ref:[fromMaterializer](./fromMaterializer.md) instead.
|
||||
|
||||
@@@
|
||||
|
||||
## Signature
|
||||
|
||||
@apidoc[Source.setup](Source$) { scala="#setup[T,M](factory:(org.apache.pekko.stream.ActorMaterializer,org.apache.pekko.stream.Attributes)=>org.apache.pekko.stream.scaladsl.Source[T,M]):org.apache.pekko.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" java="#setup(java.util.function.BiFunction)" }
|
||||
@apidoc[Flow.setup](Flow$) { scala="#setup[T,U,M](factory:(org.apache.pekko.stream.ActorMaterializer,org.apache.pekko.stream.Attributes)=>org.apache.pekko.stream.scaladsl.Flow[T,U,M]):org.apache.pekko.stream.scaladsl.Flow[T,U,scala.concurrent.Future[M]]" java="#setup(java.util.function.BiFunction)" }
|
||||
|
||||
## Description
|
||||
|
||||
Typically used when access to materializer is needed to run a different stream during the construction of a source/flow.
|
||||
Can also be used to access the underlying `ActorSystem` from `ActorMaterializer`.
|
||||
|
|
@ -1,27 +0,0 @@
|
|||
# Source.lazily
|
||||
|
||||
Deprecated by @ref:[`Source.lazySource`](lazySource.md).
|
||||
|
||||
@ref[Source operators](../index.md#source-operators)
|
||||
|
||||
## Signature
|
||||
|
||||
@apidoc[Source.lazily](Source$) { scala="#lazily[T,M](create:()=>org.apache.pekko.stream.scaladsl.Source[T,M]):org.apache.pekko.stream.scaladsl.Source[T,scala.concurrent.Future[M]]" java="#lazily(org.apache.pekko.japi.function.Creator)" }
|
||||
|
||||
|
||||
## Description
|
||||
|
||||
`lazily` is deprecated, please use @ref:[lazySource](lazySource.md) instead.
|
||||
|
||||
Defers creation and materialization of a `Source` until there is demand.
|
||||
|
||||
## Reactive Streams semantics
|
||||
|
||||
@@@div { .callout }
|
||||
|
||||
**emits** depends on the wrapped `Source`
|
||||
|
||||
**completes** depends on the wrapped `Source`
|
||||
|
||||
@@@
|
||||
|
||||
|
|
@ -1,24 +0,0 @@
|
|||
# Source.lazilyAsync
|
||||
|
||||
Deprecated by @ref:[`Source.lazyFutureSource`](lazyFutureSource.md).
|
||||
|
||||
@ref[Source operators](../index.md#source-operators)
|
||||
|
||||
## Signature
|
||||
|
||||
## Description
|
||||
|
||||
`lazilyAsync` is deprecated, please use @ref:[lazyFutureSource](lazyFutureSource.md) instead.
|
||||
|
||||
Defers creation and materialization of a `CompletionStage` until there is demand.
|
||||
|
||||
## Reactive Streams semantics
|
||||
|
||||
@@@div { .callout }
|
||||
|
||||
**emits** the future completes
|
||||
|
||||
**completes** after the future has completed
|
||||
|
||||
@@@
|
||||
|
||||
|
|
@ -27,8 +27,6 @@ These built-in sources are available from @scala[`org.apache.pekko.stream.scalad
|
|||
|Source|<a name="future"></a>@ref[future](Source/future.md)|Send the single value of the `Future` when it completes and there is demand.|
|
||||
|Source|<a name="futuresource"></a>@ref[futureSource](Source/futureSource.md)|Streams the elements of the given future source once it successfully completes.|
|
||||
|Source|<a name="iterate"></a>@ref[iterate](Source/iterate.md)|Creates a sequential `Source` by iterating with the given predicate, function and seed.|
|
||||
|Source|<a name="lazily"></a>@ref[lazily](Source/lazily.md)|Deprecated by @ref[`Source.lazySource`](Source/lazySource.md).|
|
||||
|Source|<a name="lazilyasync"></a>@ref[lazilyAsync](Source/lazilyAsync.md)|Deprecated by @ref[`Source.lazyFutureSource`](Source/lazyFutureSource.md).|
|
||||
|Source|<a name="lazycompletionstage"></a>@ref[lazyCompletionStage](Source/lazyCompletionStage.md)|Defers creation of a future of a single element source until there is demand.|
|
||||
|Source|<a name="lazycompletionstagesource"></a>@ref[lazyCompletionStageSource](Source/lazyCompletionStageSource.md)|Defers creation of a future source until there is demand.|
|
||||
|Source|<a name="lazyfuture"></a>@ref[lazyFuture](Source/lazyFuture.md)|Defers creation of a future of a single element source until there is demand.|
|
||||
|
|
@ -78,7 +76,6 @@ These built-in sinks are available from @scala[`org.apache.pekko.stream.scaladsl
|
|||
|Sink|<a name="lastoption"></a>@ref[lastOption](Sink/lastOption.md)|Materialize a @scala[`Future[Option[T]]`] @java[`CompletionStage<Optional<T>>`] which completes with the last value emitted wrapped in an @scala[`Some`] @java[`Optional`] when the stream completes.|
|
||||
|Sink|<a name="lazycompletionstagesink"></a>@ref[lazyCompletionStageSink](Sink/lazyCompletionStageSink.md)|Defers creation and materialization of a `Sink` until there is a first element.|
|
||||
|Sink|<a name="lazyfuturesink"></a>@ref[lazyFutureSink](Sink/lazyFutureSink.md)|Defers creation and materialization of a `Sink` until there is a first element.|
|
||||
|Sink|<a name="lazyinitasync"></a>@ref[lazyInitAsync](Sink/lazyInitAsync.md)|Deprecated by @ref[`Sink.lazyFutureSink`](Sink/lazyFutureSink.md).|
|
||||
|Sink|<a name="lazysink"></a>@ref[lazySink](Sink/lazySink.md)|Defers creation and materialization of a `Sink` until there is a first element.|
|
||||
|Sink|<a name="never"></a>@ref[never](Sink/never.md)|Always backpressure never cancel and never consume any elements from the stream.|
|
||||
|Sink|<a name="none"></a>@ref[none](Sink/none.md)|A `Sink` that will test the given predicate `p` for every received element and completes with the result.|
|
||||
|
|
@ -87,7 +84,6 @@ These built-in sinks are available from @scala[`org.apache.pekko.stream.scaladsl
|
|||
|Sink|<a name="queue"></a>@ref[queue](Sink/queue.md)|Materialize a `SinkQueue` that can be pulled to trigger demand through the sink.|
|
||||
|Sink|<a name="reduce"></a>@ref[reduce](Sink/reduce.md)|Apply a reduction function on the incoming elements and pass the result to the next invocation.|
|
||||
|Sink|<a name="seq"></a>@ref[seq](Sink/seq.md)|Collect values emitted from the stream into a collection.|
|
||||
|Sink|<a name="setup"></a>@ref[setup](Sink/setup.md)|Defer the creation of a `Sink` until materialization and access `ActorMaterializer` and `Attributes`|
|
||||
|Sink|<a name="takelast"></a>@ref[takeLast](Sink/takeLast.md)|Collect the last `n` values emitted from the stream into a collection.|
|
||||
|
||||
## Additional Sink and Source converters
|
||||
|
|
@ -173,7 +169,6 @@ depending on being backpressured by downstream or not.
|
|||
|Flow|<a name="lazycompletionstageflow"></a>@ref[lazyCompletionStageFlow](Flow/lazyCompletionStageFlow.md)|Defers creation and materialization of a `Flow` until there is a first element.|
|
||||
|Flow|<a name="lazyflow"></a>@ref[lazyFlow](Flow/lazyFlow.md)|Defers creation and materialization of a `Flow` until there is a first element.|
|
||||
|Flow|<a name="lazyfutureflow"></a>@ref[lazyFutureFlow](Flow/lazyFutureFlow.md)|Defers creation and materialization of a `Flow` until there is a first element.|
|
||||
|Flow|<a name="lazyinitasync"></a>@ref[lazyInitAsync](Flow/lazyInitAsync.md)|Deprecated by @ref[`Flow.lazyFutureFlow`](Flow/lazyFutureFlow.md) in combination with @ref[`prefixAndTail`](Flow/../Source-or-Flow/prefixAndTail.md).|
|
||||
|Source/Flow|<a name="limit"></a>@ref[limit](Source-or-Flow/limit.md)|Limit number of element from upstream to given `max` number.|
|
||||
|Source/Flow|<a name="limitweighted"></a>@ref[limitWeighted](Source-or-Flow/limitWeighted.md)|Limit the total weight of incoming elements|
|
||||
|Source/Flow|<a name="log"></a>@ref[log](Source-or-Flow/log.md)|Log elements flowing through the stream as well as completion and erroring.|
|
||||
|
|
@ -186,7 +181,6 @@ depending on being backpressured by downstream or not.
|
|||
|Source/Flow|<a name="reduce"></a>@ref[reduce](Source-or-Flow/reduce.md)|Start with first element and then apply the current and next value to the given function, when upstream complete the current value is emitted downstream.|
|
||||
|Source/Flow|<a name="scan"></a>@ref[scan](Source-or-Flow/scan.md)|Emit its current value, which starts at `zero`, and then apply the current and next value to the given function, emitting the next current value.|
|
||||
|Source/Flow|<a name="scanasync"></a>@ref[scanAsync](Source-or-Flow/scanAsync.md)|Just like @ref[`scan`](Source-or-Flow/./scan.md) but receives a function that results in a @scala[`Future`] @java[`CompletionStage`] to the next value.|
|
||||
|Source/Flow|<a name="setup"></a>@ref[setup](Source-or-Flow/setup.md)|Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`|
|
||||
|Source/Flow|<a name="sliding"></a>@ref[sliding](Source-or-Flow/sliding.md)|Provide a sliding window over the incoming stream and pass the windows as groups of elements downstream.|
|
||||
|Source/Flow|<a name="statefulmap"></a>@ref[statefulMap](Source-or-Flow/statefulMap.md)|Transform each stream element with the help of a state.|
|
||||
|Source/Flow|<a name="statefulmapconcat"></a>@ref[statefulMapConcat](Source-or-Flow/statefulMapConcat.md)|Transform each element into zero or more elements that are individually passed downstream.|
|
||||
|
|
@ -519,8 +513,6 @@ For more background see the @ref[Error Handling in Streams](../stream-error.md)
|
|||
* [keepAlive](Source-or-Flow/keepAlive.md)
|
||||
* [last](Sink/last.md)
|
||||
* [lastOption](Sink/lastOption.md)
|
||||
* [lazily](Source/lazily.md)
|
||||
* [lazilyAsync](Source/lazilyAsync.md)
|
||||
* [lazyCompletionStage](Source/lazyCompletionStage.md)
|
||||
* [lazyCompletionStageFlow](Flow/lazyCompletionStageFlow.md)
|
||||
* [lazyCompletionStageSink](Sink/lazyCompletionStageSink.md)
|
||||
|
|
@ -530,8 +522,6 @@ For more background see the @ref[Error Handling in Streams](../stream-error.md)
|
|||
* [lazyFutureFlow](Flow/lazyFutureFlow.md)
|
||||
* [lazyFutureSink](Sink/lazyFutureSink.md)
|
||||
* [lazyFutureSource](Source/lazyFutureSource.md)
|
||||
* [lazyInitAsync](Flow/lazyInitAsync.md)
|
||||
* [lazyInitAsync](Sink/lazyInitAsync.md)
|
||||
* [lazySingle](Source/lazySingle.md)
|
||||
* [lazySink](Sink/lazySink.md)
|
||||
* [lazySource](Source/lazySource.md)
|
||||
|
|
@ -584,8 +574,6 @@ For more background see the @ref[Error Handling in Streams](../stream-error.md)
|
|||
* [scan](Source-or-Flow/scan.md)
|
||||
* [scanAsync](Source-or-Flow/scanAsync.md)
|
||||
* [seq](Sink/seq.md)
|
||||
* [setup](Source-or-Flow/setup.md)
|
||||
* [setup](Sink/setup.md)
|
||||
* [single](Source/single.md)
|
||||
* [sink](PubSub/sink.md)
|
||||
* [sliding](Source-or-Flow/sliding.md)
|
||||
|
|
|
|||
|
|
@ -90,7 +90,6 @@ object StreamOperatorsIndexGenerator extends AutoPlugin {
|
|||
val pendingTestCases = Map(
|
||||
"Source" -> pendingSourceOrFlow,
|
||||
"Flow" -> (pendingSourceOrFlow ++ Seq(
|
||||
"lazyInit",
|
||||
"fromProcessorMat",
|
||||
"toProcessor",
|
||||
"fromProcessor",
|
||||
|
|
@ -99,7 +98,6 @@ object StreamOperatorsIndexGenerator extends AutoPlugin {
|
|||
"joinMat",
|
||||
"fromFunction")),
|
||||
"Sink" -> Seq(
|
||||
"lazyInit",
|
||||
"contramap",
|
||||
"named",
|
||||
"addAttributes",
|
||||
|
|
|
|||
|
|
@ -169,6 +169,7 @@ class DslFactoriesConsistencySpec extends AnyWordSpec with Matchers {
|
|||
Ignore(_ == pekko.stream.scaladsl.Source.getClass, _ == "actorRef", _ => true, _ => true), // Internal in scaladsl
|
||||
Ignore(_ == pekko.stream.scaladsl.Source.getClass, _ == "actorRefWithAck", _ => true, _ => true), // Internal in scaladsl
|
||||
Ignore(_ == pekko.stream.scaladsl.Source.getClass, _ == "actorRefWithBackpressure", _ => true, _ => true), // Internal in scaladsl
|
||||
Ignore(_ == pekko.stream.scaladsl.Source.getClass, _ == "tick", _ => true, _ => true), // Params are different (Scala vs Java durations)
|
||||
Ignore(_ == pekko.stream.scaladsl.BidiFlow.getClass, _ == "apply", _ == 24, _ => true),
|
||||
Ignore(_ == pekko.stream.scaladsl.BidiFlow.getClass, _ == "bidirectionalIdleTimeout", _ => true, _ => true),
|
||||
Ignore(_ == pekko.stream.scaladsl.GraphDSL.getClass, _ == "runnable", _ == 24, _ => true),
|
||||
|
|
|
|||
|
|
@ -18,18 +18,17 @@ import java.nio.file.StandardOpenOption.{ CREATE, WRITE }
|
|||
|
||||
import scala.annotation.nowarn
|
||||
import scala.collection.mutable.ListBuffer
|
||||
import scala.concurrent.{ Await, Future }
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.Success
|
||||
|
||||
import com.google.common.jimfs.{ Configuration, Jimfs }
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.dispatch.ExecutionContexts
|
||||
import pekko.stream._
|
||||
import pekko.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
|
||||
import pekko.stream.impl.StreamSupervisor.Children
|
||||
import pekko.stream.scaladsl.{ FileIO, Keep, Sink, Source }
|
||||
import pekko.stream.scaladsl.{ FileIO, Keep, Source }
|
||||
import pekko.stream.testkit._
|
||||
import pekko.stream.testkit.Utils._
|
||||
import pekko.util.ByteString
|
||||
|
|
@ -208,23 +207,6 @@ class FileSinkSpec extends StreamSpec(UnboundedMailboxConfig) with ScalaFutures
|
|||
}
|
||||
}
|
||||
|
||||
"write single line to a file from lazy sink" in {
|
||||
// LazySink must wait for result of initialization even if got upstreamComplete
|
||||
targetFile { f =>
|
||||
val completion = Source(List(TestByteStrings.head)).runWith(
|
||||
Sink
|
||||
.lazyInitAsync(() => Future.successful(FileIO.toPath(f)))
|
||||
// map a Future[Option[Future[IOResult]]] into a Future[Option[IOResult]]
|
||||
.mapMaterializedValue(_.flatMap {
|
||||
case Some(future) => future.map(Some(_))(ExecutionContexts.parasitic)
|
||||
case None => Future.successful(None)
|
||||
}(ExecutionContexts.parasitic)))
|
||||
|
||||
Await.result(completion, 3.seconds)
|
||||
checkFileContents(f, TestLines.head)
|
||||
}
|
||||
}
|
||||
|
||||
"complete materialized future with an exception when upstream fails" in {
|
||||
val te = TE("oh no")
|
||||
targetFile { f =>
|
||||
|
|
|
|||
|
|
@ -1,107 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* license agreements; and to You under the Apache License, version 2.0:
|
||||
*
|
||||
* https://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* This file is part of the Apache Pekko project, which was derived from Akka.
|
||||
*/
|
||||
|
||||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.scaladsl
|
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.concurrent.Future
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.Done
|
||||
import pekko.stream.testkit.StreamSpec
|
||||
import pekko.stream.testkit.TestSubscriber
|
||||
import pekko.testkit.DefaultTimeout
|
||||
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
|
||||
@nowarn("msg=deprecated") // tests deprecated methods
|
||||
class LazilyAsyncSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
|
||||
|
||||
import system.dispatcher
|
||||
|
||||
"A lazy async source" should {
|
||||
|
||||
"work in happy path scenario" in {
|
||||
val stream = Source
|
||||
.lazilyAsync { () =>
|
||||
Future(42)
|
||||
}
|
||||
.runWith(Sink.head)
|
||||
|
||||
stream.futureValue should ===(42)
|
||||
}
|
||||
|
||||
"call factory method on demand only" in {
|
||||
val probe = TestSubscriber.probe[Int]()
|
||||
val constructed = new AtomicBoolean(false)
|
||||
|
||||
Source
|
||||
.lazilyAsync { () =>
|
||||
constructed.set(true); Future(42)
|
||||
}
|
||||
.runWith(Sink.fromSubscriber(probe))
|
||||
probe.cancel()
|
||||
|
||||
constructed.get() should ===(false)
|
||||
}
|
||||
|
||||
"fail materialized value when downstream cancels without ever consuming any element" in {
|
||||
val materialization = Source
|
||||
.lazilyAsync { () =>
|
||||
Future(42)
|
||||
}
|
||||
.toMat(Sink.cancelled)(Keep.left)
|
||||
.run()
|
||||
|
||||
intercept[RuntimeException] {
|
||||
materialization.futureValue
|
||||
}
|
||||
}
|
||||
|
||||
"materialize when the source has been created" in {
|
||||
val probe = TestSubscriber.probe[Int]()
|
||||
|
||||
val materialization: Future[Done] =
|
||||
Source
|
||||
.lazilyAsync { () =>
|
||||
Future(42)
|
||||
}
|
||||
.mapMaterializedValue(_.map(_ => Done))
|
||||
.to(Sink.fromSubscriber(probe))
|
||||
.run()
|
||||
|
||||
materialization.value shouldEqual None
|
||||
probe.request(1)
|
||||
probe.expectNext(42)
|
||||
materialization.futureValue should ===(Done)
|
||||
|
||||
probe.cancel()
|
||||
}
|
||||
|
||||
"propagate failed future from factory" in {
|
||||
val probe = TestSubscriber.probe[Int]()
|
||||
val failure = new RuntimeException("too bad")
|
||||
Source
|
||||
.lazilyAsync { () =>
|
||||
Future.failed(failure)
|
||||
}
|
||||
.to(Sink.fromSubscriber(probe))
|
||||
.run()
|
||||
|
||||
probe.request(1)
|
||||
probe.expectError(failure)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -13,10 +13,8 @@
|
|||
|
||||
package org.apache.pekko.stream.scaladsl
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.Promise
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import org.apache.pekko
|
||||
|
|
@ -25,13 +23,9 @@ import pekko.stream.{ AbruptStageTerminationException, Attributes, Materializer,
|
|||
import pekko.stream.Attributes.Attribute
|
||||
import pekko.stream.scaladsl.AttributesSpec.AttributesFlow
|
||||
import pekko.stream.testkit.StreamSpec
|
||||
import pekko.stream.testkit.TestPublisher
|
||||
import pekko.stream.testkit.Utils._
|
||||
import pekko.stream.testkit.scaladsl.TestSink
|
||||
import pekko.stream.testkit.scaladsl.TestSource
|
||||
import pekko.testkit.TestProbe
|
||||
|
||||
@nowarn("msg=deprecated") // tests deprecated API as well
|
||||
class LazyFlowSpec extends StreamSpec("""
|
||||
pekko.stream.materializer.initial-input-buffer-size = 1
|
||||
pekko.stream.materializer.max-input-buffer-size = 1
|
||||
|
|
@ -279,129 +273,4 @@ class LazyFlowSpec extends StreamSpec("""
|
|||
}
|
||||
}
|
||||
|
||||
"The deprecated LazyFlow ops" must {
|
||||
def mapF(e: Int): () => Future[Flow[Int, String, NotUsed]] =
|
||||
() => Future.successful(Flow.fromFunction[Int, String](i => (i * e).toString))
|
||||
val flowF = Future.successful(Flow[Int])
|
||||
"work in happy case" in {
|
||||
val probe = Source(2 to 10).via(Flow.lazyInitAsync[Int, String, NotUsed](mapF(2))).runWith(TestSink.probe[String])
|
||||
probe.request(100)
|
||||
(2 to 10).map(i => (i * 2).toString).foreach(probe.expectNext)
|
||||
}
|
||||
|
||||
"work with slow flow init" in {
|
||||
val p = Promise[Flow[Int, Int, NotUsed]]()
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val flowProbe = Source
|
||||
.fromPublisher(sourceProbe)
|
||||
.via(Flow.lazyInitAsync[Int, Int, NotUsed](() => p.future))
|
||||
.runWith(TestSink.probe[Int])
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
flowProbe.request(1)
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
sourceSub.expectRequest(1)
|
||||
sourceProbe.expectNoMessage(200.millis)
|
||||
|
||||
p.success(Flow[Int])
|
||||
flowProbe.request(99)
|
||||
flowProbe.expectNext(0)
|
||||
(1 to 10).foreach(i => {
|
||||
sourceSub.sendNext(i)
|
||||
flowProbe.expectNext(i)
|
||||
})
|
||||
sourceSub.sendComplete()
|
||||
}
|
||||
|
||||
"complete when there was no elements in the stream" in {
|
||||
def flowMaker() = flowF
|
||||
val probe = Source.empty.via(Flow.lazyInitAsync(() => flowMaker())).runWith(TestSink.probe[Int])
|
||||
probe.request(1).expectComplete()
|
||||
}
|
||||
|
||||
"complete normally when upstream completes BEFORE the stage has switched to the inner flow" in {
|
||||
val promise = Promise[Flow[Int, Int, NotUsed]]()
|
||||
val (pub, sub) = TestSource
|
||||
.probe[Int]
|
||||
.viaMat(Flow.lazyInitAsync(() => promise.future))(Keep.left)
|
||||
.toMat(TestSink.probe)(Keep.both)
|
||||
.run()
|
||||
sub.request(1)
|
||||
pub.sendNext(1).sendComplete()
|
||||
promise.success(Flow[Int])
|
||||
sub.expectNext(1).expectComplete()
|
||||
}
|
||||
|
||||
"complete normally when upstream completes AFTER the stage has switched to the inner flow" in {
|
||||
val (pub, sub) = TestSource
|
||||
.probe[Int]
|
||||
.viaMat(Flow.lazyInitAsync(() => Future.successful(Flow[Int])))(Keep.left)
|
||||
.toMat(TestSink.probe)(Keep.both)
|
||||
.run()
|
||||
sub.request(1)
|
||||
pub.sendNext(1)
|
||||
sub.expectNext(1)
|
||||
pub.sendComplete()
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
"fail gracefully when flow factory function failed" in {
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val probe = Source
|
||||
.fromPublisher(sourceProbe)
|
||||
.via(Flow.lazyInitAsync[Int, Int, NotUsed](() => throw ex))
|
||||
.runWith(TestSink.probe[Int])
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
probe.request(1)
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
sourceSub.expectCancellation()
|
||||
probe.expectError(ex)
|
||||
}
|
||||
|
||||
"fail gracefully when upstream failed" in {
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val probe = Source.fromPublisher(sourceProbe).via(Flow.lazyInitAsync(() => flowF)).runWith(TestSink.probe)
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
probe.request(1).expectNext(0)
|
||||
sourceSub.sendError(ex)
|
||||
probe.expectError(ex)
|
||||
}
|
||||
|
||||
"fail gracefully when factory future failed" in {
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val flowProbe = Source
|
||||
.fromPublisher(sourceProbe)
|
||||
.via(Flow.lazyInitAsync[Int, Int, NotUsed](() => Future.failed(ex)))
|
||||
.runWith(TestSink.probe)
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
flowProbe.request(1).expectError(ex)
|
||||
}
|
||||
|
||||
"cancel upstream when the downstream is cancelled" in {
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val probe = Source
|
||||
.fromPublisher(sourceProbe)
|
||||
.via(Flow.lazyInitAsync[Int, Int, NotUsed](() => flowF))
|
||||
.runWith(TestSink.probe[Int])
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
probe.request(1)
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
sourceSub.expectRequest(1)
|
||||
probe.expectNext(0)
|
||||
probe.cancel()
|
||||
sourceSub.expectCancellation()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -13,29 +13,16 @@
|
|||
|
||||
package org.apache.pekko.stream.scaladsl
|
||||
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.Promise
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.{ Done, NotUsed }
|
||||
import pekko.Done
|
||||
import pekko.stream._
|
||||
import pekko.stream.Attributes.Attribute
|
||||
import pekko.stream.scaladsl.AttributesSpec.AttributesSink
|
||||
import pekko.stream.stage.GraphStage
|
||||
import pekko.stream.stage.GraphStageLogic
|
||||
import pekko.stream.testkit.StreamSpec
|
||||
import pekko.stream.testkit.TestPublisher
|
||||
import pekko.stream.testkit.TestSubscriber.Probe
|
||||
import pekko.stream.testkit.Utils._
|
||||
import pekko.stream.testkit.scaladsl.TestSink
|
||||
|
||||
@nowarn("msg=deprecated")
|
||||
class LazySinkSpec extends StreamSpec("""
|
||||
pekko.stream.materializer.initial-input-buffer-size = 1
|
||||
pekko.stream.materializer.max-input-buffer-size = 1
|
||||
|
|
@ -47,131 +34,6 @@ class LazySinkSpec extends StreamSpec("""
|
|||
val myAttributes = Attributes(MyAttribute())
|
||||
|
||||
"A LazySink" must {
|
||||
"work in happy case" in {
|
||||
val futureProbe = Source(0 to 10).runWith(Sink.lazyInitAsync(() => Future.successful(TestSink.probe[Int])))
|
||||
val probe = Await.result(futureProbe, remainingOrDefault).get
|
||||
probe.request(100)
|
||||
(0 to 10).foreach(probe.expectNext)
|
||||
}
|
||||
|
||||
"work with slow sink init" in {
|
||||
val p = Promise[Sink[Int, Probe[Int]]]()
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val futureProbe = Source.fromPublisher(sourceProbe).runWith(Sink.lazyInitAsync(() => p.future))
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
sourceSub.expectRequest(1)
|
||||
sourceProbe.expectNoMessage(200.millis)
|
||||
a[TimeoutException] shouldBe thrownBy { Await.result(futureProbe, remainingOrDefault) }
|
||||
|
||||
p.success(TestSink.probe[Int])
|
||||
val probe = Await.result(futureProbe, remainingOrDefault).get
|
||||
probe.request(100)
|
||||
probe.expectNext(0)
|
||||
(1 to 10).foreach(i => {
|
||||
sourceSub.sendNext(i)
|
||||
probe.expectNext(i)
|
||||
})
|
||||
sourceSub.sendComplete()
|
||||
}
|
||||
|
||||
"complete when there was no elements in stream" in {
|
||||
val futureProbe = Source.empty.runWith(Sink.lazyInitAsync(() => Future.successful(Sink.fold[Int, Int](0)(_ + _))))
|
||||
val futureResult = Await.result(futureProbe, remainingOrDefault)
|
||||
futureResult should ===(None)
|
||||
}
|
||||
|
||||
"complete normally when upstream is completed" in {
|
||||
val futureProbe = Source.single(1).runWith(Sink.lazyInitAsync(() => Future.successful(TestSink.probe[Int])))
|
||||
val futureResult = Await.result(futureProbe, remainingOrDefault).get
|
||||
futureResult.request(1).expectNext(1).expectComplete()
|
||||
}
|
||||
|
||||
"failed gracefully when sink factory method failed" in {
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val futureProbe = Source.fromPublisher(sourceProbe).runWith(Sink.lazyInitAsync[Int, Probe[Int]](() => throw ex))
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
sourceSub.expectCancellation()
|
||||
a[RuntimeException] shouldBe thrownBy { Await.result(futureProbe, remainingOrDefault) }
|
||||
}
|
||||
|
||||
"fail gracefully when upstream failed" in {
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val futureProbe =
|
||||
Source.fromPublisher(sourceProbe).runWith(Sink.lazyInitAsync(() => Future.successful(TestSink.probe[Int])))
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
val probe = Await.result(futureProbe, remainingOrDefault).get
|
||||
probe.request(1).expectNext(0)
|
||||
sourceSub.sendError(ex)
|
||||
probe.expectError(ex)
|
||||
}
|
||||
|
||||
"fail gracefully when factory future failed" in {
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val futureProbe = Source.fromPublisher(sourceProbe).runWith(Sink.lazyInitAsync(() => Future.failed(ex)))
|
||||
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
a[TE] shouldBe thrownBy { Await.result(futureProbe, remainingOrDefault) }
|
||||
}
|
||||
|
||||
"cancel upstream when internal sink is cancelled" in {
|
||||
val sourceProbe = TestPublisher.manualProbe[Int]()
|
||||
val futureProbe =
|
||||
Source.fromPublisher(sourceProbe).runWith(Sink.lazyInitAsync(() => Future.successful(TestSink.probe[Int])))
|
||||
val sourceSub = sourceProbe.expectSubscription()
|
||||
sourceSub.expectRequest(1)
|
||||
sourceSub.sendNext(0)
|
||||
sourceSub.expectRequest(1)
|
||||
val probe = Await.result(futureProbe, remainingOrDefault).get
|
||||
probe.request(1).expectNext(0)
|
||||
probe.cancel()
|
||||
sourceSub.expectCancellation()
|
||||
}
|
||||
|
||||
"fail correctly when materialization of inner sink fails" in {
|
||||
val matFail = TE("fail!")
|
||||
object FailingInnerMat extends GraphStage[SinkShape[String]] {
|
||||
val in = Inlet[String]("in")
|
||||
val shape = SinkShape(in)
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
if ("confuse IntellIJ dead code checker".length > 2) {
|
||||
throw matFail
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val result = Source(List("whatever")).runWith(Sink.lazyInitAsync[String, NotUsed](() => {
|
||||
Future.successful(Sink.fromGraph(FailingInnerMat))
|
||||
}))
|
||||
|
||||
result.failed.futureValue should ===(matFail)
|
||||
}
|
||||
|
||||
// reproducer for #25410
|
||||
"lazily propagate failure" in {
|
||||
case object MyException extends Exception
|
||||
val lazyMatVal = Source(List(1))
|
||||
.concat(Source.lazily(() => Source.failed(MyException)))
|
||||
.runWith(Sink.lazyInitAsync(() => Future.successful(Sink.seq[Int])))
|
||||
|
||||
// lazy init async materialized a sink, so we should have a some here
|
||||
val innerMatVal: Future[immutable.Seq[Int]] = lazyMatVal.futureValue.get
|
||||
|
||||
// the actual matval from Sink.seq should be failed when the stream fails
|
||||
innerMatVal.failed.futureValue should ===(MyException)
|
||||
|
||||
}
|
||||
|
||||
"provide attributes to inner sink" in {
|
||||
val attributes = Source
|
||||
.single(Done)
|
||||
|
|
|
|||
|
|
@ -1,241 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* license agreements; and to You under the Apache License, version 2.0:
|
||||
*
|
||||
* https://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* This file is part of the Apache Pekko project, which was derived from Akka.
|
||||
*/
|
||||
|
||||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.scaladsl
|
||||
|
||||
import scala.annotation.nowarn
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.NotUsed
|
||||
import pekko.stream.testkit.StreamSpec
|
||||
|
||||
@nowarn("msg=deprecated")
|
||||
class SetupSpec extends StreamSpec {
|
||||
|
||||
"Source.setup" should {
|
||||
|
||||
"expose materializer" in {
|
||||
val source = Source.setup { (mat, _) =>
|
||||
Source.single(mat.isShutdown)
|
||||
}
|
||||
|
||||
source.runWith(Sink.head).futureValue shouldBe false
|
||||
}
|
||||
|
||||
"expose attributes" in {
|
||||
val source = Source.setup { (_, attr) =>
|
||||
Source.single(attr.attributeList)
|
||||
}
|
||||
|
||||
source.runWith(Sink.head).futureValue should not be empty
|
||||
}
|
||||
|
||||
"propagate materialized value" in {
|
||||
val source = Source.setup { (_, _) =>
|
||||
Source.maybe[NotUsed]
|
||||
}
|
||||
|
||||
val (completion, element) = source.toMat(Sink.head)(Keep.both).run()
|
||||
completion.futureValue.trySuccess(Some(NotUsed))
|
||||
element.futureValue shouldBe NotUsed
|
||||
}
|
||||
|
||||
"propagate attributes" in {
|
||||
val source = Source
|
||||
.setup { (_, attr) =>
|
||||
Source.single(attr.nameLifted)
|
||||
}
|
||||
.named("my-name")
|
||||
|
||||
source.runWith(Sink.head).futureValue shouldBe Some("setup-my-name")
|
||||
}
|
||||
|
||||
"propagate attributes when nested" in {
|
||||
val source = Source
|
||||
.setup { (_, _) =>
|
||||
Source.setup { (_, attr) =>
|
||||
Source.single(attr.nameLifted)
|
||||
}
|
||||
}
|
||||
.named("my-name")
|
||||
|
||||
source.runWith(Sink.head).futureValue shouldBe Some("setup-my-name-setup")
|
||||
}
|
||||
|
||||
"handle factory failure" in {
|
||||
val error = new Error("boom")
|
||||
val source = Source.setup { (_, _) =>
|
||||
throw error
|
||||
}
|
||||
|
||||
val (materialized, completion) = source.toMat(Sink.head)(Keep.both).run()
|
||||
materialized.failed.futureValue.getCause shouldBe error
|
||||
completion.failed.futureValue.getCause shouldBe error
|
||||
}
|
||||
|
||||
"handle materialization failure" in {
|
||||
val error = new Error("boom")
|
||||
val source = Source.setup { (_, _) =>
|
||||
Source.empty.mapMaterializedValue(_ => throw error)
|
||||
}
|
||||
|
||||
val (materialized, completion) = source.toMat(Sink.head)(Keep.both).run()
|
||||
materialized.failed.futureValue.getCause shouldBe error
|
||||
completion.failed.futureValue.getCause shouldBe error
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"Flow.setup" should {
|
||||
|
||||
"expose materializer" in {
|
||||
val flow = Flow.setup { (mat, _) =>
|
||||
Flow.fromSinkAndSource(Sink.ignore, Source.single(mat.isShutdown))
|
||||
}
|
||||
|
||||
Source.empty.via(flow).runWith(Sink.head).futureValue shouldBe false
|
||||
}
|
||||
|
||||
"expose attributes" in {
|
||||
val flow = Flow.setup { (_, attr) =>
|
||||
Flow.fromSinkAndSource(Sink.ignore, Source.single(attr.attributeList))
|
||||
}
|
||||
|
||||
Source.empty.via(flow).runWith(Sink.head).futureValue should not be empty
|
||||
}
|
||||
|
||||
"propagate materialized value" in {
|
||||
val flow = Flow.setup { (_, _) =>
|
||||
Flow.fromSinkAndSourceMat(Sink.ignore, Source.maybe[NotUsed])(Keep.right)
|
||||
}
|
||||
|
||||
val (completion, element) = Source.empty.viaMat(flow)(Keep.right).toMat(Sink.head)(Keep.both).run()
|
||||
completion.futureValue.trySuccess(Some(NotUsed))
|
||||
element.futureValue shouldBe NotUsed
|
||||
}
|
||||
|
||||
"propagate attributes" in {
|
||||
val flow = Flow
|
||||
.setup { (_, attr) =>
|
||||
Flow.fromSinkAndSource(Sink.ignore, Source.single(attr.nameLifted))
|
||||
}
|
||||
.named("my-name")
|
||||
|
||||
Source.empty.via(flow).runWith(Sink.head).futureValue shouldBe Some("setup-my-name")
|
||||
}
|
||||
|
||||
"propagate attributes when nested" in {
|
||||
val flow = Flow
|
||||
.setup { (_, _) =>
|
||||
Flow.setup { (_, attr) =>
|
||||
Flow.fromSinkAndSource(Sink.ignore, Source.single(attr.nameLifted))
|
||||
}
|
||||
}
|
||||
.named("my-name")
|
||||
|
||||
Source.empty.via(flow).runWith(Sink.head).futureValue shouldBe Some("setup-my-name-setup")
|
||||
}
|
||||
|
||||
"handle factory failure" in {
|
||||
val error = new Error("boom")
|
||||
val flow = Flow.setup { (_, _) =>
|
||||
throw error
|
||||
}
|
||||
|
||||
val (materialized, completion) = Source.empty.viaMat(flow)(Keep.right).toMat(Sink.head)(Keep.both).run()
|
||||
materialized.failed.futureValue.getCause shouldBe error
|
||||
completion.failed.futureValue.getCause shouldBe error
|
||||
}
|
||||
|
||||
"handle materialization failure" in {
|
||||
val error = new Error("boom")
|
||||
val flow = Flow.setup { (_, _) =>
|
||||
Flow[NotUsed].mapMaterializedValue(_ => throw error)
|
||||
}
|
||||
|
||||
val (materialized, completion) = Source.empty.viaMat(flow)(Keep.right).toMat(Sink.head)(Keep.both).run()
|
||||
materialized.failed.futureValue.getCause shouldBe error
|
||||
completion.failed.futureValue.getCause shouldBe error
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"Sink.setup" should {
|
||||
|
||||
"expose materializer" in {
|
||||
val sink = Sink.setup { (mat, _) =>
|
||||
Sink.fold(mat.isShutdown)(Keep.left)
|
||||
}
|
||||
|
||||
Source.empty.runWith(sink).flatten.futureValue shouldBe false
|
||||
}
|
||||
|
||||
"expose attributes" in {
|
||||
val sink = Sink.setup { (_, attr) =>
|
||||
Sink.fold(attr.attributeList)(Keep.left)
|
||||
}
|
||||
|
||||
Source.empty.runWith(sink).flatten.futureValue should not be empty
|
||||
}
|
||||
|
||||
"propagate materialized value" in {
|
||||
val sink = Sink.setup { (_, _) =>
|
||||
Sink.fold(NotUsed)(Keep.left)
|
||||
}
|
||||
|
||||
Source.empty.runWith(sink).flatten.futureValue shouldBe NotUsed
|
||||
}
|
||||
|
||||
"propagate attributes" in {
|
||||
val sink = Sink
|
||||
.setup { (_, attr) =>
|
||||
Sink.fold(attr.nameLifted)(Keep.left)
|
||||
}
|
||||
.named("my-name")
|
||||
|
||||
Source.empty.runWith(sink).flatten.futureValue shouldBe Some("my-name-setup")
|
||||
}
|
||||
|
||||
"propagate attributes when nested" in {
|
||||
val sink = Sink
|
||||
.setup { (_, _) =>
|
||||
Sink.setup { (_, attr) =>
|
||||
Sink.fold(attr.nameLifted)(Keep.left)
|
||||
}
|
||||
}
|
||||
.named("my-name")
|
||||
|
||||
Source.empty.runWith(sink).flatten.flatten.futureValue shouldBe Some("my-name-setup-setup")
|
||||
}
|
||||
|
||||
"handle factory failure" in {
|
||||
val error = new Error("boom")
|
||||
val sink = Sink.setup { (_, _) =>
|
||||
throw error
|
||||
}
|
||||
|
||||
Source.empty.runWith(sink).failed.futureValue.getCause shouldBe error
|
||||
}
|
||||
|
||||
"handle materialization failure" in {
|
||||
val error = new Error("boom")
|
||||
val sink = Sink.setup { (_, _) =>
|
||||
Sink.ignore.mapMaterializedValue(_ => throw error)
|
||||
}
|
||||
|
||||
Source.empty.runWith(sink).failed.futureValue.getCause shouldBe error
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -16,6 +16,7 @@
|
|||
# under the License.
|
||||
|
||||
# Remove deprecated methods
|
||||
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.stream.ActorMaterializerHelper*")
|
||||
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.stream.FanInShape1N")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.Materializer.schedulePeriodically")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.UniformFanInShape.inSeq")
|
||||
|
|
@ -29,6 +30,22 @@ ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.stream.javadsl.Cou
|
|||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.BidiFlow.bidirectionalIdleTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.FileIO.fromFile")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.FileIO.toFile")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.lazyInitAsync")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.lazyInit")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.setup")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.groupedWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.groupedWeightedWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.delay")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.dropWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.takeWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.initialTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.completionTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.idleTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.backpressureTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.keepAlive")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.throttle")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.throttleEven")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Flow.initialDelay")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.RestartFlow.onFailuresWithBackoff")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.RestartFlow.withBackoff")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.RestartSink.withBackoff")
|
||||
|
|
@ -36,13 +53,65 @@ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.java
|
|||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.RestartSource.withBackoff")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Sink.actorRefWithAck")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Sink.foreachParallel")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Sink.lazyInitAsync")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Sink.lazyInit")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Sink.setup")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.actorRefWithAck")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.setup")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.lazilyAsync")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.lazily")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.tick")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.groupedWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.groupedWeightedWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.delay")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.dropWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.takeWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.initialTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.completionTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.idleTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.backpressureTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.keepAlive")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.throttle")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.throttleEven")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.Source.initialDelay")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.groupedWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.groupedWeightedWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.delay")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.dropWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.takeWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.initialTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.completionTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.idleTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.backpressureTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.keepAlive")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.throttle")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.throttleEven")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubFlow.initialDelay")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.groupedWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.groupedWeightedWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.dropWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.delay")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.takeWithin")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.initialTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.completionTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.idleTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.backpressureTimeout")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.keepAlive")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.throttle")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.throttleEven")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.javadsl.SubSource.initialDelay")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Balance.this")
|
||||
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.stream.scaladsl.CoupledTerminationFlow*")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.FileIO.fromFile*")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.FileIO.toFile*")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Flow.lazyInitAsync")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Flow.lazyInit")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Flow.setup")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Flow.throttleEven")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.FlowOps.throttleEven")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.GraphApply.create")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.GraphDSL.create")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.GraphDSL#Implicits#PortOpsImpl.throttleEven")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.MergeHub#MergedSourceLogic.onDownstreamFinish")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Partition.this")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.RestartFlow.onFailuresWithBackoff")
|
||||
|
|
@ -50,9 +119,18 @@ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scal
|
|||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.RestartSink.withBackoff")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.RestartSource.onFailuresWithBackoff")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.RestartSource.withBackoff")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Sink.foreachParallel")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Sink.actorRefWithAck*")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Sink.foreachParallel")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Sink.lazyInitAsync")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Sink.lazyInit")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Sink.setup")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Sink.lazyInit")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Sink.lazyInitAsync")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Source.actorRefWithAck")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Source.lazilyAsync")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Source.lazily")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Source.setup")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.Source.throttleEven")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.scaladsl.ZipWithN.inSeq")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.stage.AbstractInOutHandler.onDownstreamFinish")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.pekko.stream.stage.AbstractOutHandler.onDownstreamFinish")
|
||||
|
|
|
|||
|
|
@ -197,25 +197,6 @@ object ActorMaterializer {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[pekko] object ActorMaterializerHelper {
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@deprecated("The Materializer now has all methods the ActorMaterializer used to have", "Akka 2.6.0")
|
||||
private[pekko] def downcast(materializer: Materializer): ActorMaterializer =
|
||||
materializer match {
|
||||
case m: ActorMaterializer => m
|
||||
case _ =>
|
||||
throw new IllegalArgumentException(
|
||||
s"required [${classOf[ActorMaterializer].getName}] " +
|
||||
s"but got [${materializer.getClass.getName}]")
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* An ActorMaterializer takes a stream blueprint and turns it into a running stream.
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -19,10 +19,9 @@ import java.util.concurrent.CompletionStage
|
|||
import java.util.function.BiFunction
|
||||
import java.util.function.Supplier
|
||||
|
||||
import scala.annotation.{ nowarn, varargs }
|
||||
import scala.annotation.varargs
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
import org.apache.pekko
|
||||
|
|
@ -116,16 +115,6 @@ object Flow {
|
|||
factory: BiFunction[Materializer, Attributes, Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] =
|
||||
scaladsl.Flow.fromMaterializer((mat, attr) => factory(mat, attr).asScala).mapMaterializedValue(_.asJava).asJava
|
||||
|
||||
/**
|
||||
* Defers the creation of a [[Flow]] until materialization. The `factory` function
|
||||
* exposes [[ActorMaterializer]] which is going to be used during materialization and
|
||||
* [[Attributes]] of the [[Flow]] returned by this method.
|
||||
*/
|
||||
@deprecated("Use 'fromMaterializer' instead", "Akka 2.6.0")
|
||||
def setup[I, O, M](
|
||||
factory: BiFunction[ActorMaterializer, Attributes, Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] =
|
||||
scaladsl.Flow.setup((mat, attr) => factory(mat, attr).asScala).mapMaterializedValue(_.asJava).asJava
|
||||
|
||||
/**
|
||||
* Creates a `Flow` from a `Sink` and a `Source` where the Flow's input
|
||||
* will be sent to the Sink and the Flow's output will come from the Source.
|
||||
|
|
@ -279,61 +268,6 @@ object Flow {
|
|||
combine: function.Function2[M1, M2, M]): Flow[I, O, M] =
|
||||
new Flow(scaladsl.Flow.fromSinkAndSourceCoupledMat(sink, source)(combinerToScala(combine)))
|
||||
|
||||
/**
|
||||
* Creates a real `Flow` upon receiving the first element. Internal `Flow` will not be created
|
||||
* if there are no elements, because of completion, cancellation, or error.
|
||||
*
|
||||
* The materialized value of the `Flow` is the value that is created by the `fallback` function.
|
||||
*
|
||||
* '''Emits when''' the internal flow is successfully created and it emits
|
||||
*
|
||||
* '''Backpressures when''' the internal flow is successfully created and it backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated(
|
||||
"Use 'Flow.completionStageFlow' in combination with prefixAndTail(1) instead, see `completionStageFlow` operator docs for details",
|
||||
"Akka 2.6.0")
|
||||
def lazyInit[I, O, M](
|
||||
flowFactory: function.Function[I, CompletionStage[Flow[I, O, M]]],
|
||||
fallback: function.Creator[M]): Flow[I, O, M] = {
|
||||
import pekko.util.FutureConverters._
|
||||
val sflow = scaladsl.Flow.lazyInit(
|
||||
(flowFactory.apply(_)).andThen(_.asScala.map(_.asScala)(ExecutionContexts.parasitic)),
|
||||
fallback.create _)
|
||||
new Flow(sflow)
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a real `Flow` upon receiving the first element. Internal `Flow` will not be created
|
||||
* if there are no elements, because of completion, cancellation, or error.
|
||||
*
|
||||
* The materialized value of the `Flow` is a `Future[Option[M]]` that is completed with `Some(mat)` when the internal
|
||||
* flow gets materialized or with `None` when there where no elements. If the flow materialization (including
|
||||
* the call of the `flowFactory`) fails then the future is completed with a failure.
|
||||
*
|
||||
* '''Emits when''' the internal flow is successfully created and it emits
|
||||
*
|
||||
* '''Backpressures when''' the internal flow is successfully created and it backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use 'Flow.lazyCompletionStageFlow' instead", "Akka 2.6.0")
|
||||
def lazyInitAsync[I, O, M](
|
||||
flowFactory: function.Creator[CompletionStage[Flow[I, O, M]]]): Flow[I, O, CompletionStage[Optional[M]]] = {
|
||||
import pekko.util.FutureConverters._
|
||||
|
||||
val sflow = scaladsl.Flow
|
||||
.lazyInitAsync(() => flowFactory.create().asScala.map(_.asScala)(ExecutionContexts.parasitic))
|
||||
.mapMaterializedValue(fut =>
|
||||
fut.map(_.fold[Optional[M]](Optional.empty())(m => Optional.ofNullable(m)))(ExecutionContexts.parasitic).asJava)
|
||||
new Flow(sflow)
|
||||
}
|
||||
|
||||
/**
|
||||
* Turn a `CompletionStage<Flow>` into a flow that will consume the values of the source when the future completes successfully.
|
||||
* If the `Future` is completed with a failure the stream is failed.
|
||||
|
|
@ -1675,31 +1609,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
* `maxNumber` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def groupedWithin(maxNumber: Int, duration: FiniteDuration): javadsl.Flow[In, java.util.List[Out], Mat] =
|
||||
new Flow(delegate.groupedWithin(maxNumber, duration).map(_.asJava)) // TODO optimize to one step
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
* or limited by the given number of elements, whatever happens first.
|
||||
* Empty groups will not be emitted if no elements are received from upstream.
|
||||
* The last group before end-of-stream will contain the buffered elements
|
||||
* since the previously emitted group.
|
||||
*
|
||||
* '''Emits when''' the configured time elapses since the last group has been emitted or `n` elements is buffered
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures, and there are `n+1` buffered elements
|
||||
*
|
||||
* '''Completes when''' upstream completes (emits last group)
|
||||
*
|
||||
* '''Cancels when''' downstream completes
|
||||
*
|
||||
* `maxNumber` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def groupedWithin(maxNumber: Int, duration: java.time.Duration): javadsl.Flow[In, java.util.List[Out], Mat] =
|
||||
groupedWithin(maxNumber, duration.asScala)
|
||||
new Flow(delegate.groupedWithin(maxNumber, duration.asScala).map(_.asJava)) // TODO optimize to one step
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
|
|
@ -1719,37 +1630,11 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
* `maxWeight` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def groupedWeightedWithin(
|
||||
maxWeight: Long,
|
||||
costFn: function.Function[Out, java.lang.Long],
|
||||
duration: FiniteDuration): javadsl.Flow[In, java.util.List[Out], Mat] =
|
||||
new Flow(delegate.groupedWeightedWithin(maxWeight, duration)(costFn.apply).map(_.asJava))
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
* or limited by the weight of the elements, whatever happens first.
|
||||
* Empty groups will not be emitted if no elements are received from upstream.
|
||||
* The last group before end-of-stream will contain the buffered elements
|
||||
* since the previously emitted group.
|
||||
*
|
||||
* '''Emits when''' the configured time elapses since the last group has been emitted or weight limit reached
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures, and buffered group (+ pending element) weighs more than `maxWeight`
|
||||
*
|
||||
* '''Completes when''' upstream completes (emits last group)
|
||||
*
|
||||
* '''Cancels when''' downstream completes
|
||||
*
|
||||
* `maxWeight` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def groupedWeightedWithin(
|
||||
maxWeight: Long,
|
||||
costFn: function.Function[Out, java.lang.Long],
|
||||
duration: java.time.Duration): javadsl.Flow[In, java.util.List[Out], Mat] =
|
||||
groupedWeightedWithin(maxWeight, costFn, duration.asScala)
|
||||
new Flow(delegate.groupedWeightedWithin(maxWeight, duration.asScala)(costFn.apply).map(_.asJava))
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
|
|
@ -1802,38 +1687,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
* @param of time to shift all messages
|
||||
* @param strategy Strategy that is used when incoming elements cannot fit inside the buffer
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def delay(of: FiniteDuration, strategy: DelayOverflowStrategy): Flow[In, Out, Mat] =
|
||||
new Flow(delegate.delay(of, strategy))
|
||||
|
||||
/**
|
||||
* Shifts elements emission in time by a specified amount. It allows to store elements
|
||||
* in internal buffer while waiting for next element to be emitted. Depending on the defined
|
||||
* [[pekko.stream.DelayOverflowStrategy]] it might drop elements or backpressure the upstream if
|
||||
* there is no space available in the buffer.
|
||||
*
|
||||
* Delay precision is 10ms to avoid unnecessary timer scheduling cycles
|
||||
*
|
||||
* Internal buffer has default capacity 16. You can set buffer size by calling `addAttributes(inputBuffer)`
|
||||
*
|
||||
* '''Emits when''' there is a pending element in the buffer and configured time for this element elapsed
|
||||
* * EmitEarly - strategy do not wait to emit element if buffer is full
|
||||
*
|
||||
* '''Backpressures when''' depending on OverflowStrategy
|
||||
* * Backpressure - backpressures when buffer is full
|
||||
* * DropHead, DropTail, DropBuffer - never backpressures
|
||||
* * Fail - fails the stream if buffer gets full
|
||||
*
|
||||
* '''Completes when''' upstream completes and buffered elements have been drained
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*
|
||||
* @param of time to shift all messages
|
||||
* @param strategy Strategy that is used when incoming elements cannot fit inside the buffer
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def delay(of: java.time.Duration, strategy: DelayOverflowStrategy): Flow[In, Out, Mat] =
|
||||
delay(of.asScala, strategy)
|
||||
new Flow(delegate.delay(of.asScala, strategy))
|
||||
|
||||
/**
|
||||
* Shifts elements emission in time by an amount individually determined through delay strategy a specified amount.
|
||||
|
|
@ -1897,24 +1752,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def dropWithin(duration: FiniteDuration): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.dropWithin(duration))
|
||||
|
||||
/**
|
||||
* Discard the elements received within the given duration at beginning of the stream.
|
||||
*
|
||||
* '''Emits when''' the specified time elapsed and a new upstream element arrives
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def dropWithin(duration: java.time.Duration): javadsl.Flow[In, Out, Mat] =
|
||||
dropWithin(duration.asScala)
|
||||
new Flow(delegate.dropWithin(duration.asScala))
|
||||
|
||||
/**
|
||||
* Terminate processing (and cancel the upstream publisher) after predicate
|
||||
|
|
@ -2292,32 +2131,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
*
|
||||
* See also [[Flow.limit]], [[Flow.limitWeighted]]
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def takeWithin(duration: FiniteDuration): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.takeWithin(duration))
|
||||
|
||||
/**
|
||||
* Terminate processing (and cancel the upstream publisher) after the given
|
||||
* duration. Due to input buffering some elements may have been
|
||||
* requested from upstream publishers that will then not be processed downstream
|
||||
* of this step.
|
||||
*
|
||||
* Note that this can be combined with [[#take]] to limit the number of elements
|
||||
* within the duration.
|
||||
*
|
||||
* '''Emits when''' an upstream element arrives
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or timer fires
|
||||
*
|
||||
* '''Cancels when''' downstream cancels or timer fires
|
||||
*
|
||||
* See also [[Flow.limit]], [[Flow.limitWeighted]]
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def takeWithin(duration: java.time.Duration): javadsl.Flow[In, Out, Mat] =
|
||||
takeWithin(duration.asScala)
|
||||
new Flow(delegate.takeWithin(duration.asScala))
|
||||
|
||||
/**
|
||||
* Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary
|
||||
|
|
@ -3844,25 +3659,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def initialTimeout(timeout: FiniteDuration): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.initialTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the first element has not passed through this operator before the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.InitialTimeoutException]].
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses before first element arrives
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def initialTimeout(timeout: java.time.Duration): javadsl.Flow[In, Out, Mat] =
|
||||
initialTimeout(timeout.asScala)
|
||||
new Flow(delegate.initialTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the completion of the stream does not happen until the provided timeout, the stream is failed
|
||||
|
|
@ -3876,25 +3674,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def completionTimeout(timeout: FiniteDuration): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.completionTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the completion of the stream does not happen until the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.CompletionTimeoutException]].
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses before upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def completionTimeout(timeout: java.time.Duration): javadsl.Flow[In, Out, Mat] =
|
||||
completionTimeout(timeout.asScala)
|
||||
new Flow(delegate.completionTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the time between two processed elements exceeds the provided timeout, the stream is failed
|
||||
|
|
@ -3909,26 +3690,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def idleTimeout(timeout: FiniteDuration): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.idleTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the time between two processed elements exceeds the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.StreamIdleTimeoutException]]. The timeout is checked periodically,
|
||||
* so the resolution of the check is one period (equals to timeout value).
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses between two emitted elements
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def idleTimeout(timeout: java.time.Duration): javadsl.Flow[In, Out, Mat] =
|
||||
idleTimeout(timeout.asScala)
|
||||
new Flow(delegate.idleTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the time between the emission of an element and the following downstream demand exceeds the provided timeout,
|
||||
|
|
@ -3943,26 +3706,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def backpressureTimeout(timeout: FiniteDuration): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.backpressureTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the time between the emission of an element and the following downstream demand exceeds the provided timeout,
|
||||
* the stream is failed with a [[org.apache.pekko.stream.BackpressureTimeoutException]]. The timeout is checked periodically,
|
||||
* so the resolution of the check is one period (equals to timeout value).
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses between element emission and downstream demand.
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def backpressureTimeout(timeout: java.time.Duration): javadsl.Flow[In, Out, Mat] =
|
||||
backpressureTimeout(timeout.asScala)
|
||||
new Flow(delegate.backpressureTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* Injects additional elements if upstream does not emit for a configured amount of time. In other words, this
|
||||
|
|
@ -3981,30 +3726,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def keepAlive(maxIdle: FiniteDuration, injectedElem: function.Creator[Out]): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.keepAlive(maxIdle, () => injectedElem.create()))
|
||||
|
||||
/**
|
||||
* Injects additional elements if upstream does not emit for a configured amount of time. In other words, this
|
||||
* operator attempts to maintains a base rate of emitted elements towards the downstream.
|
||||
*
|
||||
* If the downstream backpressures then no element is injected until downstream demand arrives. Injected elements
|
||||
* do not accumulate during this period.
|
||||
*
|
||||
* Upstream elements are always preferred over injected elements.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element or if the upstream was idle for the configured period
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def keepAlive(maxIdle: java.time.Duration, injectedElem: function.Creator[Out]): javadsl.Flow[In, Out, Mat] =
|
||||
keepAlive(maxIdle.asScala, injectedElem)
|
||||
new Flow(delegate.keepAlive(maxIdle.asScala, () => injectedElem.create()))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
|
|
@ -4038,45 +3761,6 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
def throttle(elements: Int, per: java.time.Duration): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.throttle(elements, per.asScala))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
* for emitting messages. This operator works for streams where all elements have the same cost or length.
|
||||
*
|
||||
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
|
||||
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
|
||||
* to allow some burstiness. Whenever stream wants to send an element, it takes as many
|
||||
* tokens from the bucket as element costs. If there isn't any, throttle waits until the
|
||||
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
|
||||
* to their cost minus available tokens, meeting the target rate. Bucket is full when stream just materialized and started.
|
||||
*
|
||||
* Parameter `mode` manages behavior when upstream is faster than throttle rate:
|
||||
* - [[pekko.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
|
||||
* - [[pekko.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate
|
||||
*
|
||||
* It is recommended to use non-zero burst sizes as they improve both performance and throttling precision by allowing
|
||||
* the implementation to avoid using the scheduler when input rates fall below the enforced limit and to reduce
|
||||
* most of the inaccuracy caused by the scheduler resolution (which is in the range of milliseconds).
|
||||
*
|
||||
* WARNING: Be aware that throttle is using scheduler to slow down the stream. This scheduler has minimal time of triggering
|
||||
* next push. Consequently it will slow down the stream as it has minimal pause for emitting. This can happen in
|
||||
* case burst is 0 and speed is higher than 30 events per second. You need to increase the `maximumBurst` if
|
||||
* elements arrive with small interval (30 milliseconds or less). Use the overloaded `throttle` method without
|
||||
* `maximumBurst` parameter to automatically calculate the `maximumBurst` based on the given rate (`cost/per`).
|
||||
* In other words the throttler always enforces the rate limit when `maximumBurst` parameter is given, but in
|
||||
* certain cases (mostly due to limited scheduler resolution) it enforces a tighter bound than what was prescribed.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element and configured time per each element elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or the incoming rate is higher than the speed limit
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttle(elements: Int, per: FiniteDuration, maximumBurst: Int, mode: ThrottleMode): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.throttle(elements, per, maximumBurst, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
* for emitting messages. This operator works for streams where all elements have the same cost or length.
|
||||
|
|
@ -4119,53 +3803,6 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
mode: ThrottleMode): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.throttle(elements, per.asScala, maximumBurst, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `cost/per`. Cost is
|
||||
* calculating for each element individually by calling `calculateCost` function.
|
||||
* This operator works for streams when elements have different cost(length).
|
||||
* Streams of `ByteString` for example.
|
||||
*
|
||||
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
|
||||
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
|
||||
* to allow some burstiness. Whenever stream wants to send an element, it takes as many
|
||||
* tokens from the bucket as element costs. If there isn't any, throttle waits until the
|
||||
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
|
||||
* to their cost minus available tokens, meeting the target rate. Bucket is full when stream just materialized and started.
|
||||
*
|
||||
* Parameter `mode` manages behavior when upstream is faster than throttle rate:
|
||||
* - [[pekko.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
|
||||
* - [[pekko.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate. Enforcing
|
||||
* cannot emit elements that cost more than the maximumBurst
|
||||
*
|
||||
* It is recommended to use non-zero burst sizes as they improve both performance and throttling precision by allowing
|
||||
* the implementation to avoid using the scheduler when input rates fall below the enforced limit and to reduce
|
||||
* most of the inaccuracy caused by the scheduler resolution (which is in the range of milliseconds).
|
||||
*
|
||||
* WARNING: Be aware that throttle is using scheduler to slow down the stream. This scheduler has minimal time of triggering
|
||||
* next push. Consequently it will slow down the stream as it has minimal pause for emitting. This can happen in
|
||||
* case burst is 0 and speed is higher than 30 events per second. You need to increase the `maximumBurst` if
|
||||
* elements arrive with small interval (30 milliseconds or less). Use the overloaded `throttle` method without
|
||||
* `maximumBurst` parameter to automatically calculate the `maximumBurst` based on the given rate (`cost/per`).
|
||||
* In other words the throttler always enforces the rate limit when `maximumBurst` parameter is given, but in
|
||||
* certain cases (mostly due to limited scheduler resolution) it enforces a tighter bound than what was prescribed.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element and configured time per each element elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or the incoming rate is higher than the speed limit
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttle(
|
||||
cost: Int,
|
||||
per: FiniteDuration,
|
||||
maximumBurst: Int,
|
||||
costCalculation: function.Function[Out, Integer],
|
||||
mode: ThrottleMode): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.throttle(cost, per, maximumBurst, costCalculation.apply, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `cost/per`. Cost is
|
||||
* calculating for each element individually by calling `calculateCost` function.
|
||||
|
|
@ -4249,70 +3886,6 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
mode: ThrottleMode): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.throttle(cost, per.asScala, maximumBurst, costCalculation.apply, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: FiniteDuration, mode: ThrottleMode): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.throttleEven(elements, per, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: java.time.Duration, mode: ThrottleMode): javadsl.Flow[In, Out, Mat] =
|
||||
throttleEven(elements, per.asScala, mode)
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(
|
||||
cost: Int,
|
||||
per: FiniteDuration,
|
||||
costCalculation: function.Function[Out, Integer],
|
||||
mode: ThrottleMode): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.throttleEven(cost, per, costCalculation.apply, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(
|
||||
cost: Int,
|
||||
per: java.time.Duration,
|
||||
costCalculation: function.Function[Out, Integer],
|
||||
mode: ThrottleMode): javadsl.Flow[In, Out, Mat] =
|
||||
throttleEven(cost, per.asScala, costCalculation, mode)
|
||||
|
||||
/**
|
||||
* Detaches upstream demand from downstream demand without detaching the
|
||||
* stream rates; in other words acts like a buffer of size 1.
|
||||
|
|
@ -4381,24 +3954,8 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def initialDelay(delay: FiniteDuration): javadsl.Flow[In, Out, Mat] =
|
||||
new Flow(delegate.initialDelay(delay))
|
||||
|
||||
/**
|
||||
* Delays the initial element by the specified duration.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element if the initial delay is already elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or initial delay is not yet elapsed
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def initialDelay(delay: java.time.Duration): javadsl.Flow[In, Out, Mat] =
|
||||
initialDelay(delay.asScala)
|
||||
new Flow(delegate.initialDelay(delay.asScala))
|
||||
|
||||
/**
|
||||
* Replace the attributes of this [[Flow]] with the given ones. If this Flow is a composite
|
||||
|
|
|
|||
|
|
@ -393,15 +393,6 @@ object Sink {
|
|||
def fromMaterializer[T, M](factory: BiFunction[Materializer, Attributes, Sink[T, M]]): Sink[T, CompletionStage[M]] =
|
||||
scaladsl.Sink.fromMaterializer((mat, attr) => factory(mat, attr).asScala).mapMaterializedValue(_.asJava).asJava
|
||||
|
||||
/**
|
||||
* Defers the creation of a [[Sink]] until materialization. The `factory` function
|
||||
* exposes [[ActorMaterializer]] which is going to be used during materialization and
|
||||
* [[Attributes]] of the [[Sink]] returned by this method.
|
||||
*/
|
||||
@deprecated("Use 'fromMaterializer' instead", "Akka 2.6.0")
|
||||
def setup[T, M](factory: BiFunction[ActorMaterializer, Attributes, Sink[T, M]]): Sink[T, CompletionStage[M]] =
|
||||
scaladsl.Sink.setup((mat, attr) => factory(mat, attr).asScala).mapMaterializedValue(_.asJava).asJava
|
||||
|
||||
/**
|
||||
* Combine several sinks with fan-out strategy like `Broadcast` or `Balance` and returns `Sink`.
|
||||
*/
|
||||
|
|
@ -487,45 +478,6 @@ object Sink {
|
|||
*/
|
||||
def queue[T](): Sink[T, SinkQueueWithCancel[T]] = queue(1)
|
||||
|
||||
/**
|
||||
* Creates a real `Sink` upon receiving the first element. Internal `Sink` will not be created if there are no elements,
|
||||
* because of completion or error.
|
||||
*
|
||||
* If upstream completes before an element was received then the `Future` is completed with the value created by fallback.
|
||||
* If upstream fails before an element was received, `sinkFactory` throws an exception, or materialization of the internal
|
||||
* sink fails then the `Future` is completed with the exception.
|
||||
* Otherwise the `Future` is completed with the materialized value of the internal sink.
|
||||
*/
|
||||
@deprecated("Use 'Sink.lazyCompletionStageSink' in combination with 'Flow.prefixAndTail(1)' instead", "Akka 2.6.0")
|
||||
def lazyInit[T, M](
|
||||
sinkFactory: function.Function[T, CompletionStage[Sink[T, M]]],
|
||||
fallback: function.Creator[M]): Sink[T, CompletionStage[M]] =
|
||||
new Sink(
|
||||
scaladsl.Sink
|
||||
.lazyInit[T, M](
|
||||
t => sinkFactory.apply(t).asScala.map(_.asScala)(ExecutionContexts.parasitic),
|
||||
() => fallback.create())
|
||||
.mapMaterializedValue(_.asJava))
|
||||
|
||||
/**
|
||||
* Creates a real `Sink` upon receiving the first element. Internal `Sink` will not be created if there are no elements,
|
||||
* because of completion or error.
|
||||
*
|
||||
* If upstream completes before an element was received then the `Future` is completed with `None`.
|
||||
* If upstream fails before an element was received, `sinkFactory` throws an exception, or materialization of the internal
|
||||
* sink fails then the `Future` is completed with the exception.
|
||||
* Otherwise the `Future` is completed with the materialized value of the internal sink.
|
||||
*/
|
||||
@deprecated("Use 'Sink.lazyCompletionStageSink' instead", "Akka 2.6.0")
|
||||
def lazyInitAsync[T, M](
|
||||
sinkFactory: function.Creator[CompletionStage[Sink[T, M]]]): Sink[T, CompletionStage[Optional[M]]] = {
|
||||
val sSink = scaladsl.Sink
|
||||
.lazyInitAsync[T, M](() => sinkFactory.create().asScala.map(_.asScala)(ExecutionContexts.parasitic))
|
||||
.mapMaterializedValue(fut =>
|
||||
fut.map(_.fold(Optional.empty[M]())(m => Optional.ofNullable(m)))(ExecutionContexts.parasitic).asJava)
|
||||
new Sink(sSink)
|
||||
}
|
||||
|
||||
/**
|
||||
* Turn a `Future[Sink]` into a Sink that will consume the values of the source when the future completes successfully.
|
||||
* If the `Future` is completed with a failure the stream is failed.
|
||||
|
|
|
|||
|
|
@ -22,7 +22,6 @@ import scala.annotation.{ nowarn, varargs }
|
|||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
import org.apache.pekko
|
||||
|
|
@ -271,20 +270,8 @@ object Source {
|
|||
* element is produced it will not receive that tick element later. It will
|
||||
* receive new tick elements as soon as it has requested more elements.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def tick[O](initialDelay: FiniteDuration, interval: FiniteDuration, tick: O): javadsl.Source[O, Cancellable] =
|
||||
new Source(scaladsl.Source.tick(initialDelay, interval, tick))
|
||||
|
||||
/**
|
||||
* Elements are emitted periodically with the specified interval.
|
||||
* The tick element will be delivered to downstream consumers that has requested any elements.
|
||||
* If a consumer has not requested any elements at the point in time when the tick
|
||||
* element is produced it will not receive that tick element later. It will
|
||||
* receive new tick elements as soon as it has requested more elements.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def tick[O](initialDelay: java.time.Duration, interval: java.time.Duration, tick: O): javadsl.Source[O, Cancellable] =
|
||||
Source.tick(initialDelay.asScala, interval.asScala, tick)
|
||||
new Source(scaladsl.Source.tick(initialDelay.asScala, interval.asScala, tick))
|
||||
|
||||
/**
|
||||
* Create a `Source` with one element.
|
||||
|
|
@ -339,26 +326,6 @@ object Source {
|
|||
def failed[T](cause: Throwable): Source[T, NotUsed] =
|
||||
new Source(scaladsl.Source.failed(cause))
|
||||
|
||||
/**
|
||||
* Creates a `Source` that is not materialized until there is downstream demand, when the source gets materialized
|
||||
* the materialized future is completed with its value, if downstream cancels or fails without any demand the
|
||||
* `create` factory is never called and the materialized `CompletionStage` is failed.
|
||||
*/
|
||||
@deprecated("Use 'Source.lazySource' instead", "Akka 2.6.0")
|
||||
def lazily[T, M](create: function.Creator[Source[T, M]]): Source[T, CompletionStage[M]] =
|
||||
scaladsl.Source.lazily[T, M](() => create.create().asScala).mapMaterializedValue(_.asJava).asJava
|
||||
|
||||
/**
|
||||
* Creates a `Source` from supplied future factory that is not called until downstream demand. When source gets
|
||||
* materialized the materialized future is completed with the value from the factory. If downstream cancels or fails
|
||||
* without any demand the create factory is never called and the materialized `Future` is failed.
|
||||
*
|
||||
* @see [[Source.lazily]]
|
||||
*/
|
||||
@deprecated("Use 'Source.lazyCompletionStage' instead", "Akka 2.6.0")
|
||||
def lazilyAsync[T](create: function.Creator[CompletionStage[T]]): Source[T, Future[NotUsed]] =
|
||||
scaladsl.Source.lazilyAsync[T](() => create.create().asScala).asJava
|
||||
|
||||
/**
|
||||
* Emits a single value when the given Scala `Future` is successfully completed and then completes the stream.
|
||||
* The stream fails if the `Future` is completed with a failure.
|
||||
|
|
@ -634,15 +601,6 @@ object Source {
|
|||
factory: BiFunction[Materializer, Attributes, Source[T, M]]): Source[T, CompletionStage[M]] =
|
||||
scaladsl.Source.fromMaterializer((mat, attr) => factory(mat, attr).asScala).mapMaterializedValue(_.asJava).asJava
|
||||
|
||||
/**
|
||||
* Defers the creation of a [[Source]] until materialization. The `factory` function
|
||||
* exposes [[ActorMaterializer]] which is going to be used during materialization and
|
||||
* [[Attributes]] of the [[Source]] returned by this method.
|
||||
*/
|
||||
@deprecated("Use 'fromMaterializer' instead", "Akka 2.6.0")
|
||||
def setup[T, M](factory: BiFunction[ActorMaterializer, Attributes, Source[T, M]]): Source[T, CompletionStage[M]] =
|
||||
scaladsl.Source.setup((mat, attr) => factory(mat, attr).asScala).mapMaterializedValue(_.asJava).asJava
|
||||
|
||||
/**
|
||||
* Combines several sources with fan-in strategy like [[Merge]] or [[Concat]] into a single [[Source]].
|
||||
*/
|
||||
|
|
@ -3381,35 +3339,10 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
* `maxNumber` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def groupedWithin(
|
||||
maxNumber: Int,
|
||||
duration: FiniteDuration): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
|
||||
new Source(delegate.groupedWithin(maxNumber, duration).map(_.asJava)) // TODO optimize to one step
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
* or limited by the given number of elements, whatever happens first.
|
||||
* Empty groups will not be emitted if no elements are received from upstream.
|
||||
* The last group before end-of-stream will contain the buffered elements
|
||||
* since the previously emitted group.
|
||||
*
|
||||
* '''Emits when''' the configured time elapses since the last group has been emitted or `n` elements is buffered
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures, and there are `n+1` buffered elements
|
||||
*
|
||||
* '''Completes when''' upstream completes (emits last group)
|
||||
*
|
||||
* '''Cancels when''' downstream completes
|
||||
*
|
||||
* `maxNumber` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def groupedWithin(
|
||||
maxNumber: Int,
|
||||
duration: java.time.Duration): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
|
||||
groupedWithin(maxNumber, duration.asScala)
|
||||
new Source(delegate.groupedWithin(maxNumber, duration.asScala).map(_.asJava)) // TODO optimize to one step
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
|
|
@ -3429,37 +3362,11 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
* `maxWeight` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def groupedWeightedWithin(
|
||||
maxWeight: Long,
|
||||
costFn: function.Function[Out, java.lang.Long],
|
||||
duration: FiniteDuration): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
|
||||
new Source(delegate.groupedWeightedWithin(maxWeight, duration)(costFn.apply).map(_.asJava))
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
* or limited by the weight of the elements, whatever happens first.
|
||||
* Empty groups will not be emitted if no elements are received from upstream.
|
||||
* The last group before end-of-stream will contain the buffered elements
|
||||
* since the previously emitted group.
|
||||
*
|
||||
* '''Emits when''' the configured time elapses since the last group has been emitted or weight limit reached
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures, and buffered group (+ pending element) weighs more than `maxWeight`
|
||||
*
|
||||
* '''Completes when''' upstream completes (emits last group)
|
||||
*
|
||||
* '''Cancels when''' downstream completes
|
||||
*
|
||||
* `maxWeight` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def groupedWeightedWithin(
|
||||
maxWeight: Long,
|
||||
costFn: function.Function[Out, java.lang.Long],
|
||||
duration: java.time.Duration): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
|
||||
groupedWeightedWithin(maxWeight, costFn, duration.asScala)
|
||||
new Source(delegate.groupedWeightedWithin(maxWeight, duration.asScala)(costFn.apply).map(_.asJava))
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
|
|
@ -3512,38 +3419,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
* @param of time to shift all messages
|
||||
* @param strategy Strategy that is used when incoming elements cannot fit inside the buffer
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def delay(of: FiniteDuration, strategy: DelayOverflowStrategy): Source[Out, Mat] =
|
||||
new Source(delegate.delay(of, strategy))
|
||||
|
||||
/**
|
||||
* Shifts elements emission in time by a specified amount. It allows to store elements
|
||||
* in internal buffer while waiting for next element to be emitted. Depending on the defined
|
||||
* [[pekko.stream.DelayOverflowStrategy]] it might drop elements or backpressure the upstream if
|
||||
* there is no space available in the buffer.
|
||||
*
|
||||
* Delay precision is 10ms to avoid unnecessary timer scheduling cycles
|
||||
*
|
||||
* Internal buffer has default capacity 16. You can set buffer size by calling `withAttributes(inputBuffer)`
|
||||
*
|
||||
* '''Emits when''' there is a pending element in the buffer and configured time for this element elapsed
|
||||
* * EmitEarly - strategy do not wait to emit element if buffer is full
|
||||
*
|
||||
* '''Backpressures when''' depending on OverflowStrategy
|
||||
* * Backpressure - backpressures when buffer is full
|
||||
* * DropHead, DropTail, DropBuffer - never backpressures
|
||||
* * Fail - fails the stream if buffer gets full
|
||||
*
|
||||
* '''Completes when''' upstream completes and buffered elements has been drained
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*
|
||||
* @param of time to shift all messages
|
||||
* @param strategy Strategy that is used when incoming elements cannot fit inside the buffer
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def delay(of: java.time.Duration, strategy: DelayOverflowStrategy): Source[Out, Mat] =
|
||||
delay(of.asScala, strategy)
|
||||
new Source(delegate.delay(of.asScala, strategy))
|
||||
|
||||
/**
|
||||
* Shifts elements emission in time by an amount individually determined through delay strategy a specified amount.
|
||||
|
|
@ -3607,24 +3484,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def dropWithin(duration: FiniteDuration): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.dropWithin(duration))
|
||||
|
||||
/**
|
||||
* Discard the elements received within the given duration at beginning of the stream.
|
||||
*
|
||||
* '''Emits when''' the specified time elapsed and a new upstream element arrives
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def dropWithin(duration: java.time.Duration): javadsl.Source[Out, Mat] =
|
||||
dropWithin(duration.asScala)
|
||||
new Source(delegate.dropWithin(duration.asScala))
|
||||
|
||||
/**
|
||||
* Terminate processing (and cancel the upstream publisher) after predicate
|
||||
|
|
@ -3745,30 +3606,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
*
|
||||
* '''Cancels when''' downstream cancels or timer fires
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def takeWithin(duration: FiniteDuration): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.takeWithin(duration))
|
||||
|
||||
/**
|
||||
* Terminate processing (and cancel the upstream publisher) after the given
|
||||
* duration. Due to input buffering some elements may have been
|
||||
* requested from upstream publishers that will then not be processed downstream
|
||||
* of this step.
|
||||
*
|
||||
* Note that this can be combined with [[#take]] to limit the number of elements
|
||||
* within the duration.
|
||||
*
|
||||
* '''Emits when''' an upstream element arrives
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or timer fires
|
||||
*
|
||||
* '''Cancels when''' downstream cancels or timer fires
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def takeWithin(duration: java.time.Duration): javadsl.Source[Out, Mat] =
|
||||
takeWithin(duration.asScala)
|
||||
new Source(delegate.takeWithin(duration.asScala))
|
||||
|
||||
/**
|
||||
* Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary
|
||||
|
|
@ -4381,25 +4220,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def initialTimeout(timeout: FiniteDuration): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.initialTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the first element has not passed through this operator before the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.InitialTimeoutException]].
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses before first element arrives
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def initialTimeout(timeout: java.time.Duration): javadsl.Source[Out, Mat] =
|
||||
initialTimeout(timeout.asScala)
|
||||
new Source(delegate.initialTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the completion of the stream does not happen until the provided timeout, the stream is failed
|
||||
|
|
@ -4413,25 +4235,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def completionTimeout(timeout: FiniteDuration): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.completionTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the completion of the stream does not happen until the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.CompletionTimeoutException]].
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses before upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def completionTimeout(timeout: java.time.Duration): javadsl.Source[Out, Mat] =
|
||||
completionTimeout(timeout.asScala)
|
||||
new Source(delegate.completionTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the time between two processed elements exceeds the provided timeout, the stream is failed
|
||||
|
|
@ -4446,26 +4251,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def idleTimeout(timeout: FiniteDuration): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.idleTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the time between two processed elements exceeds the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.StreamIdleTimeoutException]]. The timeout is checked periodically,
|
||||
* so the resolution of the check is one period (equals to timeout value).
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses between two emitted elements
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def idleTimeout(timeout: java.time.Duration): javadsl.Source[Out, Mat] =
|
||||
idleTimeout(timeout.asScala)
|
||||
new Source(delegate.idleTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the time between the emission of an element and the following downstream demand exceeds the provided timeout,
|
||||
|
|
@ -4480,26 +4267,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def backpressureTimeout(timeout: FiniteDuration): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.backpressureTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the time between the emission of an element and the following downstream demand exceeds the provided timeout,
|
||||
* the stream is failed with a [[org.apache.pekko.stream.BackpressureTimeoutException]]. The timeout is checked periodically,
|
||||
* so the resolution of the check is one period (equals to timeout value).
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses between element emission and downstream demand.
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def backpressureTimeout(timeout: java.time.Duration): javadsl.Source[Out, Mat] =
|
||||
backpressureTimeout(timeout.asScala)
|
||||
new Source(delegate.backpressureTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* Injects additional elements if upstream does not emit for a configured amount of time. In other words, this
|
||||
|
|
@ -4518,30 +4287,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def keepAlive(maxIdle: FiniteDuration, injectedElem: function.Creator[Out]): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.keepAlive(maxIdle, () => injectedElem.create()))
|
||||
|
||||
/**
|
||||
* Injects additional elements if upstream does not emit for a configured amount of time. In other words, this
|
||||
* operator attempts to maintains a base rate of emitted elements towards the downstream.
|
||||
*
|
||||
* If the downstream backpressures then no element is injected until downstream demand arrives. Injected elements
|
||||
* do not accumulate during this period.
|
||||
*
|
||||
* Upstream elements are always preferred over injected elements.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element or if the upstream was idle for the configured period
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def keepAlive(maxIdle: java.time.Duration, injectedElem: function.Creator[Out]): javadsl.Source[Out, Mat] =
|
||||
keepAlive(maxIdle.asScala, injectedElem)
|
||||
new Source(delegate.keepAlive(maxIdle.asScala, () => injectedElem.create()))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
|
|
@ -4575,45 +4322,6 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
def throttle(elements: Int, per: java.time.Duration): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.throttle(elements, per.asScala))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
* for emitting messages. This operator works for streams where all elements have the same cost or length.
|
||||
*
|
||||
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
|
||||
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
|
||||
* to allow some burstiness. Whenever stream wants to send an element, it takes as many
|
||||
* tokens from the bucket as element costs. If there isn't any, throttle waits until the
|
||||
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
|
||||
* to their cost minus available tokens, meeting the target rate. Bucket is full when stream just materialized and started.
|
||||
*
|
||||
* Parameter `mode` manages behavior when upstream is faster than throttle rate:
|
||||
* - [[pekko.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
|
||||
* - [[pekko.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate
|
||||
*
|
||||
* It is recommended to use non-zero burst sizes as they improve both performance and throttling precision by allowing
|
||||
* the implementation to avoid using the scheduler when input rates fall below the enforced limit and to reduce
|
||||
* most of the inaccuracy caused by the scheduler resolution (which is in the range of milliseconds).
|
||||
*
|
||||
* WARNING: Be aware that throttle is using scheduler to slow down the stream. This scheduler has minimal time of triggering
|
||||
* next push. Consequently it will slow down the stream as it has minimal pause for emitting. This can happen in
|
||||
* case burst is 0 and speed is higher than 30 events per second. You need to increase the `maximumBurst` if
|
||||
* elements arrive with small interval (30 milliseconds or less). Use the overloaded `throttle` method without
|
||||
* `maximumBurst` parameter to automatically calculate the `maximumBurst` based on the given rate (`cost/per`).
|
||||
* In other words the throttler always enforces the rate limit when `maximumBurst` parameter is given, but in
|
||||
* certain cases (mostly due to limited scheduler resolution) it enforces a tighter bound than what was prescribed.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element and configured time per each element elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or the incoming rate is higher than the speed limit
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttle(elements: Int, per: FiniteDuration, maximumBurst: Int, mode: ThrottleMode): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.throttle(elements, per, maximumBurst, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
* for emitting messages. This operator works for streams where all elements have the same cost or length.
|
||||
|
|
@ -4693,53 +4401,6 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
costCalculation: function.Function[Out, Integer]): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.throttle(cost, per.asScala, costCalculation.apply _))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `cost/per`. Cost is
|
||||
* calculating for each element individually by calling `calculateCost` function.
|
||||
* This operator works for streams when elements have different cost(length).
|
||||
* Streams of `ByteString` for example.
|
||||
*
|
||||
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
|
||||
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
|
||||
* to allow some burstiness. Whenever stream wants to send an element, it takes as many
|
||||
* tokens from the bucket as element costs. If there isn't any, throttle waits until the
|
||||
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
|
||||
* to their cost minus available tokens, meeting the target rate. Bucket is full when stream just materialized and started.
|
||||
*
|
||||
* Parameter `mode` manages behavior when upstream is faster than throttle rate:
|
||||
* - [[pekko.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
|
||||
* - [[pekko.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate. Enforcing
|
||||
* cannot emit elements that cost more than the maximumBurst
|
||||
*
|
||||
* It is recommended to use non-zero burst sizes as they improve both performance and throttling precision by allowing
|
||||
* the implementation to avoid using the scheduler when input rates fall below the enforced limit and to reduce
|
||||
* most of the inaccuracy caused by the scheduler resolution (which is in the range of milliseconds).
|
||||
*
|
||||
* WARNING: Be aware that throttle is using scheduler to slow down the stream. This scheduler has minimal time of triggering
|
||||
* next push. Consequently it will slow down the stream as it has minimal pause for emitting. This can happen in
|
||||
* case burst is 0 and speed is higher than 30 events per second. You need to increase the `maximumBurst` if
|
||||
* elements arrive with small interval (30 milliseconds or less). Use the overloaded `throttle` method without
|
||||
* `maximumBurst` parameter to automatically calculate the `maximumBurst` based on the given rate (`cost/per`).
|
||||
* In other words the throttler always enforces the rate limit when `maximumBurst` parameter is given, but in
|
||||
* certain cases (mostly due to limited scheduler resolution) it enforces a tighter bound than what was prescribed.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element and configured time per each element elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or the incoming rate is higher than the speed limit
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttle(
|
||||
cost: Int,
|
||||
per: FiniteDuration,
|
||||
maximumBurst: Int,
|
||||
costCalculation: function.Function[Out, Integer],
|
||||
mode: ThrottleMode): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.throttle(cost, per, maximumBurst, costCalculation.apply _, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `cost/per`. Cost is
|
||||
* calculating for each element individually by calling `calculateCost` function.
|
||||
|
|
@ -4786,70 +4447,6 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
mode: ThrottleMode): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.throttle(cost, per.asScala, maximumBurst, costCalculation.apply _, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: FiniteDuration, mode: ThrottleMode): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.throttleEven(elements, per, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: java.time.Duration, mode: ThrottleMode): javadsl.Source[Out, Mat] =
|
||||
throttleEven(elements, per.asScala, mode)
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(
|
||||
cost: Int,
|
||||
per: FiniteDuration,
|
||||
costCalculation: (Out) => Int,
|
||||
mode: ThrottleMode): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.throttleEven(cost, per, costCalculation.apply _, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(
|
||||
cost: Int,
|
||||
per: java.time.Duration,
|
||||
costCalculation: (Out) => Int,
|
||||
mode: ThrottleMode): javadsl.Source[Out, Mat] =
|
||||
throttleEven(cost, per.asScala, costCalculation, mode)
|
||||
|
||||
/**
|
||||
* Detaches upstream demand from downstream demand without detaching the
|
||||
* stream rates; in other words acts like a buffer of size 1.
|
||||
|
|
@ -4916,24 +4513,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def initialDelay(delay: FiniteDuration): javadsl.Source[Out, Mat] =
|
||||
new Source(delegate.initialDelay(delay))
|
||||
|
||||
/**
|
||||
* Delays the initial element by the specified duration.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element if the initial delay is already elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or initial delay is not yet elapsed
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def initialDelay(delay: java.time.Duration): javadsl.Source[Out, Mat] =
|
||||
initialDelay(delay.asScala)
|
||||
new Source(delegate.initialDelay(delay.asScala))
|
||||
|
||||
/**
|
||||
* Replace the attributes of this [[Source]] with the given ones. If this Source is a composite
|
||||
|
|
|
|||
|
|
@ -17,10 +17,9 @@ import java.util.{ Comparator, Optional }
|
|||
import java.util.concurrent.CompletionStage
|
||||
import java.util.function.Supplier
|
||||
|
||||
import scala.annotation.{ nowarn, varargs }
|
||||
import scala.annotation.varargs
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
import org.apache.pekko
|
||||
|
|
@ -1024,35 +1023,10 @@ final class SubFlow[In, Out, Mat](
|
|||
* `maxNumber` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def groupedWithin(
|
||||
maxNumber: Int,
|
||||
duration: FiniteDuration): SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
|
||||
new SubFlow(delegate.groupedWithin(maxNumber, duration).map(_.asJava)) // TODO optimize to one step
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
* or limited by the given number of elements, whatever happens first.
|
||||
* Empty groups will not be emitted if no elements are received from upstream.
|
||||
* The last group before end-of-stream will contain the buffered elements
|
||||
* since the previously emitted group.
|
||||
*
|
||||
* '''Emits when''' the configured time elapses since the last group has been emitted or `n` elements is buffered
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures, and there are `n+1` buffered elements
|
||||
*
|
||||
* '''Completes when''' upstream completes (emits last group)
|
||||
*
|
||||
* '''Cancels when''' downstream completes
|
||||
*
|
||||
* `maxNumber` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def groupedWithin(
|
||||
maxNumber: Int,
|
||||
duration: java.time.Duration): SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
|
||||
groupedWithin(maxNumber, duration.asScala)
|
||||
new SubFlow(delegate.groupedWithin(maxNumber, duration.asScala).map(_.asJava)) // TODO optimize to one step
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
|
|
@ -1072,37 +1046,11 @@ final class SubFlow[In, Out, Mat](
|
|||
* `maxWeight` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def groupedWeightedWithin(
|
||||
maxWeight: Long,
|
||||
costFn: function.Function[Out, java.lang.Long],
|
||||
duration: FiniteDuration): javadsl.SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
|
||||
new SubFlow(delegate.groupedWeightedWithin(maxWeight, duration)(costFn.apply).map(_.asJava))
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
* or limited by the weight of the elements, whatever happens first.
|
||||
* Empty groups will not be emitted if no elements are received from upstream.
|
||||
* The last group before end-of-stream will contain the buffered elements
|
||||
* since the previously emitted group.
|
||||
*
|
||||
* '''Emits when''' the configured time elapses since the last group has been emitted or weight limit reached
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures, and buffered group (+ pending element) weighs more than `maxWeight`
|
||||
*
|
||||
* '''Completes when''' upstream completes (emits last group)
|
||||
*
|
||||
* '''Cancels when''' downstream completes
|
||||
*
|
||||
* `maxWeight` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def groupedWeightedWithin(
|
||||
maxWeight: Long,
|
||||
costFn: function.Function[Out, java.lang.Long],
|
||||
duration: java.time.Duration): javadsl.SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
|
||||
groupedWeightedWithin(maxWeight, costFn, duration.asScala)
|
||||
new SubFlow(delegate.groupedWeightedWithin(maxWeight, duration.asScala)(costFn.apply).map(_.asJava))
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
|
|
@ -1155,38 +1103,8 @@ final class SubFlow[In, Out, Mat](
|
|||
* @param of time to shift all messages
|
||||
* @param strategy Strategy that is used when incoming elements cannot fit inside the buffer
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def delay(of: FiniteDuration, strategy: DelayOverflowStrategy): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.delay(of, strategy))
|
||||
|
||||
/**
|
||||
* Shifts elements emission in time by a specified amount. It allows to store elements
|
||||
* in internal buffer while waiting for next element to be emitted. Depending on the defined
|
||||
* [[pekko.stream.DelayOverflowStrategy]] it might drop elements or backpressure the upstream if
|
||||
* there is no space available in the buffer.
|
||||
*
|
||||
* Delay precision is 10ms to avoid unnecessary timer scheduling cycles
|
||||
*
|
||||
* Internal buffer has default capacity 16. You can set buffer size by calling `withAttributes(inputBuffer)`
|
||||
*
|
||||
* '''Emits when''' there is a pending element in the buffer and configured time for this element elapsed
|
||||
* * EmitEarly - strategy do not wait to emit element if buffer is full
|
||||
*
|
||||
* '''Backpressures when''' depending on OverflowStrategy
|
||||
* * Backpressure - backpressures when buffer is full
|
||||
* * DropHead, DropTail, DropBuffer - never backpressures
|
||||
* * Fail - fails the stream if buffer gets full
|
||||
*
|
||||
* '''Completes when''' upstream completes and buffered elements has been drained
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*
|
||||
* @param of time to shift all messages
|
||||
* @param strategy Strategy that is used when incoming elements cannot fit inside the buffer
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def delay(of: java.time.Duration, strategy: DelayOverflowStrategy): SubFlow[In, Out, Mat] =
|
||||
delay(of.asScala, strategy)
|
||||
new SubFlow(delegate.delay(of.asScala, strategy))
|
||||
|
||||
/**
|
||||
* Shifts elements emission in time by an amount individually determined through delay strategy a specified amount.
|
||||
|
|
@ -1250,24 +1168,8 @@ final class SubFlow[In, Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def dropWithin(duration: FiniteDuration): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.dropWithin(duration))
|
||||
|
||||
/**
|
||||
* Discard the elements received within the given duration at beginning of the stream.
|
||||
*
|
||||
* '''Emits when''' the specified time elapsed and a new upstream element arrives
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def dropWithin(duration: java.time.Duration): SubFlow[In, Out, Mat] =
|
||||
dropWithin(duration.asScala)
|
||||
new SubFlow(delegate.dropWithin(duration.asScala))
|
||||
|
||||
/**
|
||||
* Terminate processing (and cancel the upstream publisher) after predicate
|
||||
|
|
@ -1553,30 +1455,8 @@ final class SubFlow[In, Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels or timer fires
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def takeWithin(duration: FiniteDuration): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.takeWithin(duration))
|
||||
|
||||
/**
|
||||
* Terminate processing (and cancel the upstream publisher) after the given
|
||||
* duration. Due to input buffering some elements may have been
|
||||
* requested from upstream publishers that will then not be processed downstream
|
||||
* of this step.
|
||||
*
|
||||
* Note that this can be combined with [[#take]] to limit the number of elements
|
||||
* within the duration.
|
||||
*
|
||||
* '''Emits when''' an upstream element arrives
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or timer fires
|
||||
*
|
||||
* '''Cancels when''' downstream cancels or timer fires
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def takeWithin(duration: java.time.Duration): SubFlow[In, Out, Mat] =
|
||||
takeWithin(duration.asScala)
|
||||
new SubFlow(delegate.takeWithin(duration.asScala))
|
||||
|
||||
/**
|
||||
* Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary
|
||||
|
|
@ -2427,25 +2307,8 @@ final class SubFlow[In, Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def initialTimeout(timeout: FiniteDuration): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.initialTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the first element has not passed through this operator before the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.InitialTimeoutException]].
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses before first element arrives
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def initialTimeout(timeout: java.time.Duration): SubFlow[In, Out, Mat] =
|
||||
initialTimeout(timeout.asScala)
|
||||
new SubFlow(delegate.initialTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the completion of the stream does not happen until the provided timeout, the stream is failed
|
||||
|
|
@ -2459,25 +2322,8 @@ final class SubFlow[In, Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def completionTimeout(timeout: FiniteDuration): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.completionTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the completion of the stream does not happen until the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.CompletionTimeoutException]].
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses before upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def completionTimeout(timeout: java.time.Duration): SubFlow[In, Out, Mat] =
|
||||
completionTimeout(timeout.asScala)
|
||||
new SubFlow(delegate.completionTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the time between two processed elements exceeds the provided timeout, the stream is failed
|
||||
|
|
@ -2492,26 +2338,8 @@ final class SubFlow[In, Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def idleTimeout(timeout: FiniteDuration): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.idleTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the time between two processed elements exceeds the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.StreamIdleTimeoutException]]. The timeout is checked periodically,
|
||||
* so the resolution of the check is one period (equals to timeout value).
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses between two emitted elements
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def idleTimeout(timeout: java.time.Duration): SubFlow[In, Out, Mat] =
|
||||
idleTimeout(timeout.asScala)
|
||||
new SubFlow(delegate.idleTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the time between the emission of an element and the following downstream demand exceeds the provided timeout,
|
||||
|
|
@ -2526,26 +2354,8 @@ final class SubFlow[In, Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def backpressureTimeout(timeout: FiniteDuration): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.backpressureTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the time between the emission of an element and the following downstream demand exceeds the provided timeout,
|
||||
* the stream is failed with a [[org.apache.pekko.stream.BackpressureTimeoutException]]. The timeout is checked periodically,
|
||||
* so the resolution of the check is one period (equals to timeout value).
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses between element emission and downstream demand.
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def backpressureTimeout(timeout: java.time.Duration): SubFlow[In, Out, Mat] =
|
||||
backpressureTimeout(timeout.asScala)
|
||||
new SubFlow(delegate.backpressureTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* Injects additional elements if upstream does not emit for a configured amount of time. In other words, this
|
||||
|
|
@ -2564,30 +2374,8 @@ final class SubFlow[In, Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def keepAlive(maxIdle: FiniteDuration, injectedElem: function.Creator[Out]): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.keepAlive(maxIdle, () => injectedElem.create()))
|
||||
|
||||
/**
|
||||
* Injects additional elements if upstream does not emit for a configured amount of time. In other words, this
|
||||
* operator attempts to maintains a base rate of emitted elements towards the downstream.
|
||||
*
|
||||
* If the downstream backpressures then no element is injected until downstream demand arrives. Injected elements
|
||||
* do not accumulate during this period.
|
||||
*
|
||||
* Upstream elements are always preferred over injected elements.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element or if the upstream was idle for the configured period
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def keepAlive(maxIdle: java.time.Duration, injectedElem: function.Creator[Out]): SubFlow[In, Out, Mat] =
|
||||
keepAlive(maxIdle.asScala, injectedElem)
|
||||
new SubFlow(delegate.keepAlive(maxIdle.asScala, () => injectedElem.create()))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
|
|
@ -2621,49 +2409,6 @@ final class SubFlow[In, Out, Mat](
|
|||
def throttle(elements: Int, per: java.time.Duration): javadsl.SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.throttle(elements, per.asScala))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
* for emitting messages. This operator works for streams where all elements have the same cost or length.
|
||||
*
|
||||
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
|
||||
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
|
||||
* to allow some burstiness. Whenever stream wants to send an element, it takes as many
|
||||
* tokens from the bucket as element costs. If there isn't any, throttle waits until the
|
||||
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
|
||||
* to their cost minus available tokens, meeting the target rate. Bucket is full when stream just materialized and started.
|
||||
*
|
||||
* Parameter `mode` manages behavior when upstream is faster than throttle rate:
|
||||
* - [[pekko.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
|
||||
* - [[pekko.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate
|
||||
*
|
||||
* It is recommended to use non-zero burst sizes as they improve both performance and throttling precision by allowing
|
||||
* the implementation to avoid using the scheduler when input rates fall below the enforced limit and to reduce
|
||||
* most of the inaccuracy caused by the scheduler resolution (which is in the range of milliseconds).
|
||||
*
|
||||
* WARNING: Be aware that throttle is using scheduler to slow down the stream. This scheduler has minimal time of triggering
|
||||
* next push. Consequently it will slow down the stream as it has minimal pause for emitting. This can happen in
|
||||
* case burst is 0 and speed is higher than 30 events per second. You need to increase the `maximumBurst` if
|
||||
* elements arrive with small interval (30 milliseconds or less). Use the overloaded `throttle` method without
|
||||
* `maximumBurst` parameter to automatically calculate the `maximumBurst` based on the given rate (`cost/per`).
|
||||
* In other words the throttler always enforces the rate limit when `maximumBurst` parameter is given, but in
|
||||
* certain cases (mostly due to limited scheduler resolution) it enforces a tighter bound than what was prescribed.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element and configured time per each element elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or the incoming rate is higher than the speed limit
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttle(
|
||||
elements: Int,
|
||||
per: FiniteDuration,
|
||||
maximumBurst: Int,
|
||||
mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.throttle(elements, per, maximumBurst, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
* for emitting messages. This operator works for streams where all elements have the same cost or length.
|
||||
|
|
@ -2743,53 +2488,6 @@ final class SubFlow[In, Out, Mat](
|
|||
costCalculation: function.Function[Out, Integer]): javadsl.SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.throttle(cost, per.asScala, costCalculation.apply))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `cost/per`. Cost is
|
||||
* calculating for each element individually by calling `calculateCost` function.
|
||||
* This operator works for streams when elements have different cost(length).
|
||||
* Streams of `ByteString` for example.
|
||||
*
|
||||
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
|
||||
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
|
||||
* to allow some burstiness. Whenever stream wants to send an element, it takes as many
|
||||
* tokens from the bucket as element costs. If there isn't any, throttle waits until the
|
||||
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
|
||||
* to their cost minus available tokens, meeting the target rate. Bucket is full when stream just materialized and started.
|
||||
*
|
||||
* Parameter `mode` manages behavior when upstream is faster than throttle rate:
|
||||
* - [[pekko.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
|
||||
* - [[pekko.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate. Enforcing
|
||||
* cannot emit elements that cost more than the maximumBurst
|
||||
*
|
||||
* It is recommended to use non-zero burst sizes as they improve both performance and throttling precision by allowing
|
||||
* the implementation to avoid using the scheduler when input rates fall below the enforced limit and to reduce
|
||||
* most of the inaccuracy caused by the scheduler resolution (which is in the range of milliseconds).
|
||||
*
|
||||
* WARNING: Be aware that throttle is using scheduler to slow down the stream. This scheduler has minimal time of triggering
|
||||
* next push. Consequently it will slow down the stream as it has minimal pause for emitting. This can happen in
|
||||
* case burst is 0 and speed is higher than 30 events per second. You need to increase the `maximumBurst` if
|
||||
* elements arrive with small interval (30 milliseconds or less). Use the overloaded `throttle` method without
|
||||
* `maximumBurst` parameter to automatically calculate the `maximumBurst` based on the given rate (`cost/per`).
|
||||
* In other words the throttler always enforces the rate limit when `maximumBurst` parameter is given, but in
|
||||
* certain cases (mostly due to limited scheduler resolution) it enforces a tighter bound than what was prescribed.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element and configured time per each element elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or the incoming rate is higher than the speed limit
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttle(
|
||||
cost: Int,
|
||||
per: FiniteDuration,
|
||||
maximumBurst: Int,
|
||||
costCalculation: function.Function[Out, Integer],
|
||||
mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.throttle(cost, per, maximumBurst, costCalculation.apply, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `cost/per`. Cost is
|
||||
* calculating for each element individually by calling `calculateCost` function.
|
||||
|
|
@ -2836,70 +2534,6 @@ final class SubFlow[In, Out, Mat](
|
|||
mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.throttle(cost, per.asScala, maximumBurst, costCalculation.apply, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: FiniteDuration, mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.throttleEven(elements, per, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: java.time.Duration, mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
|
||||
throttleEven(elements, per.asScala, mode)
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(
|
||||
cost: Int,
|
||||
per: FiniteDuration,
|
||||
costCalculation: function.Function[Out, Integer],
|
||||
mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.throttleEven(cost, per, costCalculation.apply, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(
|
||||
cost: Int,
|
||||
per: java.time.Duration,
|
||||
costCalculation: function.Function[Out, Integer],
|
||||
mode: ThrottleMode): javadsl.SubFlow[In, Out, Mat] =
|
||||
throttleEven(cost, per.asScala, costCalculation, mode)
|
||||
|
||||
/**
|
||||
* Detaches upstream demand from downstream demand without detaching the
|
||||
* stream rates; in other words acts like a buffer of size 1.
|
||||
|
|
@ -2925,24 +2559,8 @@ final class SubFlow[In, Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def initialDelay(delay: FiniteDuration): SubFlow[In, Out, Mat] =
|
||||
new SubFlow(delegate.initialDelay(delay))
|
||||
|
||||
/**
|
||||
* Delays the initial element by the specified duration.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element if the initial delay is already elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or initial delay is not yet elapsed
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def initialDelay(delay: java.time.Duration): SubFlow[In, Out, Mat] =
|
||||
initialDelay(delay.asScala)
|
||||
new SubFlow(delegate.initialDelay(delay.asScala))
|
||||
|
||||
/**
|
||||
* Change the attributes of this [[Source]] to the given ones and seal the list
|
||||
|
|
|
|||
|
|
@ -17,10 +17,9 @@ import java.util.{ Comparator, Optional }
|
|||
import java.util.concurrent.CompletionStage
|
||||
import java.util.function.Supplier
|
||||
|
||||
import scala.annotation.{ nowarn, varargs }
|
||||
import scala.annotation.varargs
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
import org.apache.pekko
|
||||
|
|
@ -1010,33 +1009,10 @@ final class SubSource[Out, Mat](
|
|||
* `maxNumber` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def groupedWithin(maxNumber: Int, duration: FiniteDuration): SubSource[java.util.List[Out @uncheckedVariance], Mat] =
|
||||
new SubSource(delegate.groupedWithin(maxNumber, duration).map(_.asJava)) // TODO optimize to one step
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
* or limited by the given number of elements, whatever happens first.
|
||||
* Empty groups will not be emitted if no elements are received from upstream.
|
||||
* The last group before end-of-stream will contain the buffered elements
|
||||
* since the previously emitted group.
|
||||
*
|
||||
* '''Emits when''' the configured time elapses since the last group has been emitted or `n` elements is buffered
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures, and there are `n+1` buffered elements
|
||||
*
|
||||
* '''Completes when''' upstream completes (emits last group)
|
||||
*
|
||||
* '''Cancels when''' downstream completes
|
||||
*
|
||||
* `maxNumber` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def groupedWithin(
|
||||
maxNumber: Int,
|
||||
duration: java.time.Duration): SubSource[java.util.List[Out @uncheckedVariance], Mat] =
|
||||
groupedWithin(maxNumber, duration.asScala)
|
||||
new SubSource(delegate.groupedWithin(maxNumber, duration.asScala).map(_.asJava)) // TODO optimize to one step
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
|
|
@ -1056,37 +1032,11 @@ final class SubSource[Out, Mat](
|
|||
* `maxWeight` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def groupedWeightedWithin(
|
||||
maxWeight: Long,
|
||||
costFn: function.Function[Out, java.lang.Long],
|
||||
duration: FiniteDuration): javadsl.SubSource[java.util.List[Out @uncheckedVariance], Mat] =
|
||||
new SubSource(delegate.groupedWeightedWithin(maxWeight, duration)(costFn.apply).map(_.asJava))
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
* or limited by the weight of the elements, whatever happens first.
|
||||
* Empty groups will not be emitted if no elements are received from upstream.
|
||||
* The last group before end-of-stream will contain the buffered elements
|
||||
* since the previously emitted group.
|
||||
*
|
||||
* '''Emits when''' the configured time elapses since the last group has been emitted or weight limit reached
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures, and buffered group (+ pending element) weighs more than `maxWeight`
|
||||
*
|
||||
* '''Completes when''' upstream completes (emits last group)
|
||||
*
|
||||
* '''Cancels when''' downstream completes
|
||||
*
|
||||
* `maxWeight` must be positive, and `duration` must be greater than 0 seconds, otherwise
|
||||
* IllegalArgumentException is thrown.
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def groupedWeightedWithin(
|
||||
maxWeight: Long,
|
||||
costFn: function.Function[Out, java.lang.Long],
|
||||
duration: java.time.Duration): javadsl.SubSource[java.util.List[Out @uncheckedVariance], Mat] =
|
||||
groupedWeightedWithin(maxWeight, costFn, duration.asScala)
|
||||
new SubSource(delegate.groupedWeightedWithin(maxWeight, duration.asScala)(costFn.apply).map(_.asJava))
|
||||
|
||||
/**
|
||||
* Chunk up this stream into groups of elements received within a time window,
|
||||
|
|
@ -1140,24 +1090,8 @@ final class SubSource[Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def dropWithin(duration: FiniteDuration): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.dropWithin(duration))
|
||||
|
||||
/**
|
||||
* Discard the elements received within the given duration at beginning of the stream.
|
||||
*
|
||||
* '''Emits when''' the specified time elapsed and a new upstream element arrives
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def dropWithin(duration: java.time.Duration): SubSource[Out, Mat] =
|
||||
dropWithin(duration.asScala)
|
||||
new SubSource(delegate.dropWithin(duration.asScala))
|
||||
|
||||
/**
|
||||
* Terminate processing (and cancel the upstream publisher) after predicate
|
||||
|
|
@ -1261,38 +1195,8 @@ final class SubSource[Out, Mat](
|
|||
* @param of time to shift all messages
|
||||
* @param strategy Strategy that is used when incoming elements cannot fit inside the buffer
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def delay(of: FiniteDuration, strategy: DelayOverflowStrategy): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.delay(of, strategy))
|
||||
|
||||
/**
|
||||
* Shifts elements emission in time by a specified amount. It allows to store elements
|
||||
* in internal buffer while waiting for next element to be emitted. Depending on the defined
|
||||
* [[pekko.stream.DelayOverflowStrategy]] it might drop elements or backpressure the upstream if
|
||||
* there is no space available in the buffer.
|
||||
*
|
||||
* Delay precision is 10ms to avoid unnecessary timer scheduling cycles
|
||||
*
|
||||
* Internal buffer has default capacity 16. You can set buffer size by calling `withAttributes(inputBuffer)`
|
||||
*
|
||||
* '''Emits when''' there is a pending element in the buffer and configured time for this element elapsed
|
||||
* * EmitEarly - strategy do not wait to emit element if buffer is full
|
||||
*
|
||||
* '''Backpressures when''' depending on OverflowStrategy
|
||||
* * Backpressure - backpressures when buffer is full
|
||||
* * DropHead, DropTail, DropBuffer - never backpressures
|
||||
* * Fail - fails the stream if buffer gets full
|
||||
*
|
||||
* '''Completes when''' upstream completes and buffered elements has been drained
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*
|
||||
* @param of time to shift all messages
|
||||
* @param strategy Strategy that is used when incoming elements cannot fit inside the buffer
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def delay(of: java.time.Duration, strategy: DelayOverflowStrategy): SubSource[Out, Mat] =
|
||||
delay(of.asScala, strategy)
|
||||
new SubSource(delegate.delay(of.asScala, strategy))
|
||||
|
||||
/**
|
||||
* Shifts elements emission in time by an amount individually determined through delay strategy a specified amount.
|
||||
|
|
@ -1526,30 +1430,8 @@ final class SubSource[Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels or timer fires
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def takeWithin(duration: FiniteDuration): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.takeWithin(duration))
|
||||
|
||||
/**
|
||||
* Terminate processing (and cancel the upstream publisher) after the given
|
||||
* duration. Due to input buffering some elements may have been
|
||||
* requested from upstream publishers that will then not be processed downstream
|
||||
* of this step.
|
||||
*
|
||||
* Note that this can be combined with [[#take]] to limit the number of elements
|
||||
* within the duration.
|
||||
*
|
||||
* '''Emits when''' an upstream element arrives
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or timer fires
|
||||
*
|
||||
* '''Cancels when''' downstream cancels or timer fires
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def takeWithin(duration: java.time.Duration): SubSource[Out, Mat] =
|
||||
takeWithin(duration.asScala)
|
||||
new SubSource(delegate.takeWithin(duration.asScala))
|
||||
|
||||
/**
|
||||
* Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary
|
||||
|
|
@ -2399,25 +2281,8 @@ final class SubSource[Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def initialTimeout(timeout: FiniteDuration): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.initialTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the first element has not passed through this operator before the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.InitialTimeoutException]].
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses before first element arrives
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def initialTimeout(timeout: java.time.Duration): SubSource[Out, Mat] =
|
||||
initialTimeout(timeout.asScala)
|
||||
new SubSource(delegate.initialTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the completion of the stream does not happen until the provided timeout, the stream is failed
|
||||
|
|
@ -2431,25 +2296,8 @@ final class SubSource[Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def completionTimeout(timeout: FiniteDuration): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.completionTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the completion of the stream does not happen until the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.CompletionTimeoutException]].
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses before upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def completionTimeout(timeout: java.time.Duration): SubSource[Out, Mat] =
|
||||
completionTimeout(timeout.asScala)
|
||||
new SubSource(delegate.completionTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the time between two processed elements exceeds the provided timeout, the stream is failed
|
||||
|
|
@ -2464,26 +2312,8 @@ final class SubSource[Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def idleTimeout(timeout: FiniteDuration): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.idleTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the time between two processed elements exceeds the provided timeout, the stream is failed
|
||||
* with a [[org.apache.pekko.stream.StreamIdleTimeoutException]]. The timeout is checked periodically,
|
||||
* so the resolution of the check is one period (equals to timeout value).
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses between two emitted elements
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def idleTimeout(timeout: java.time.Duration): SubSource[Out, Mat] =
|
||||
idleTimeout(timeout.asScala)
|
||||
new SubSource(delegate.idleTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* If the time between the emission of an element and the following downstream demand exceeds the provided timeout,
|
||||
|
|
@ -2498,26 +2328,8 @@ final class SubSource[Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def backpressureTimeout(timeout: FiniteDuration): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.backpressureTimeout(timeout))
|
||||
|
||||
/**
|
||||
* If the time between the emission of an element and the following downstream demand exceeds the provided timeout,
|
||||
* the stream is failed with a [[org.apache.pekko.stream.BackpressureTimeoutException]]. The timeout is checked periodically,
|
||||
* so the resolution of the check is one period (equals to timeout value).
|
||||
*
|
||||
* '''Emits when''' upstream emits an element
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes or fails if timeout elapses between element emission and downstream demand.
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def backpressureTimeout(timeout: java.time.Duration): SubSource[Out, Mat] =
|
||||
backpressureTimeout(timeout.asScala)
|
||||
new SubSource(delegate.backpressureTimeout(timeout.asScala))
|
||||
|
||||
/**
|
||||
* Injects additional elements if upstream does not emit for a configured amount of time. In other words, this
|
||||
|
|
@ -2536,30 +2348,8 @@ final class SubSource[Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def keepAlive(maxIdle: FiniteDuration, injectedElem: function.Creator[Out]): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.keepAlive(maxIdle, () => injectedElem.create()))
|
||||
|
||||
/**
|
||||
* Injects additional elements if upstream does not emit for a configured amount of time. In other words, this
|
||||
* operator attempts to maintains a base rate of emitted elements towards the downstream.
|
||||
*
|
||||
* If the downstream backpressures then no element is injected until downstream demand arrives. Injected elements
|
||||
* do not accumulate during this period.
|
||||
*
|
||||
* Upstream elements are always preferred over injected elements.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element or if the upstream was idle for the configured period
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def keepAlive(maxIdle: java.time.Duration, injectedElem: function.Creator[Out]): SubSource[Out, Mat] =
|
||||
keepAlive(maxIdle.asScala, injectedElem)
|
||||
new SubSource(delegate.keepAlive(maxIdle.asScala, () => injectedElem.create()))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
|
|
@ -2593,45 +2383,6 @@ final class SubSource[Out, Mat](
|
|||
def throttle(elements: Int, per: java.time.Duration): javadsl.SubSource[Out, Mat] =
|
||||
new SubSource(delegate.throttle(elements, per.asScala))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
* for emitting messages. This operator works for streams where all elements have the same cost or length.
|
||||
*
|
||||
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
|
||||
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
|
||||
* to allow some burstiness. Whenever stream wants to send an element, it takes as many
|
||||
* tokens from the bucket as element costs If there isn't any, throttle waits until the
|
||||
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
|
||||
* to their cost minus available tokens, meeting the target rate. Bucket is full when stream just materialized and started.
|
||||
*
|
||||
* Parameter `mode` manages behavior when upstream is faster than throttle rate:
|
||||
* - [[pekko.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
|
||||
* - [[pekko.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate
|
||||
*
|
||||
* It is recommended to use non-zero burst sizes as they improve both performance and throttling precision by allowing
|
||||
* the implementation to avoid using the scheduler when input rates fall below the enforced limit and to reduce
|
||||
* most of the inaccuracy caused by the scheduler resolution (which is in the range of milliseconds).
|
||||
*
|
||||
* WARNING: Be aware that throttle is using scheduler to slow down the stream. This scheduler has minimal time of triggering
|
||||
* next push. Consequently it will slow down the stream as it has minimal pause for emitting. This can happen in
|
||||
* case burst is 0 and speed is higher than 30 events per second. You need to increase the `maximumBurst` if
|
||||
* elements arrive with small interval (30 milliseconds or less). Use the overloaded `throttle` method without
|
||||
* `maximumBurst` parameter to automatically calculate the `maximumBurst` based on the given rate (`cost/per`).
|
||||
* In other words the throttler always enforces the rate limit when `maximumBurst` parameter is given, but in
|
||||
* certain cases (mostly due to limited scheduler resolution) it enforces a tighter bound than what was prescribed.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element and configured time per each element elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or the incoming rate is higher than the speed limit
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttle(elements: Int, per: FiniteDuration, maximumBurst: Int, mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
|
||||
new SubSource(delegate.throttle(elements, per, maximumBurst, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `elements/per`. In other words, this operator set the maximum rate
|
||||
* for emitting messages. This operator works for streams where all elements have the same cost or length.
|
||||
|
|
@ -2711,53 +2462,6 @@ final class SubSource[Out, Mat](
|
|||
costCalculation: function.Function[Out, Integer]): javadsl.SubSource[Out, Mat] =
|
||||
new SubSource(delegate.throttle(cost, per.asScala, costCalculation.apply _))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `cost/per`. Cost is
|
||||
* calculating for each element individually by calling `calculateCost` function.
|
||||
* This operator works for streams when elements have different cost(length).
|
||||
* Streams of `ByteString` for example.
|
||||
*
|
||||
* Throttle implements the token bucket model. There is a bucket with a given token capacity (burst size or maximumBurst).
|
||||
* Tokens drops into the bucket at a given rate and can be `spared` for later use up to bucket capacity
|
||||
* to allow some burstiness. Whenever stream wants to send an element, it takes as many
|
||||
* tokens from the bucket as element costs. If there isn't any, throttle waits until the
|
||||
* bucket accumulates enough tokens. Elements that costs more than the allowed burst will be delayed proportionally
|
||||
* to their cost minus available tokens, meeting the target rate. Bucket is full when stream just materialized and started.
|
||||
*
|
||||
* Parameter `mode` manages behavior when upstream is faster than throttle rate:
|
||||
* - [[pekko.stream.ThrottleMode.Shaping]] makes pauses before emitting messages to meet throttle rate
|
||||
* - [[pekko.stream.ThrottleMode.Enforcing]] fails with exception when upstream is faster than throttle rate. Enforcing
|
||||
* cannot emit elements that cost more than the maximumBurst
|
||||
*
|
||||
* It is recommended to use non-zero burst sizes as they improve both performance and throttling precision by allowing
|
||||
* the implementation to avoid using the scheduler when input rates fall below the enforced limit and to reduce
|
||||
* most of the inaccuracy caused by the scheduler resolution (which is in the range of milliseconds).
|
||||
*
|
||||
* WARNING: Be aware that throttle is using scheduler to slow down the stream. This scheduler has minimal time of triggering
|
||||
* next push. Consequently it will slow down the stream as it has minimal pause for emitting. This can happen in
|
||||
* case burst is 0 and speed is higher than 30 events per second. You need to increase the `maximumBurst` if
|
||||
* elements arrive with small interval (30 milliseconds or less). Use the overloaded `throttle` method without
|
||||
* `maximumBurst` parameter to automatically calculate the `maximumBurst` based on the given rate (`cost/per`).
|
||||
* In other words the throttler always enforces the rate limit when `maximumBurst` parameter is given, but in
|
||||
* certain cases (mostly due to limited scheduler resolution) it enforces a tighter bound than what was prescribed.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element and configured time per each element elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or the incoming rate is higher than the speed limit
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def throttle(
|
||||
cost: Int,
|
||||
per: FiniteDuration,
|
||||
maximumBurst: Int,
|
||||
costCalculation: function.Function[Out, Integer],
|
||||
mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
|
||||
new SubSource(delegate.throttle(cost, per, maximumBurst, costCalculation.apply _, mode))
|
||||
|
||||
/**
|
||||
* Sends elements downstream with speed limited to `cost/per`. Cost is
|
||||
* calculating for each element individually by calling `calculateCost` function.
|
||||
|
|
@ -2804,70 +2508,6 @@ final class SubSource[Out, Mat](
|
|||
mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
|
||||
new SubSource(delegate.throttle(cost, per.asScala, maximumBurst, costCalculation.apply _, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: FiniteDuration, mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
|
||||
new SubSource(delegate.throttleEven(elements, per, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: java.time.Duration, mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
|
||||
throttleEven(elements, per.asScala, mode)
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(
|
||||
cost: Int,
|
||||
per: FiniteDuration,
|
||||
costCalculation: (Out) => Int,
|
||||
mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
|
||||
new SubSource(delegate.throttleEven(cost, per, costCalculation.apply _, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[#throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(
|
||||
cost: Int,
|
||||
per: java.time.Duration,
|
||||
costCalculation: (Out) => Int,
|
||||
mode: ThrottleMode): javadsl.SubSource[Out, Mat] =
|
||||
throttleEven(cost, per.asScala, costCalculation, mode)
|
||||
|
||||
/**
|
||||
* Detaches upstream demand from downstream demand without detaching the
|
||||
* stream rates; in other words acts like a buffer of size 1.
|
||||
|
|
@ -2893,24 +2533,8 @@ final class SubSource[Out, Mat](
|
|||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@deprecated("Use the overloaded one which accepts java.time.Duration instead.", since = "Akka 2.5.12")
|
||||
def initialDelay(delay: FiniteDuration): SubSource[Out, Mat] =
|
||||
new SubSource(delegate.initialDelay(delay))
|
||||
|
||||
/**
|
||||
* Delays the initial element by the specified duration.
|
||||
*
|
||||
* '''Emits when''' upstream emits an element if the initial delay is already elapsed
|
||||
*
|
||||
* '''Backpressures when''' downstream backpressures or initial delay is not yet elapsed
|
||||
*
|
||||
* '''Completes when''' upstream completes
|
||||
*
|
||||
* '''Cancels when''' downstream cancels
|
||||
*/
|
||||
@nowarn("msg=deprecated")
|
||||
def initialDelay(delay: java.time.Duration): SubSource[Out, Mat] =
|
||||
initialDelay(delay.asScala)
|
||||
new SubSource(delegate.initialDelay(delay.asScala))
|
||||
|
||||
/**
|
||||
* Change the attributes of this [[Source]] to the given ones and seal the list
|
||||
|
|
|
|||
|
|
@ -504,16 +504,6 @@ object Flow {
|
|||
def fromMaterializer[T, U, M](factory: (Materializer, Attributes) => Flow[T, U, M]): Flow[T, U, Future[M]] =
|
||||
Flow.fromGraph(new SetupFlowStage(factory))
|
||||
|
||||
/**
|
||||
* Defers the creation of a [[Flow]] until materialization. The `factory` function
|
||||
* exposes [[ActorMaterializer]] which is going to be used during materialization and
|
||||
* [[Attributes]] of the [[Flow]] returned by this method.
|
||||
*/
|
||||
@deprecated("Use 'fromMaterializer' instead", "Akka 2.6.0")
|
||||
def setup[T, U, M](factory: (ActorMaterializer, Attributes) => Flow[T, U, M]): Flow[T, U, Future[M]] =
|
||||
Flow.fromGraph(new SetupFlowStage((materializer, attributes) =>
|
||||
factory(ActorMaterializerHelper.downcast(materializer), attributes)))
|
||||
|
||||
/**
|
||||
* Creates a `Flow` from a `Sink` and a `Source` where the Flow's input
|
||||
* will be sent to the Sink and the Flow's output will come from the Source.
|
||||
|
|
@ -672,63 +662,6 @@ object Flow {
|
|||
})
|
||||
// format: ON
|
||||
|
||||
/**
|
||||
* Creates a real `Flow` upon receiving the first element. Internal `Flow` will not be created
|
||||
* if there are no elements, because of completion, cancellation, or error.
|
||||
*
|
||||
* The materialized value of the `Flow` is the value that is created by the `fallback` function.
|
||||
*
|
||||
* '''Emits when''' the internal flow is successfully created and it emits
|
||||
*
|
||||
* '''Backpressures when''' the internal flow is successfully created and it backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
||||
*
|
||||
* '''Cancels when''' downstream cancels (see below)
|
||||
*
|
||||
* The operator's default behavior in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||
* This behavior can be controlled by setting the [[pekko.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||
* this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||
*/
|
||||
@deprecated(
|
||||
"Use 'Flow.futureFlow' in combination with prefixAndTail(1) instead, see `futureFlow` operator docs for details",
|
||||
"Akka 2.6.0")
|
||||
def lazyInit[I, O, M](flowFactory: I => Future[Flow[I, O, M]], fallback: () => M): Flow[I, O, M] =
|
||||
Flow[I]
|
||||
.flatMapPrefix(1) {
|
||||
case Seq(a) => futureFlow(flowFactory(a)).mapMaterializedValue(_ => NotUsed)
|
||||
case Nil => Flow[I].asInstanceOf[Flow[I, O, NotUsed]]
|
||||
case _ => throw new RuntimeException() // won't happen, compiler exhaustiveness check pleaser
|
||||
}
|
||||
.mapMaterializedValue(_ => fallback())
|
||||
|
||||
/**
|
||||
* Creates a real `Flow` upon receiving the first element. Internal `Flow` will not be created
|
||||
* if there are no elements, because of completion, cancellation, or error.
|
||||
*
|
||||
* The materialized value of the `Flow` is a `Future[Option[M]]` that is completed with `Some(mat)` when the internal
|
||||
* flow gets materialized or with `None` when there where no elements. If the flow materialization (including
|
||||
* the call of the `flowFactory`) fails then the future is completed with a failure.
|
||||
*
|
||||
* '''Emits when''' the internal flow is successfully created and it emits
|
||||
*
|
||||
* '''Backpressures when''' the internal flow is successfully created and it backpressures
|
||||
*
|
||||
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
||||
*
|
||||
* '''Cancels when''' downstream cancels (see below)
|
||||
*
|
||||
* The operator's default behavior in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||
* This behavior can be controlled by setting the [[pekko.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||
* this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||
*/
|
||||
@deprecated("Use 'Flow.lazyFutureFlow' instead", "Akka 2.6.0")
|
||||
def lazyInitAsync[I, O, M](flowFactory: () => Future[Flow[I, O, M]]): Flow[I, O, Future[Option[M]]] =
|
||||
Flow.lazyFutureFlow(flowFactory).mapMaterializedValue {
|
||||
implicit val ec = pekko.dispatch.ExecutionContexts.parasitic
|
||||
_.map(Some.apply).recover { case _: NeverMaterializedException => None }
|
||||
}
|
||||
|
||||
/**
|
||||
* Turn a `Future[Flow]` into a flow that will consume the values of the source when the future completes successfully.
|
||||
* If the `Future` is completed with a failure the stream is failed.
|
||||
|
|
@ -3162,35 +3095,6 @@ trait FlowOps[+Out, +Mat] {
|
|||
mode: ThrottleMode): Repr[Out] =
|
||||
via(new Throttle(cost, per, maximumBurst, costCalculation, mode))
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval. throttleEven using
|
||||
* best effort approach to meet throttle rate.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(elements: Int, per: FiniteDuration, mode: ThrottleMode): Repr[Out] =
|
||||
throttle(elements, per, Throttle.AutomaticMaximumBurst, ConstantFun.oneInt, mode)
|
||||
|
||||
/**
|
||||
* This is a simplified version of throttle that spreads events evenly across the given time interval.
|
||||
*
|
||||
* Use this operator when you need just slow down a stream without worrying about exact amount
|
||||
* of time between events.
|
||||
*
|
||||
* If you want to be sure that no time interval has no more than specified number of events you need to use
|
||||
* [[throttle]] with maximumBurst attribute.
|
||||
* @see [[throttle]]
|
||||
*/
|
||||
@deprecated("Use throttle without `maximumBurst` parameter instead.", "Akka 2.5.12")
|
||||
def throttleEven(cost: Int, per: FiniteDuration, costCalculation: (Out) => Int, mode: ThrottleMode): Repr[Out] =
|
||||
throttle(cost, per, Throttle.AutomaticMaximumBurst, costCalculation, mode)
|
||||
|
||||
/**
|
||||
* Detaches upstream demand from downstream demand without detaching the
|
||||
* stream rates; in other words acts like a buffer of size 1.
|
||||
|
|
|
|||
|
|
@ -177,17 +177,6 @@ object Sink {
|
|||
}
|
||||
.to(Sink.head)
|
||||
|
||||
/**
|
||||
* Defers the creation of a [[Sink]] until materialization. The `factory` function
|
||||
* exposes [[ActorMaterializer]] which is going to be used during materialization and
|
||||
* [[Attributes]] of the [[Sink]] returned by this method.
|
||||
*/
|
||||
@deprecated("Use 'fromMaterializer' instead", "Akka 2.6.0")
|
||||
def setup[T, M](factory: (ActorMaterializer, Attributes) => Sink[T, M]): Sink[T, Future[M]] =
|
||||
fromMaterializer { (mat, attr) =>
|
||||
factory(ActorMaterializerHelper.downcast(mat), attr)
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to create [[Sink]] from `Subscriber`.
|
||||
*/
|
||||
|
|
@ -718,37 +707,6 @@ object Sink {
|
|||
*/
|
||||
def queue[T](): Sink[T, SinkQueueWithCancel[T]] = queue(1)
|
||||
|
||||
/**
|
||||
* Creates a real `Sink` upon receiving the first element. Internal `Sink` will not be created if there are no elements,
|
||||
* because of completion or error.
|
||||
*
|
||||
* If upstream completes before an element was received then the `Future` is completed with the value created by fallback.
|
||||
* If upstream fails before an element was received, `sinkFactory` throws an exception, or materialization of the internal
|
||||
* sink fails then the `Future` is completed with the exception.
|
||||
* Otherwise the `Future` is completed with the materialized value of the internal sink.
|
||||
*/
|
||||
@deprecated("Use 'Sink.lazyFutureSink' in combination with 'Flow.prefixAndTail(1)' instead", "Akka 2.6.0")
|
||||
def lazyInit[T, M](sinkFactory: T => Future[Sink[T, M]], fallback: () => M): Sink[T, Future[M]] =
|
||||
Sink
|
||||
.fromGraph(new LazySink[T, M](sinkFactory))
|
||||
.mapMaterializedValue(_.recover { case _: NeverMaterializedException => fallback() }(ExecutionContexts.parasitic))
|
||||
|
||||
/**
|
||||
* Creates a real `Sink` upon receiving the first element. Internal `Sink` will not be created if there are no elements,
|
||||
* because of completion or error.
|
||||
*
|
||||
* If upstream completes before an element was received then the `Future` is completed with `None`.
|
||||
* If upstream fails before an element was received, `sinkFactory` throws an exception, or materialization of the internal
|
||||
* sink fails then the `Future` is completed with the exception.
|
||||
* Otherwise the `Future` is completed with the materialized value of the internal sink.
|
||||
*/
|
||||
@deprecated("Use 'Sink.lazyFutureSink' instead", "Akka 2.6.0")
|
||||
def lazyInitAsync[T, M](sinkFactory: () => Future[Sink[T, M]]): Sink[T, Future[Option[M]]] =
|
||||
Sink.fromGraph(new LazySink[T, M](_ => sinkFactory())).mapMaterializedValue { m =>
|
||||
implicit val ec = ExecutionContexts.parasitic
|
||||
m.map(Option.apply _).recover { case _: NeverMaterializedException => None }
|
||||
}
|
||||
|
||||
/**
|
||||
* Turn a `Future[Sink]` into a Sink that will consume the values of the source when the future completes successfully.
|
||||
* If the `Future` is completed with a failure the stream is failed.
|
||||
|
|
|
|||
|
|
@ -386,16 +386,6 @@ object Source {
|
|||
def fromMaterializer[T, M](factory: (Materializer, Attributes) => Source[T, M]): Source[T, Future[M]] =
|
||||
Source.fromGraph(new SetupSourceStage(factory))
|
||||
|
||||
/**
|
||||
* Defers the creation of a [[Source]] until materialization. The `factory` function
|
||||
* exposes [[ActorMaterializer]] which is going to be used during materialization and
|
||||
* [[Attributes]] of the [[Source]] returned by this method.
|
||||
*/
|
||||
@deprecated("Use 'fromMaterializer' instead", "Akka 2.6.0")
|
||||
def setup[T, M](factory: (ActorMaterializer, Attributes) => Source[T, M]): Source[T, Future[M]] =
|
||||
Source.fromGraph(new SetupSourceStage((materializer, attributes) =>
|
||||
factory(ActorMaterializerHelper.downcast(materializer), attributes)))
|
||||
|
||||
/**
|
||||
* Helper to create [[Source]] from `Iterable`.
|
||||
* Example usage: `Source(Seq(1,2,3))`
|
||||
|
|
@ -557,26 +547,6 @@ object Source {
|
|||
def failed[T](cause: Throwable): Source[T, NotUsed] =
|
||||
Source.fromGraph(new FailedSource[T](cause))
|
||||
|
||||
/**
|
||||
* Creates a `Source` that is not materialized until there is downstream demand, when the source gets materialized
|
||||
* the materialized future is completed with its value, if downstream cancels or fails without any demand the
|
||||
* create factory is never called and the materialized `Future` is failed.
|
||||
*/
|
||||
@deprecated("Use 'Source.lazySource' instead", "Akka 2.6.0")
|
||||
def lazily[T, M](create: () => Source[T, M]): Source[T, Future[M]] =
|
||||
Source.fromGraph(new LazySource[T, M](create))
|
||||
|
||||
/**
|
||||
* Creates a `Source` from supplied future factory that is not called until downstream demand. When source gets
|
||||
* materialized the materialized future is completed with the value from the factory. If downstream cancels or fails
|
||||
* without any demand the create factory is never called and the materialized `Future` is failed.
|
||||
*
|
||||
* @see [[Source.lazily]]
|
||||
*/
|
||||
@deprecated("Use 'Source.lazyFuture' instead", "Akka 2.6.0")
|
||||
def lazilyAsync[T](create: () => Future[T]): Source[T, Future[NotUsed]] =
|
||||
lazily(() => fromFuture(create()))
|
||||
|
||||
/**
|
||||
* Emits a single value when the given `Future` is successfully completed and then completes the stream.
|
||||
* The stream fails if the `Future` is completed with a failure.
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue