fusable SplitWhen and reusable HTTP blueprint

plus further optimizations

Lots of contributions from drewhk (Endre Varga).
This commit is contained in:
Roland Kuhn 2015-12-20 12:54:05 +01:00
parent 3c3545020b
commit 469e15581f
45 changed files with 1251 additions and 838 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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