fusable SplitWhen and reusable HTTP blueprint
plus further optimizations Lots of contributions from drewhk (Endre Varga).
This commit is contained in:
parent
3c3545020b
commit
469e15581f
45 changed files with 1251 additions and 838 deletions
|
|
@ -48,6 +48,7 @@ class KeepGoingStageSpec extends AkkaSpec {
|
|||
private var listener: Option[ActorRef] = None
|
||||
|
||||
override def preStart(): Unit = {
|
||||
setKeepGoing(keepAlive)
|
||||
promise.trySuccess(PingRef(getAsyncCallback(onCommand)))
|
||||
}
|
||||
|
||||
|
|
@ -73,8 +74,6 @@ class KeepGoingStageSpec extends AkkaSpec {
|
|||
override def onUpstreamFinish(): Unit = listener.foreach(_ ! UpstreamCompleted)
|
||||
})
|
||||
|
||||
override def keepGoingAfterAllPortsClosed: Boolean = keepAlive
|
||||
|
||||
override def postStop(): Unit = listener.foreach(_ ! PostStop)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -43,7 +43,7 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"work together with SplitWhen" in {
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
val subscriber = TestSubscriber.probe[Int]()
|
||||
Source(1 to 10)
|
||||
.splitWhen(_ % 2 == 0)
|
||||
.prefixAndTail(0)
|
||||
|
|
@ -51,11 +51,11 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
.concatSubstreams
|
||||
.flatMapConcat(ConstantFun.scalaIdentityFunction)
|
||||
.runWith(Sink.fromSubscriber(subscriber))
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(10)
|
||||
for (i ← (1 to 10))
|
||||
subscriber.expectNext() shouldBe i
|
||||
subscription.request(1)
|
||||
|
||||
for (i ← 1 to 10)
|
||||
subscriber.requestNext() shouldBe i
|
||||
|
||||
subscriber.request(1)
|
||||
subscriber.expectComplete()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -86,7 +86,7 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
|
|||
|
||||
"bubble up substream exceptions" in {
|
||||
val ex = new Exception("buh")
|
||||
intercept[TestFailedException] {
|
||||
val result = intercept[TestFailedException] {
|
||||
Source(List(blocked, blocked, Source.failed(ex)))
|
||||
.flatMapMerge(10, identity)
|
||||
.runWith(Sink.head)
|
||||
|
|
|
|||
|
|
@ -101,7 +101,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
|
|||
|
||||
val subscriber2 = TestSubscriber.probe[Int]()
|
||||
tail.to(Sink.fromSubscriber(subscriber2)).run()
|
||||
subscriber2.expectSubscriptionAndError().getMessage should ===("Tail Source cannot be materialized more than once.")
|
||||
subscriber2.expectSubscriptionAndError().getMessage should ===("Substream Source cannot be materialized more than once")
|
||||
|
||||
subscriber1.requestNext(2).expectComplete()
|
||||
|
||||
|
|
@ -122,7 +122,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
|
|||
Thread.sleep(1000)
|
||||
|
||||
tail.to(Sink.fromSubscriber(subscriber)).run()(tightTimeoutMaterializer)
|
||||
subscriber.expectSubscriptionAndError().getMessage should ===("Tail Source has not been materialized in 500 milliseconds.")
|
||||
subscriber.expectSubscriptionAndError().getMessage should ===("Substream Source has not been materialized in 500 milliseconds")
|
||||
}
|
||||
|
||||
"shut down main stage if substream is empty, even when not subscribed" in assertAllStagesStopped {
|
||||
|
|
|
|||
|
|
@ -12,6 +12,7 @@ import akka.stream.testkit.TestPublisher
|
|||
import akka.stream.testkit.TestSubscriber
|
||||
import akka.stream.testkit.Utils._
|
||||
import org.reactivestreams.Publisher
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.StreamSubscriptionTimeoutSettings
|
||||
import akka.stream.StreamSubscriptionTimeoutTerminationMode
|
||||
|
|
@ -115,6 +116,14 @@ class FlowSplitAfterSpec extends AkkaSpec {
|
|||
}
|
||||
}
|
||||
|
||||
"work with single elem splits" in assertAllStagesStopped {
|
||||
Await.result(
|
||||
Source(1 to 10).splitAfter(_ ⇒ true).lift
|
||||
.mapAsync(1)(_.runWith(Sink.head)) // Please note that this line *also* implicitly asserts nonempty substreams
|
||||
.grouped(10).runWith(Sink.head),
|
||||
3.second) should ===(1 to 10)
|
||||
}
|
||||
|
||||
"support cancelling substreams" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(splitAfter = 5, elementCount = 8) {
|
||||
val s1 = StreamPuppet(expectSubFlow().runWith(Sink.asPublisher(false)))
|
||||
|
|
@ -181,6 +190,8 @@ class FlowSplitAfterSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"resume stream when splitAfter function throws" in assertAllStagesStopped {
|
||||
info("Supervision is not supported fully by GraphStages yet")
|
||||
pending
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val exc = TE("test")
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe)
|
||||
|
|
|
|||
|
|
@ -3,16 +3,14 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
import akka.stream.ActorAttributes
|
||||
import akka.stream._
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.impl.SubscriptionTimeoutException
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit._
|
||||
import org.reactivestreams.Publisher
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.StreamSubscriptionTimeoutSettings
|
||||
import akka.stream.StreamSubscriptionTimeoutTerminationMode
|
||||
|
||||
class FlowSplitWhenSpec extends AkkaSpec {
|
||||
import FlowSplitAfterSpec._
|
||||
|
|
@ -85,6 +83,16 @@ class FlowSplitWhenSpec extends AkkaSpec {
|
|||
}
|
||||
}
|
||||
|
||||
"not emit substreams if the parent stream is empty" in assertAllStagesStopped {
|
||||
|
||||
Await.result(
|
||||
Source.empty[Int]
|
||||
.splitWhen(_ ⇒ true).lift
|
||||
.mapAsync(1)(_.runWith(Sink.headOption)).grouped(10).runWith(Sink.headOption),
|
||||
3.seconds) should ===(None) // rather tricky way of saying that no empty substream should be emitted (vs. Some(None))
|
||||
|
||||
}
|
||||
|
||||
"work when first element is split-by" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(1, elementCount = 3) {
|
||||
val s1 = StreamPuppet(getSubFlow().runWith(Sink.asPublisher(false)))
|
||||
|
|
@ -137,7 +145,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
|
|||
substream.cancel()
|
||||
|
||||
masterStream.expectNext(())
|
||||
masterStream.expectNoMsg(1.second)
|
||||
masterStream.expectNoMsg(100.millis)
|
||||
masterStream.cancel()
|
||||
inputs.expectCancellation()
|
||||
|
||||
|
|
@ -173,126 +181,166 @@ class FlowSplitWhenSpec extends AkkaSpec {
|
|||
src2.runWith(Sink.fromSubscriber(substream4))
|
||||
|
||||
substream4.requestNext(2)
|
||||
substream4.expectNoMsg(1.second)
|
||||
masterStream3.expectNoMsg(1.second)
|
||||
substream4.expectNoMsg(100.millis)
|
||||
masterStream3.expectNoMsg(100.millis)
|
||||
inputs3.expectRequest()
|
||||
inputs3.expectRequest()
|
||||
inputs3.expectNoMsg(1.second)
|
||||
inputs3.expectNoMsg(100.millis)
|
||||
|
||||
substream4.cancel()
|
||||
inputs3.expectNoMsg(1.second)
|
||||
masterStream3.expectNoMsg(1.second)
|
||||
inputs3.expectNoMsg(100.millis)
|
||||
masterStream3.expectNoMsg(100.millis)
|
||||
|
||||
masterStream3.cancel()
|
||||
inputs3.expectCancellation()
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
"support cancelling the master stream" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(splitWhen = 5, elementCount = 8) {
|
||||
val s1 = StreamPuppet(getSubFlow().runWith(Sink.asPublisher(false)))
|
||||
masterSubscription.cancel()
|
||||
s1.request(4)
|
||||
s1.expectNext(1)
|
||||
s1.expectNext(2)
|
||||
s1.expectNext(3)
|
||||
s1.expectNext(4)
|
||||
s1.request(1)
|
||||
s1.expectComplete()
|
||||
"support cancelling the master stream" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(splitWhen = 5, elementCount = 8) {
|
||||
val s1 = StreamPuppet(getSubFlow().runWith(Sink.asPublisher(false)))
|
||||
masterSubscription.cancel()
|
||||
s1.request(4)
|
||||
s1.expectNext(1)
|
||||
s1.expectNext(2)
|
||||
s1.expectNext(3)
|
||||
s1.expectNext(4)
|
||||
s1.request(1)
|
||||
s1.expectComplete()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"fail stream when splitWhen function throws" in assertAllStagesStopped {
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val exc = TE("test")
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe)
|
||||
.splitWhen(elem ⇒ if (elem == 3) throw exc else elem % 3 == 0)
|
||||
.lift
|
||||
.runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
|
||||
publisher.subscribe(subscriber)
|
||||
"fail stream when splitWhen function throws" in assertAllStagesStopped {
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val exc = TE("test")
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe)
|
||||
.splitWhen(elem ⇒ if (elem == 3) throw exc else elem % 3 == 0)
|
||||
.lift
|
||||
.runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
||||
val upstreamSubscription = publisherProbeProbe.expectSubscription()
|
||||
val upstreamSubscription = publisherProbeProbe.expectSubscription()
|
||||
|
||||
val downstreamSubscription = subscriber.expectSubscription()
|
||||
downstreamSubscription.request(100)
|
||||
val downstreamSubscription = subscriber.expectSubscription()
|
||||
downstreamSubscription.request(100)
|
||||
|
||||
upstreamSubscription.sendNext(1)
|
||||
upstreamSubscription.sendNext(1)
|
||||
|
||||
val substream = subscriber.expectNext()
|
||||
val substreamPuppet = StreamPuppet(substream.runWith(Sink.asPublisher(false)))
|
||||
val substream = subscriber.expectNext()
|
||||
val substreamPuppet = StreamPuppet(substream.runWith(Sink.asPublisher(false)))
|
||||
|
||||
substreamPuppet.request(10)
|
||||
substreamPuppet.expectNext(1)
|
||||
substreamPuppet.request(10)
|
||||
substreamPuppet.expectNext(1)
|
||||
|
||||
upstreamSubscription.sendNext(2)
|
||||
substreamPuppet.expectNext(2)
|
||||
upstreamSubscription.sendNext(2)
|
||||
substreamPuppet.expectNext(2)
|
||||
|
||||
upstreamSubscription.sendNext(3)
|
||||
upstreamSubscription.sendNext(3)
|
||||
|
||||
subscriber.expectError(exc)
|
||||
substreamPuppet.expectError(exc)
|
||||
upstreamSubscription.expectCancellation()
|
||||
}
|
||||
subscriber.expectError(exc)
|
||||
substreamPuppet.expectError(exc)
|
||||
upstreamSubscription.expectCancellation()
|
||||
}
|
||||
|
||||
"resume stream when splitWhen function throws" in assertAllStagesStopped {
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val exc = TE("test")
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe)
|
||||
.splitWhen(elem ⇒ if (elem == 3) throw exc else elem % 3 == 0)
|
||||
.lift
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
|
||||
.runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
|
||||
publisher.subscribe(subscriber)
|
||||
"work with single elem splits" in assertAllStagesStopped {
|
||||
Await.result(
|
||||
Source(1 to 100).splitWhen(_ ⇒ true).lift
|
||||
.mapAsync(1)(_.runWith(Sink.head)) // Please note that this line *also* implicitly asserts nonempty substreams
|
||||
.grouped(200).runWith(Sink.head),
|
||||
3.second) should ===(1 to 100)
|
||||
}
|
||||
|
||||
val upstreamSubscription = publisherProbeProbe.expectSubscription()
|
||||
"fail substream if materialized twice" in assertAllStagesStopped {
|
||||
an[IllegalStateException] mustBe thrownBy {
|
||||
Await.result(
|
||||
Source.single(1).splitWhen(_ ⇒ true).lift
|
||||
.mapAsync(1) { src ⇒ src.runWith(Sink.ignore); src.runWith(Sink.ignore) } // Sink.ignore+mapAsync pipes error back
|
||||
.runWith(Sink.ignore),
|
||||
3.seconds)
|
||||
}
|
||||
}
|
||||
|
||||
val downstreamSubscription = subscriber.expectSubscription()
|
||||
downstreamSubscription.request(100)
|
||||
"fail stream if substream not materialized in time" in assertAllStagesStopped {
|
||||
val tightTimeoutMaterializer =
|
||||
ActorMaterializer(ActorMaterializerSettings(system)
|
||||
.withSubscriptionTimeoutSettings(
|
||||
StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 500.millisecond)))
|
||||
|
||||
upstreamSubscription.sendNext(1)
|
||||
val testSource = Source.single(1).concat(Source.maybe).splitWhen(_ ⇒ true)
|
||||
|
||||
val substream1 = subscriber.expectNext()
|
||||
val substreamPuppet1 = StreamPuppet(substream1.runWith(Sink.asPublisher(false)))
|
||||
a[SubscriptionTimeoutException] mustBe thrownBy {
|
||||
Await.result(
|
||||
testSource.lift
|
||||
.delay(1.second)
|
||||
.flatMapConcat(identity)
|
||||
.runWith(Sink.ignore)(tightTimeoutMaterializer),
|
||||
3.seconds)
|
||||
}
|
||||
}
|
||||
|
||||
substreamPuppet1.request(10)
|
||||
substreamPuppet1.expectNext(1)
|
||||
"resume stream when splitWhen function throws" in assertAllStagesStopped {
|
||||
info("Supervision is not supported fully by GraphStages yet")
|
||||
pending
|
||||
|
||||
upstreamSubscription.sendNext(2)
|
||||
substreamPuppet1.expectNext(2)
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val exc = TE("test")
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe)
|
||||
.splitWhen(elem ⇒ if (elem == 3) throw exc else elem % 3 == 0)
|
||||
.lift
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
|
||||
.runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
||||
upstreamSubscription.sendNext(3)
|
||||
upstreamSubscription.sendNext(4)
|
||||
substreamPuppet1.expectNext(4) // note that 3 was dropped
|
||||
val upstreamSubscription = publisherProbeProbe.expectSubscription()
|
||||
|
||||
upstreamSubscription.sendNext(5)
|
||||
substreamPuppet1.expectNext(5)
|
||||
val downstreamSubscription = subscriber.expectSubscription()
|
||||
downstreamSubscription.request(100)
|
||||
|
||||
upstreamSubscription.sendNext(6)
|
||||
substreamPuppet1.expectComplete()
|
||||
val substream2 = subscriber.expectNext()
|
||||
val substreamPuppet2 = StreamPuppet(substream2.runWith(Sink.asPublisher(false)))
|
||||
substreamPuppet2.request(10)
|
||||
substreamPuppet2.expectNext(6)
|
||||
upstreamSubscription.sendNext(1)
|
||||
|
||||
upstreamSubscription.sendComplete()
|
||||
subscriber.expectComplete()
|
||||
substreamPuppet2.expectComplete()
|
||||
}
|
||||
val substream1 = subscriber.expectNext()
|
||||
val substreamPuppet1 = StreamPuppet(substream1.runWith(Sink.asPublisher(false)))
|
||||
|
||||
"pass along early cancellation" in assertAllStagesStopped {
|
||||
val up = TestPublisher.manualProbe[Int]()
|
||||
val down = TestSubscriber.manualProbe[Source[Int, Unit]]()
|
||||
substreamPuppet1.request(10)
|
||||
substreamPuppet1.expectNext(1)
|
||||
|
||||
val flowSubscriber = Source.asSubscriber[Int].splitWhen(_ % 3 == 0).lift.to(Sink.fromSubscriber(down)).run()
|
||||
upstreamSubscription.sendNext(2)
|
||||
substreamPuppet1.expectNext(2)
|
||||
|
||||
upstreamSubscription.sendNext(3)
|
||||
upstreamSubscription.sendNext(4)
|
||||
substreamPuppet1.expectNext(4) // note that 3 was dropped
|
||||
|
||||
upstreamSubscription.sendNext(5)
|
||||
substreamPuppet1.expectNext(5)
|
||||
|
||||
upstreamSubscription.sendNext(6)
|
||||
substreamPuppet1.expectComplete()
|
||||
val substream2 = subscriber.expectNext()
|
||||
val substreamPuppet2 = StreamPuppet(substream2.runWith(Sink.asPublisher(false)))
|
||||
substreamPuppet2.request(10)
|
||||
substreamPuppet2.expectNext(6)
|
||||
|
||||
upstreamSubscription.sendComplete()
|
||||
subscriber.expectComplete()
|
||||
substreamPuppet2.expectComplete()
|
||||
}
|
||||
|
||||
"pass along early cancellation" in assertAllStagesStopped {
|
||||
val up = TestPublisher.manualProbe[Int]()
|
||||
val down = TestSubscriber.manualProbe[Source[Int, Unit]]()
|
||||
|
||||
val flowSubscriber = Source.asSubscriber[Int].splitWhen(_ % 3 == 0).lift.to(Sink.fromSubscriber(down)).run()
|
||||
|
||||
val downstream = down.expectSubscription()
|
||||
downstream.cancel()
|
||||
up.subscribe(flowSubscriber)
|
||||
val upsub = up.expectSubscription()
|
||||
upsub.expectCancellation()
|
||||
}
|
||||
|
||||
val downstream = down.expectSubscription()
|
||||
downstream.cancel()
|
||||
up.subscribe(flowSubscriber)
|
||||
val upsub = up.expectSubscription()
|
||||
upsub.expectCancellation()
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -84,7 +84,7 @@ class SinkForeachParallelSpec extends AkkaSpec {
|
|||
}).withAttributes(supervisionStrategy(resumingDecider)))
|
||||
|
||||
latch.countDown()
|
||||
probe.expectMsgAllOf(1, 2, 4)
|
||||
probe.expectMsgAllOf(1, 2, 4, 5)
|
||||
|
||||
Await.result(p, 5.seconds)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -72,7 +72,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
|
|||
val (_, s3) = subscriber.expectNext()
|
||||
|
||||
// sleep long enough for it to be cleaned up
|
||||
Thread.sleep(1000)
|
||||
Thread.sleep(1500)
|
||||
|
||||
val f = s3.runWith(Sink.head).recover { case _: SubscriptionTimeoutException ⇒ "expected" }
|
||||
Await.result(f, 300.millis) should equal("expected")
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue