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:
PJ Fanning 2025-08-03 10:55:37 +01:00 committed by GitHub
parent c8960e2783
commit 0dd219fab4
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
24 changed files with 132 additions and 2718 deletions

View file

@ -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).
@@@

View file

@ -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
@@@

View file

@ -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`.

View file

@ -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`.

View file

@ -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`
@@@

View file

@ -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
@@@

View file

@ -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)

View file

@ -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",

View file

@ -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),

View file

@ -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 =>

View file

@ -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)
}
}
}

View file

@ -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()
}
}
}

View file

@ -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)

View file

@ -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
}
}
}

View file

@ -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")

View file

@ -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.
*/

View file

@ -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

View file

@ -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.

View file

@ -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

View file

@ -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

View file

@ -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

View file

@ -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.

View file

@ -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.

View file

@ -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.