parent
6a22cdebfc
commit
c4417a8c39
19 changed files with 209 additions and 52 deletions
|
|
@ -3,5 +3,8 @@ akka {
|
|||
actor {
|
||||
serialize-creators = on
|
||||
serialize-messages = on
|
||||
default-dispatcher.throughput = 1 // Amplify the effects of fuzzing
|
||||
}
|
||||
|
||||
stream.materializer.debug.fuzzing-mode = on
|
||||
}
|
||||
|
|
@ -384,6 +384,8 @@ class GraphInterpreterPortsSpec extends AkkaSpec with GraphInterpreterSpecKit {
|
|||
"ignore pull while completing" in new PortTestSetup {
|
||||
out.complete()
|
||||
in.pull()
|
||||
// While the pull event is not enqueued at this point, we should still report the state correctly
|
||||
in.hasBeenPulled should be(true)
|
||||
|
||||
stepAll()
|
||||
|
||||
|
|
@ -648,6 +650,8 @@ class GraphInterpreterPortsSpec extends AkkaSpec with GraphInterpreterSpecKit {
|
|||
|
||||
in.cancel()
|
||||
out.push(0)
|
||||
// While the push event is not enqueued at this point, we should still report the state correctly
|
||||
out.isAvailable should be(false)
|
||||
|
||||
stepAll()
|
||||
|
||||
|
|
@ -1066,6 +1070,7 @@ class GraphInterpreterPortsSpec extends AkkaSpec with GraphInterpreterSpecKit {
|
|||
"ignore pull while failing" in new PortTestSetup {
|
||||
out.fail(TE("test"))
|
||||
in.pull()
|
||||
in.hasBeenPulled should be(true)
|
||||
|
||||
stepAll()
|
||||
|
||||
|
|
|
|||
|
|
@ -3,7 +3,8 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.{ OverflowStrategy, Attributes }
|
||||
import akka.stream.stage.AbstractStage.PushPullGraphStage
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.scaladsl.{ Merge, Broadcast, Balance, Zip }
|
||||
import GraphInterpreter._
|
||||
|
|
@ -341,6 +342,43 @@ class GraphInterpreterSpec extends AkkaSpec with GraphInterpreterSpecKit {
|
|||
// The cycle is now empty
|
||||
interpreter.isSuspended should be(false)
|
||||
}
|
||||
|
||||
"implement buffer" in new TestSetup {
|
||||
val source = new UpstreamProbe[String]("source")
|
||||
val sink = new DownstreamProbe[String]("sink")
|
||||
val buffer = new PushPullGraphStage[String, String, Unit](
|
||||
(_) ⇒ new Buffer[String](2, OverflowStrategy.backpressure),
|
||||
Attributes.none)
|
||||
|
||||
builder(buffer)
|
||||
.connect(source, buffer.shape.inlet)
|
||||
.connect(buffer.shape.outlet, sink)
|
||||
.init()
|
||||
|
||||
stepAll()
|
||||
lastEvents() should ===(Set(RequestOne(source)))
|
||||
|
||||
sink.requestOne()
|
||||
lastEvents() should ===(Set.empty)
|
||||
|
||||
source.onNext("A")
|
||||
lastEvents() should ===(Set(RequestOne(source), OnNext(sink, "A")))
|
||||
|
||||
source.onNext("B")
|
||||
lastEvents() should ===(Set(RequestOne(source)))
|
||||
|
||||
source.onNext("C", eventLimit = 0)
|
||||
sink.requestOne()
|
||||
lastEvents() should ===(Set(OnNext(sink, "B"), RequestOne(source)))
|
||||
|
||||
sink.requestOne(eventLimit = 0)
|
||||
source.onComplete(eventLimit = 3)
|
||||
lastEvents() should ===(Set(OnNext(sink, "C")))
|
||||
|
||||
sink.requestOne()
|
||||
lastEvents() should ===(Set(OnComplete(sink)))
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ trait GraphInterpreterSpecKit {
|
|||
in.id = 0
|
||||
}
|
||||
|
||||
class AssemblyBuilder(stages: Seq[GraphStage[_ <: Shape]]) {
|
||||
class AssemblyBuilder(stages: Seq[GraphStageWithMaterializedValue[_ <: Shape, _]]) {
|
||||
var upstreams = Vector.empty[(UpstreamBoundaryStageLogic[_], Inlet[_])]
|
||||
var downstreams = Vector.empty[(Outlet[_], DownstreamBoundaryStageLogic[_])]
|
||||
var connections = Vector.empty[(Outlet[_], Inlet[_])]
|
||||
|
|
@ -71,7 +71,8 @@ trait GraphInterpreterSpecKit {
|
|||
val assembly = buildAssembly()
|
||||
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize(Attributes.none)
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, NoLogging, inHandlers, outHandlers, logics, (_, _, _) ⇒ ())
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, NoLogging, inHandlers, outHandlers, logics,
|
||||
(_, _, _) ⇒ (), fuzzingMode = false)
|
||||
|
||||
for ((upstream, i) ← upstreams.zipWithIndex) {
|
||||
_interpreter.attachUpstreamBoundary(i, upstream._1)
|
||||
|
|
@ -87,10 +88,11 @@ trait GraphInterpreterSpecKit {
|
|||
|
||||
def manualInit(assembly: GraphAssembly): Unit = {
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize(Attributes.none)
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, NoLogging, inHandlers, outHandlers, logics, (_, _, _) ⇒ ())
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, NoLogging, inHandlers, outHandlers, logics,
|
||||
(_, _, _) ⇒ (), fuzzingMode = false)
|
||||
}
|
||||
|
||||
def builder(stages: GraphStage[_ <: Shape]*): AssemblyBuilder = new AssemblyBuilder(stages)
|
||||
def builder(stages: GraphStageWithMaterializedValue[_ <: Shape, _]*): AssemblyBuilder = new AssemblyBuilder(stages)
|
||||
}
|
||||
|
||||
abstract class TestSetup extends Builder {
|
||||
|
|
@ -132,6 +134,18 @@ trait GraphInterpreterSpecKit {
|
|||
push(out, elem)
|
||||
interpreter.execute(eventLimit)
|
||||
}
|
||||
|
||||
def onComplete(eventLimit: Int = Int.MaxValue): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"----- COMPLETE $this")
|
||||
complete(out)
|
||||
interpreter.execute(eventLimit)
|
||||
}
|
||||
|
||||
def onFailure(eventLimit: Int = Int.MaxValue, ex: Throwable): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"----- FAIL $this")
|
||||
fail(out, ex)
|
||||
interpreter.execute(eventLimit)
|
||||
}
|
||||
}
|
||||
|
||||
class DownstreamProbe[T](override val toString: String) extends DownstreamBoundaryStageLogic[T] {
|
||||
|
|
@ -149,6 +163,12 @@ trait GraphInterpreterSpecKit {
|
|||
pull(in)
|
||||
interpreter.execute(eventLimit)
|
||||
}
|
||||
|
||||
def cancel(eventLimit: Int = Int.MaxValue): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"----- CANCEL $this")
|
||||
cancel(in)
|
||||
interpreter.execute(eventLimit)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -19,7 +19,10 @@ import scala.concurrent.duration._
|
|||
import java.net.BindException
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-enabled=auto\nakka.stream.materializer.subscription-timeout.timeout = 3s") with TcpHelper {
|
||||
class TcpSpec extends AkkaSpec(
|
||||
"""
|
||||
|akka.io.tcp.windows-connection-abort-workaround-enabled=auto
|
||||
|akka.stream.materializer.subscription-timeout.timeout = 3s""".stripMargin) with TcpHelper {
|
||||
var demand = 0L
|
||||
|
||||
"Outgoing TCP stream" must {
|
||||
|
|
|
|||
|
|
@ -6,9 +6,7 @@ package akka.stream.scaladsl
|
|||
import scala.concurrent.Await
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, OverflowStrategy }
|
||||
import akka.stream.OverflowStrategy.Fail.BufferOverflowException
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.scaladsl._
|
||||
|
|
@ -77,6 +75,9 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
// Fill up buffer
|
||||
for (i ← 1 to 200) publisher.sendNext(i)
|
||||
|
||||
// The next request would be otherwise in race with the last onNext in the above loop
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
|
||||
// drain
|
||||
for (i ← 101 to 200) {
|
||||
sub.request(1)
|
||||
|
|
@ -103,6 +104,9 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
// Fill up buffer
|
||||
for (i ← 1 to 200) publisher.sendNext(i)
|
||||
|
||||
// The next request would be otherwise in race with the last onNext in the above loop
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
|
||||
// drain
|
||||
for (i ← 1 to 99) {
|
||||
sub.request(1)
|
||||
|
|
@ -132,6 +136,9 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
// Fill up buffer
|
||||
for (i ← 1 to 150) publisher.sendNext(i)
|
||||
|
||||
// The next request would be otherwise in race with the last onNext in the above loop
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
|
||||
// drain
|
||||
for (i ← 101 to 150) {
|
||||
sub.request(1)
|
||||
|
|
@ -151,9 +158,14 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
"drop new elements if buffer is full and configured so" in {
|
||||
val (publisher, subscriber) = TestSource.probe[Int].buffer(100, overflowStrategy = OverflowStrategy.dropNew).toMat(TestSink.probe[Int])(Keep.both).run()
|
||||
|
||||
subscriber.ensureSubscription()
|
||||
|
||||
// Fill up buffer
|
||||
for (i ← 1 to 150) publisher.sendNext(i)
|
||||
|
||||
// The next request would be otherwise in race with the last onNext in the above loop
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
|
||||
// drain
|
||||
for (i ← 1 to 100) {
|
||||
subscriber.requestNext(i)
|
||||
|
|
@ -205,6 +217,8 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
// Fill up buffer
|
||||
for (i ← 1 to 200) publisher.sendNext(i)
|
||||
|
||||
// The request below is in race otherwise with the onNext(200) above
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
sub.request(1)
|
||||
subscriber.expectNext(200)
|
||||
|
||||
|
|
|
|||
|
|
@ -70,17 +70,19 @@ class FlowConflateSpec extends AkkaSpec {
|
|||
subscriber.expectNext(1)
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(1.second)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
publisher.sendNext(2)
|
||||
subscriber.expectNext(2)
|
||||
|
||||
publisher.sendNext(3)
|
||||
publisher.sendNext(4)
|
||||
// The request can be in race with the above onNext(4) so the result would be either 3 or 7.
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
sub.request(1)
|
||||
subscriber.expectNext(7)
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(1.second)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
sub.cancel()
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,8 +7,7 @@ import scala.concurrent.Await
|
|||
import scala.concurrent.duration._
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
|
||||
|
||||
import akka.stream.testkit._
|
||||
|
||||
|
|
@ -22,6 +21,9 @@ class FlowExpandSpec extends AkkaSpec {
|
|||
"Expand" must {
|
||||
|
||||
"pass-through elements unchanged when there is no rate difference" in {
|
||||
// Shadow the fuzzed materializer (see the ordering guarantee needed by the for loop below).
|
||||
implicit val materializer = ActorMaterializer(settings.withFuzzing(false))
|
||||
|
||||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.probe[Int]()
|
||||
|
||||
|
|
@ -51,6 +53,9 @@ class FlowExpandSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
publisher.sendNext(-42)
|
||||
|
||||
// The request below is otherwise in race with the above sendNext
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.requestNext(-42)
|
||||
|
||||
subscriber.cancel()
|
||||
|
|
@ -69,6 +74,9 @@ class FlowExpandSpec extends AkkaSpec {
|
|||
publisher.sendNext(2)
|
||||
publisher.sendComplete()
|
||||
|
||||
// The request below is otherwise in race with the above sendNext(2) (and completion)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
|
||||
subscriber.requestNext(2)
|
||||
subscriber.expectComplete()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -137,10 +137,10 @@ class FlowIdleInjectSpec extends AkkaSpec {
|
|||
Source(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink(downstream))
|
||||
|
||||
downstream.request(2)
|
||||
downstream.expectNoMsg(0.5.second)
|
||||
downstream.expectNoMsg(500.millis)
|
||||
downstream.expectNext(0)
|
||||
|
||||
downstream.expectNoMsg(0.5 second)
|
||||
downstream.expectNoMsg(500.millis)
|
||||
downstream.expectNext(0)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -247,7 +247,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
}
|
||||
|
||||
try {
|
||||
val N = 100000
|
||||
val N = 10000
|
||||
Source(1 to N)
|
||||
.mapAsync(parallelism)(i ⇒ deferred())
|
||||
.runFold(0)((c, _) ⇒ c + 1)
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ import scala.concurrent.Await
|
|||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ ActorMaterializerSettings, ActorMaterializer }
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.scaladsl._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
|
@ -235,7 +235,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
|
|||
}
|
||||
|
||||
try {
|
||||
val N = 100000
|
||||
val N = 10000
|
||||
Source(1 to N)
|
||||
.mapAsyncUnordered(parallelism)(i ⇒ deferred())
|
||||
.runFold(0)((c, _) ⇒ c + 1)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue