+str #18798: Fuzzing mode for the interpreter

*PHEAR ME*
This commit is contained in:
Endre Sándor Varga 2015-11-09 16:19:12 +01:00
parent 6a22cdebfc
commit c4417a8c39
19 changed files with 209 additions and 52 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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