Remove warnings from akka-stream-tests (#27114)
This commit is contained in:
parent
03588c3036
commit
231f0d6bb1
122 changed files with 546 additions and 551 deletions
|
|
@ -84,4 +84,11 @@ package object ccompat {
|
|||
}
|
||||
|
||||
object JavaConverters extends scala.collection.convert.DecorateAsJava with scala.collection.convert.DecorateAsScala
|
||||
|
||||
implicit def toTraversableOnceExtensionMethods[A](self: TraversableOnce[A]): TraversableOnceExtensionMethods[A] =
|
||||
new TraversableOnceExtensionMethods[A](self)
|
||||
}
|
||||
|
||||
class TraversableOnceExtensionMethods[A](private val self: c.TraversableOnce[A]) extends AnyVal {
|
||||
def iterator: Iterator[A] = self.toIterator
|
||||
}
|
||||
|
|
|
|||
|
|
@ -326,7 +326,7 @@ public class SourceTest extends StreamTest {
|
|||
|
||||
List<Object> output = probe.receiveN(5);
|
||||
assertEquals(Arrays.asList(4, 3, 2, 1, 0), output);
|
||||
probe.expectNoMsg(FiniteDuration.create(500, TimeUnit.MILLISECONDS));
|
||||
probe.expectNoMessage(FiniteDuration.create(500, TimeUnit.MILLISECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
|||
|
|
@ -11,11 +11,13 @@ import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
|
|||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
import akka.stream.testkit.{ StreamSpec, TestPublisher }
|
||||
import akka.testkit.{ ImplicitSender, TestActor, TestProbe }
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.{ Failure, Try }
|
||||
|
||||
@silent // tests deprecated APIs
|
||||
class ActorMaterializerSpec extends StreamSpec with ImplicitSender {
|
||||
|
||||
"ActorMaterializer" must {
|
||||
|
|
@ -79,7 +81,7 @@ class ActorMaterializerSpec extends StreamSpec with ImplicitSender {
|
|||
|
||||
p.expectMsg("hello")
|
||||
a ! PoisonPill
|
||||
val Failure(ex) = p.expectMsgType[Try[Done]]
|
||||
val Failure(_) = p.expectMsgType[Try[Done]]
|
||||
}
|
||||
|
||||
"handle properly broken Props" in {
|
||||
|
|
|
|||
|
|
@ -107,6 +107,7 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers {
|
|||
jFactoryOption.toList.flatMap(f => getJMethods(f).map(unspecializeName.andThen(curryLikeJava))))
|
||||
}
|
||||
}
|
||||
case invalid => throw new RuntimeException("Invalid testcase: " + invalid)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -118,7 +119,7 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers {
|
|||
sClass.getMethods.filterNot(scalaIgnore contains _.getName).map(toMethod).filterNot(ignore).toList
|
||||
|
||||
private def toMethod(m: java.lang.reflect.Method): Method =
|
||||
Method(m.getName, List(m.getParameterTypes: _*), m.getReturnType, m.getDeclaringClass)
|
||||
Method(m.getName, List(m.getParameterTypes.toIndexedSeq: _*), m.getReturnType, m.getDeclaringClass)
|
||||
|
||||
private case class Ignore(
|
||||
cls: Class[_] => Boolean,
|
||||
|
|
@ -259,7 +260,7 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers {
|
|||
(scalaName, javaName) match {
|
||||
case (s, j) if s == j => true
|
||||
case t if `scala -> java aliases` contains t => true
|
||||
case t => false
|
||||
case _ => false
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -93,7 +93,7 @@ class FanoutProcessorSpec extends StreamSpec {
|
|||
|
||||
"not leak running actors on failed downstream" in assertAllStagesStopped {
|
||||
val probe = TestProbe()
|
||||
val (promise, publisher) = Source.repeat(1).toMat(Sink.asPublisher(true))(Keep.both).run()
|
||||
val (_, publisher) = Source.repeat(1).toMat(Sink.asPublisher(true))(Keep.both).run()
|
||||
val publisherRef = publisher.asInstanceOf[ActorPublisher[Int]].impl
|
||||
probe.watch(publisherRef)
|
||||
Source.fromPublisher(publisher).map(_ => throw TE("boom")).runWith(Sink.ignore)
|
||||
|
|
|
|||
|
|
@ -131,9 +131,9 @@ class FixedBufferSpec extends StreamSpec {
|
|||
|
||||
"produce FixedSizeBuffers when max-fixed-buffer-size < BoundedBufferSize" in {
|
||||
val settings = default.withMaxFixedBufferSize(9)
|
||||
Buffer(5, default) shouldBe a[FixedSizeBuffer.ModuloFixedSizeBuffer[_]]
|
||||
Buffer(10, default) shouldBe a[FixedSizeBuffer.ModuloFixedSizeBuffer[_]]
|
||||
Buffer(16, default) shouldBe a[FixedSizeBuffer.PowerOfTwoFixedSizeBuffer[_]]
|
||||
Buffer(5, settings) shouldBe a[FixedSizeBuffer.ModuloFixedSizeBuffer[_]]
|
||||
Buffer(10, settings) shouldBe a[FixedSizeBuffer.ModuloFixedSizeBuffer[_]]
|
||||
Buffer(16, settings) shouldBe a[FixedSizeBuffer.PowerOfTwoFixedSizeBuffer[_]]
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -231,7 +231,7 @@ class GraphStageLogicSpec extends StreamSpec with GraphInterpreterSpecKit with S
|
|||
interpreter.execute(2)
|
||||
|
||||
expectMsg("postStop2")
|
||||
expectNoMsg(Duration.Zero)
|
||||
expectNoMessage(Duration.Zero)
|
||||
|
||||
interpreter.isCompleted should ===(false)
|
||||
interpreter.isSuspended should ===(false)
|
||||
|
|
|
|||
|
|
@ -152,7 +152,7 @@ class ResizableMultiReaderRingBufferSpec extends WordSpec with Matchers {
|
|||
val random = new Random
|
||||
for {
|
||||
bit <- 1 to MAXSIZEBIT_LIMIT
|
||||
n <- 1 to 2
|
||||
_ <- 1 to 2
|
||||
} {
|
||||
var counter = 1
|
||||
var activeCursors = List.tabulate(random.nextInt(8) + 1)(new StressTestCursor(_, 1 << bit))
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ class TimeoutsSpec extends StreamSpec {
|
|||
Source.maybe[Int].initialTimeout(1.second).runWith(Sink.fromSubscriber(downstreamProbe))
|
||||
|
||||
downstreamProbe.expectSubscription()
|
||||
downstreamProbe.expectNoMsg(500.millis)
|
||||
downstreamProbe.expectNoMessage(500.millis)
|
||||
|
||||
val ex = downstreamProbe.expectError()
|
||||
ex.getMessage should ===("The first element has not yet passed through in 1 second.")
|
||||
|
|
@ -74,11 +74,11 @@ class TimeoutsSpec extends StreamSpec {
|
|||
|
||||
upstreamProbe.sendNext(1)
|
||||
downstreamProbe.requestNext(1)
|
||||
downstreamProbe.expectNoMsg(500.millis) // No timeout yet
|
||||
downstreamProbe.expectNoMessage(500.millis) // No timeout yet
|
||||
|
||||
upstreamProbe.sendNext(2)
|
||||
downstreamProbe.requestNext(2)
|
||||
downstreamProbe.expectNoMsg(500.millis) // No timeout yet
|
||||
downstreamProbe.expectNoMessage(500.millis) // No timeout yet
|
||||
|
||||
val ex = downstreamProbe.expectError()
|
||||
ex.getMessage should ===("The stream has not been completed in 2 seconds.")
|
||||
|
|
@ -111,7 +111,7 @@ class TimeoutsSpec extends StreamSpec {
|
|||
for (_ <- 1 to 4) {
|
||||
upstreamProbe.sendNext(1)
|
||||
downstreamProbe.requestNext(1)
|
||||
downstreamProbe.expectNoMsg(500.millis) // No timeout yet
|
||||
downstreamProbe.expectNoMessage(500.millis) // No timeout yet
|
||||
}
|
||||
|
||||
val ex = downstreamProbe.expectError()
|
||||
|
|
@ -134,7 +134,7 @@ class TimeoutsSpec extends StreamSpec {
|
|||
|
||||
for (i <- 1 to 3) {
|
||||
subscriber.requestNext(i)
|
||||
subscriber.expectNoMsg(250.millis)
|
||||
subscriber.expectNoMessage(250.millis)
|
||||
}
|
||||
|
||||
subscriber.requestNext(4)
|
||||
|
|
@ -149,11 +149,11 @@ class TimeoutsSpec extends StreamSpec {
|
|||
|
||||
subscriber.request(2)
|
||||
|
||||
subscriber.expectNoMsg(1.second)
|
||||
subscriber.expectNoMessage(1.second)
|
||||
publisher.sendNext("Quick Msg")
|
||||
subscriber.expectNext("Quick Msg")
|
||||
|
||||
subscriber.expectNoMsg(3.seconds)
|
||||
subscriber.expectNoMessage(3.seconds)
|
||||
publisher.sendNext("Slow Msg")
|
||||
subscriber.expectNext("Slow Msg")
|
||||
|
||||
|
|
@ -168,7 +168,7 @@ class TimeoutsSpec extends StreamSpec {
|
|||
Source.fromPublisher(publisher).backpressureTimeout(1.second).runWith(Sink.fromSubscriber(subscriber))
|
||||
|
||||
subscriber.request(16)
|
||||
subscriber.expectNoMsg(2.second)
|
||||
subscriber.expectNoMessage(2.second)
|
||||
|
||||
publisher.sendComplete()
|
||||
subscriber.expectComplete()
|
||||
|
|
@ -212,7 +212,7 @@ class TimeoutsSpec extends StreamSpec {
|
|||
publisher.sendNext(1)
|
||||
subscriber.expectNext(1)
|
||||
|
||||
subscriber.expectNoMsg(2.second)
|
||||
subscriber.expectNoMessage(2.second)
|
||||
|
||||
publisher.sendComplete()
|
||||
subscriber.expectComplete()
|
||||
|
|
@ -294,7 +294,7 @@ class TimeoutsSpec extends StreamSpec {
|
|||
downWrite.sendNext(2)
|
||||
upRead.expectNext(2)
|
||||
|
||||
upRead.expectNoMsg(500.millis)
|
||||
upRead.expectNoMessage(500.millis)
|
||||
val error1 = upRead.expectError()
|
||||
val error2 = downRead.expectError()
|
||||
|
||||
|
|
|
|||
|
|
@ -455,7 +455,7 @@ class TraversalBuilderSpec extends AkkaSpec {
|
|||
val sub = TestSubscriber.probe[Int]()
|
||||
val graph = Source.repeat(1).take(10).toMat(Sink.asPublisher(false))(Keep.right)
|
||||
|
||||
val pub = graph.run().subscribe(sub)
|
||||
graph.run().subscribe(sub)
|
||||
|
||||
sub.request(10)
|
||||
sub.expectNextN(List(1, 1, 1, 1, 1, 1, 1, 1, 1, 1))
|
||||
|
|
|
|||
|
|
@ -41,7 +41,6 @@ class ChasingEventsSpec extends AkkaSpec {
|
|||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
||||
new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
private var first = true
|
||||
override def onPush(): Unit = {
|
||||
push(out, grab(in))
|
||||
complete(out)
|
||||
|
|
@ -59,7 +58,6 @@ class ChasingEventsSpec extends AkkaSpec {
|
|||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
||||
new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
private var first = true
|
||||
override def onPush(): Unit = {
|
||||
push(out, grab(in))
|
||||
fail(out, TE("test failure"))
|
||||
|
|
|
|||
|
|
@ -42,7 +42,7 @@ class GraphInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
|||
|
||||
// Constructing an assembly by hand and resolving ambiguities
|
||||
val (logics, _, _) = GraphInterpreterSpecKit.createLogics(Array(identity), Array(source), Array(sink))
|
||||
val connections = GraphInterpreterSpecKit.createLinearFlowConnections(logics)
|
||||
val connections = GraphInterpreterSpecKit.createLinearFlowConnections(logics.toIndexedSeq)
|
||||
|
||||
manualInit(logics, connections)
|
||||
|
||||
|
|
|
|||
|
|
@ -16,9 +16,11 @@ import akka.stream.impl.fusing.GraphInterpreter.{
|
|||
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler, _ }
|
||||
import akka.stream.testkit.StreamSpec
|
||||
import akka.stream.testkit.Utils.TE
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
import scala.collection.{ Map => SMap }
|
||||
|
||||
@silent
|
||||
object GraphInterpreterSpecKit {
|
||||
|
||||
/**
|
||||
|
|
@ -135,7 +137,6 @@ object GraphInterpreterSpecKit {
|
|||
* Create interpreter connections for all the given `connectedPorts`.
|
||||
*/
|
||||
private[stream] def createConnections(
|
||||
logics: Seq[GraphStageLogic],
|
||||
connectedPorts: Seq[(Outlet[_], Inlet[_])],
|
||||
inOwners: SMap[Inlet[_], GraphStageLogic],
|
||||
outOwners: SMap[Outlet[_], GraphStageLogic]): Array[Connection] = {
|
||||
|
|
@ -246,7 +247,7 @@ trait GraphInterpreterSpecKit extends StreamSpec {
|
|||
|
||||
def init(): Unit = {
|
||||
val (logics, inOwners, outOwners) = createLogics(operators.toArray, upstreams.toArray, downstreams.toArray)
|
||||
val conns = createConnections(logics, connectedPorts, inOwners, outOwners)
|
||||
val conns = createConnections(connectedPorts, inOwners, outOwners)
|
||||
|
||||
manualInit(logics.toArray, conns)
|
||||
}
|
||||
|
|
@ -415,14 +416,14 @@ trait GraphInterpreterSpecKit extends StreamSpec {
|
|||
if (!chasing) {
|
||||
val logics = Array[GraphStageLogic](out, in)
|
||||
setLogicIds(logics)
|
||||
val connections = GraphInterpreterSpecKit.createLinearFlowConnections(logics)
|
||||
val connections = GraphInterpreterSpecKit.createLinearFlowConnections(logics.toIndexedSeq)
|
||||
(logics, connections)
|
||||
} else {
|
||||
val propagateStage = new EventPropagateStage
|
||||
setPortIds(propagateStage)
|
||||
val logics = Array[GraphStageLogic](out, propagateStage.createLogic(Attributes.none), in)
|
||||
setLogicIds(logics)
|
||||
val connections = GraphInterpreterSpecKit.createLinearFlowConnections(logics)
|
||||
val connections = GraphInterpreterSpecKit.createLinearFlowConnections(logics.toIndexedSeq)
|
||||
(logics, connections)
|
||||
}
|
||||
|
||||
|
|
@ -520,7 +521,7 @@ trait GraphInterpreterSpecKit extends StreamSpec {
|
|||
val supervision = ActorAttributes.supervisionStrategy(decider)
|
||||
val attributes = Array.fill[Attributes](ops.length)(supervision)
|
||||
val (logics, _, _) = createLogics(ops.toArray, Array(upstream), Array(downstream), attributes)
|
||||
val connections = createLinearFlowConnections(logics)
|
||||
val connections = createLinearFlowConnections(logics.toIndexedSeq)
|
||||
manualInit(logics, connections)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import akka.testkit.EventFilter
|
|||
import akka.stream._
|
||||
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
||||
import akka.util.ConstantFun
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
class InterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
||||
|
||||
|
|
@ -533,6 +534,7 @@ class InterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
@silent
|
||||
private[akka] final case class Doubler[T]() extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
||||
|
|
@ -564,6 +566,7 @@ class InterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
@silent
|
||||
private[akka] final case class KeepGoing[T]() extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
||||
|
|
|
|||
|
|
@ -102,7 +102,7 @@ class KeepGoingStageSpec extends StreamSpec {
|
|||
maybePromise.trySuccess(None)
|
||||
expectMsg(UpstreamCompleted)
|
||||
|
||||
expectNoMsg(200.millis)
|
||||
expectNoMessage(200.millis)
|
||||
|
||||
pinger.ping()
|
||||
expectMsg(Pong)
|
||||
|
|
@ -133,7 +133,7 @@ class KeepGoingStageSpec extends StreamSpec {
|
|||
maybePromise.trySuccess(None)
|
||||
expectMsg(UpstreamCompleted)
|
||||
|
||||
expectNoMsg(200.millis)
|
||||
expectNoMessage(200.millis)
|
||||
|
||||
pinger.ping()
|
||||
expectMsg(Pong)
|
||||
|
|
@ -164,7 +164,7 @@ class KeepGoingStageSpec extends StreamSpec {
|
|||
maybePromise.trySuccess(None)
|
||||
expectMsg(UpstreamCompleted)
|
||||
|
||||
expectNoMsg(200.millis)
|
||||
expectNoMessage(200.millis)
|
||||
|
||||
pinger.ping()
|
||||
expectMsg(Pong)
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ class LifecycleInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
|||
expectMsg("start-a")
|
||||
expectMsg("start-b")
|
||||
expectMsg("start-c")
|
||||
expectNoMsg(300.millis)
|
||||
expectNoMessage(300.millis)
|
||||
upstream.onComplete()
|
||||
}
|
||||
|
||||
|
|
@ -44,7 +44,7 @@ class LifecycleInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
|||
expectMsg("stop-b")
|
||||
expectMsg("complete-c")
|
||||
expectMsg("stop-c")
|
||||
expectNoMsg(300.millis)
|
||||
expectNoMessage(300.millis)
|
||||
}
|
||||
|
||||
"call postStop in order on stages - when upstream onErrors" in new OneBoundedSetup[String](
|
||||
|
|
@ -55,7 +55,7 @@ class LifecycleInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
|||
upstream.onError(TE(msg))
|
||||
expectMsg(msg)
|
||||
expectMsg("stop-c")
|
||||
expectNoMsg(300.millis)
|
||||
expectNoMessage(300.millis)
|
||||
}
|
||||
|
||||
"call postStop in order on stages - when downstream cancels" in new OneBoundedSetup[String](
|
||||
|
|
@ -66,16 +66,16 @@ class LifecycleInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
|||
expectMsg("stop-c")
|
||||
expectMsg("stop-b")
|
||||
expectMsg("stop-a")
|
||||
expectNoMsg(300.millis)
|
||||
expectNoMessage(300.millis)
|
||||
}
|
||||
|
||||
"call preStart before postStop" in new OneBoundedSetup[String](
|
||||
PreStartAndPostStopIdentity(onStart = () => testActor ! "start-a", onStop = () => testActor ! "stop-a")) {
|
||||
expectMsg("start-a")
|
||||
expectNoMsg(300.millis)
|
||||
expectNoMessage(300.millis)
|
||||
upstream.onComplete()
|
||||
expectMsg("stop-a")
|
||||
expectNoMsg(300.millis)
|
||||
expectNoMessage(300.millis)
|
||||
}
|
||||
|
||||
"onError when preStart fails" in new OneBoundedSetup[String](PreStartFailer(() => throw TE("Boom!"))) {
|
||||
|
|
@ -147,7 +147,7 @@ class LifecycleInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
|
|||
onStart: () => Unit = () => (),
|
||||
onStop: () => Unit = () => (),
|
||||
onUpstreamCompleted: () => Unit = () => (),
|
||||
onUpstreamFailed: Throwable => Unit = ex => ())
|
||||
onUpstreamFailed: Throwable => Unit = _ => ())
|
||||
extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic(attributes: Attributes): GraphStageLogic =
|
||||
|
|
|
|||
|
|
@ -147,7 +147,7 @@ class ByteStringParserSpec extends StreamSpec {
|
|||
out.request(1L)
|
||||
in.expectRequest()
|
||||
in.sendNext(ByteString("aha!"))
|
||||
out.expectNoMsg(100.millis)
|
||||
out.expectNoMessage(100.millis)
|
||||
// no new pull
|
||||
in.sendComplete()
|
||||
out.expectError() shouldBe an[IllegalStateException]
|
||||
|
|
|
|||
|
|
@ -145,7 +145,7 @@ class FileSinkSpec extends StreamSpec(UnboundedMailboxConfig) {
|
|||
.runWith(FileIO.toPath(f, options = Set(WRITE, CREATE), startPosition = startPosition))
|
||||
|
||||
val completion1 = write()
|
||||
val result1 = Await.result(completion1, 3.seconds)
|
||||
Await.result(completion1, 3.seconds)
|
||||
|
||||
val completion2 = write(testLinesPart2, startPosition)
|
||||
val result2 = Await.result(completion2, 3.seconds)
|
||||
|
|
|
|||
|
|
@ -9,7 +9,6 @@ import java.nio.file.{ Files, NoSuchFileException }
|
|||
import java.util.Random
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.stream.IOResult._
|
||||
import akka.stream._
|
||||
import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
|
||||
|
|
@ -186,7 +185,7 @@ class FileSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
|
|||
}
|
||||
|
||||
sub.request(demandAllButOneChunks)
|
||||
for (i <- 1 to demandAllButOneChunks) c.expectNext().utf8String should ===(nextChunk())
|
||||
for (_ <- 1 to demandAllButOneChunks) c.expectNext().utf8String should ===(nextChunk())
|
||||
c.expectNoMessage(300.millis)
|
||||
|
||||
sub.request(1)
|
||||
|
|
|
|||
|
|
@ -24,8 +24,6 @@ import akka.util.ByteString
|
|||
import scala.concurrent.duration._
|
||||
import java.util.concurrent.ThreadLocalRandom
|
||||
|
||||
import akka.dispatch.Dispatchers
|
||||
|
||||
import scala.concurrent.{ Await, Future }
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
|
|
|
|||
|
|
@ -30,7 +30,7 @@ class InputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
|
|||
override def read(): Int = 42
|
||||
})
|
||||
|
||||
Await.result(f.takeWithin(5.seconds).runForeach(it => ()), 10.seconds)
|
||||
Await.result(f.takeWithin(5.seconds).runForeach(_ => ()), 10.seconds)
|
||||
}
|
||||
|
||||
"read bytes from InputStream" in assertAllStagesStopped {
|
||||
|
|
|
|||
|
|
@ -8,18 +8,14 @@ import java.io.IOException
|
|||
import java.lang.management.ManagementFactory
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.stream.Attributes.inputBuffer
|
||||
import akka.stream._
|
||||
import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
|
||||
import akka.stream.impl.StreamSupervisor.Children
|
||||
import akka.stream.impl.io.OutputStreamSourceStage
|
||||
import akka.stream.scaladsl.{ Keep, Sink, StreamConverters }
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit.scaladsl.StreamTestKit._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.scaladsl.TestSink
|
||||
import akka.testkit.TestProbe
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -120,7 +116,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
|
|||
val f = Future(outputStream.write(bytesArray))
|
||||
|
||||
expectTimeout(f, timeout)
|
||||
probe.expectNoMsg(Zero)
|
||||
probe.expectNoMessage(Zero)
|
||||
|
||||
s.request(17)
|
||||
expectSuccess(f, ())
|
||||
|
|
@ -171,7 +167,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
|
|||
// make sure previous tests didn't leak
|
||||
assertNoBlockedThreads()
|
||||
|
||||
val (outputStream, probe) =
|
||||
val (_, probe) =
|
||||
StreamConverters.asOutputStream(timeout).toMat(TestSink.probe[ByteString])(Keep.both).run()(materializer)
|
||||
|
||||
val sub = probe.expectSubscription()
|
||||
|
|
@ -186,7 +182,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
|
|||
|
||||
"not leave blocked threads when materializer shutdown" in {
|
||||
val materializer2 = ActorMaterializer(settings)
|
||||
val (outputStream, probe) =
|
||||
val (_, probe) =
|
||||
StreamConverters.asOutputStream(timeout).toMat(TestSink.probe[ByteString])(Keep.both).run()(materializer2)
|
||||
|
||||
val sub = probe.expectSubscription()
|
||||
|
|
@ -203,7 +199,6 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
|
|||
// actually this was a race, so it only happened in at least one of 20 runs
|
||||
|
||||
val bufSize = 4
|
||||
val sourceProbe = TestProbe()
|
||||
val (outputStream, probe) = StreamConverters
|
||||
.asOutputStream(timeout)
|
||||
.addAttributes(Attributes.inputBuffer(bufSize, bufSize))
|
||||
|
|
|
|||
|
|
@ -148,7 +148,7 @@ trait TcpHelper { this: TestKitBase =>
|
|||
connectionActor ! PingClose(connectionProbe.ref)
|
||||
connectionProbe.fishForMessage(max) {
|
||||
case c: ConnectionClosed if p(c) => true
|
||||
case other => false
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -344,7 +344,6 @@ class TcpSpec extends StreamSpec("""
|
|||
|
||||
"shut down both streams when connection is aborted remotely" in assertAllStagesStopped {
|
||||
// Client gets a PeerClosed event and does not know that the write side is also closed
|
||||
val testData = ByteString(1, 2, 3, 4, 5)
|
||||
val server = new Server()
|
||||
|
||||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
|
|
@ -462,7 +461,6 @@ class TcpSpec extends StreamSpec("""
|
|||
""").withFallback(system.settings.config))
|
||||
|
||||
try {
|
||||
implicit val ec: ExecutionContext = system2.dispatcher
|
||||
val mat2 = ActorMaterializer.create(system2)
|
||||
|
||||
val serverAddress = temporaryServerAddress()
|
||||
|
|
@ -686,7 +684,7 @@ class TcpSpec extends StreamSpec("""
|
|||
|
||||
val (bindingFuture, connection) = Tcp(system).bind("localhost", 0).toMat(Sink.head)(Keep.both).run()
|
||||
|
||||
val proxy = connection.map { c =>
|
||||
connection.map { c =>
|
||||
c.handleWith(Flow[ByteString])
|
||||
}
|
||||
|
||||
|
|
@ -852,8 +850,8 @@ class TcpSpec extends StreamSpec("""
|
|||
defaultParams.setCipherSuites(cipherSuites)
|
||||
|
||||
val negotiateNewSession = TLSProtocol.NegotiateNewSession
|
||||
.withCipherSuites(cipherSuites: _*)
|
||||
.withProtocols(protocols: _*)
|
||||
.withCipherSuites(cipherSuites.toIndexedSeq: _*)
|
||||
.withProtocols(protocols.toIndexedSeq: _*)
|
||||
.withParameters(defaultParams)
|
||||
.withClientAuth(TLSClientAuth.None)
|
||||
|
||||
|
|
|
|||
|
|
@ -16,7 +16,6 @@ import scala.concurrent.Await
|
|||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.Random
|
||||
import akka.actor.ActorSystem
|
||||
import akka.pattern.{ after => later }
|
||||
import akka.stream._
|
||||
import akka.stream.TLSProtocol._
|
||||
|
|
@ -60,8 +59,7 @@ object TlsSpec {
|
|||
* independent of the traffic going through. The purpose is to include the last seen
|
||||
* element in the exception message to help in figuring out what went wrong.
|
||||
*/
|
||||
class Timeout(duration: FiniteDuration)(implicit system: ActorSystem)
|
||||
extends GraphStage[FlowShape[ByteString, ByteString]] {
|
||||
class Timeout(duration: FiniteDuration) extends GraphStage[FlowShape[ByteString, ByteString]] {
|
||||
|
||||
private val in = Inlet[ByteString]("in")
|
||||
private val out = Outlet[ByteString]("out")
|
||||
|
|
@ -200,7 +198,7 @@ class TlsSpec extends StreamSpec(TlsSpec.configOverrides) with WithLogCapturing
|
|||
case SessionBytes(s, b) if s != session =>
|
||||
setSession(s)
|
||||
SendBytes(ByteString("NEWSESSION") ++ b)
|
||||
case SessionBytes(s, b) => SendBytes(b)
|
||||
case SessionBytes(_, b) => SendBytes(b)
|
||||
}
|
||||
}
|
||||
def leftClosing: TLSClosing = IgnoreComplete
|
||||
|
|
@ -329,7 +327,7 @@ class TlsSpec extends StreamSpec(TlsSpec.configOverrides) with WithLogCapturing
|
|||
case SessionBytes(s, b) if s != session =>
|
||||
setSession(s)
|
||||
SendBytes(ByteString(s.getCipherSuite) ++ b)
|
||||
case SessionBytes(s, b) => SendBytes(b)
|
||||
case SessionBytes(_, b) => SendBytes(b)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -144,7 +144,7 @@ abstract class CoderSpec(codecName: String) extends WordSpec with CodecSpecSuppo
|
|||
|
||||
val sizesAfterRoundtrip =
|
||||
Source
|
||||
.fromIterator(() => sizes.toIterator.map(createByteString))
|
||||
.fromIterator(() => sizes.iterator.map(createByteString))
|
||||
.via(encoderFlow)
|
||||
.via(decoderFlow())
|
||||
.runFold(Seq.empty[Int])(_ :+ _.size)
|
||||
|
|
|
|||
|
|
@ -15,6 +15,7 @@ import scala.concurrent.duration.Duration
|
|||
import scala.util.{ Failure, Success }
|
||||
|
||||
// a few useful helpers copied over from akka-http
|
||||
@ccompatUsedUntil213
|
||||
object CompressionTestingTools {
|
||||
implicit class AddFutureAwaitResult[T](val future: Future[T]) extends AnyVal {
|
||||
|
||||
|
|
@ -31,7 +32,7 @@ object CompressionTestingTools {
|
|||
}
|
||||
}
|
||||
implicit class EnhancedByteStringTraversableOnce(val byteStrings: IterableOnce[ByteString]) extends AnyVal {
|
||||
def join: ByteString = byteStrings.foldLeft(ByteString.empty)(_ ++ _)
|
||||
def join: ByteString = byteStrings.iterator.foldLeft(ByteString.empty)(_ ++ _)
|
||||
}
|
||||
implicit class EnhancedByteStringSource[Mat](val byteStringStream: Source[ByteString, Mat]) extends AnyVal {
|
||||
def join(implicit materializer: Materializer): Future[ByteString] =
|
||||
|
|
|
|||
|
|
@ -98,7 +98,7 @@ class ActorRefBackpressureSinkSpec extends StreamSpec {
|
|||
expectMsg(initMessage)
|
||||
|
||||
publisher.sendNext(1)
|
||||
expectNoMsg(200.millis)
|
||||
expectNoMessage(200.millis)
|
||||
fw ! TriggerAckMessage
|
||||
expectMsg(1)
|
||||
|
||||
|
|
@ -150,7 +150,7 @@ class ActorRefBackpressureSinkSpec extends StreamSpec {
|
|||
expectMsg(1)
|
||||
|
||||
fw ! TriggerAckMessage
|
||||
expectNoMsg(200.millis) // Ack received but buffer empty
|
||||
expectNoMessage(200.millis) // Ack received but buffer empty
|
||||
|
||||
publisher.sendNext(2) // Buffer this value
|
||||
fw ! TriggerAckMessage
|
||||
|
|
@ -177,7 +177,7 @@ class ActorRefBackpressureSinkSpec extends StreamSpec {
|
|||
.actorRefWithAck[String](probe.ref, initMessage, ackMessage, completeMessage)
|
||||
.withAttributes(inputBuffer(1, 1))
|
||||
|
||||
val maybe = Source.maybe[String].to(sink).run()(mat)
|
||||
Source.maybe[String].to(sink).run()(mat)
|
||||
|
||||
probe.expectMsg(initMessage)
|
||||
mat.shutdown()
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import akka.actor.Status
|
|||
import akka.stream.testkit.Utils.TE
|
||||
import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped
|
||||
import akka.stream.testkit.scaladsl.TestSink
|
||||
import akka.stream.{ ActorMaterializer, Attributes }
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.testkit.StreamSpec
|
||||
import akka.testkit.TestProbe
|
||||
|
||||
|
|
|
|||
|
|
@ -14,7 +14,6 @@ import akka.stream._
|
|||
import scala.concurrent.duration._
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.stream.testkit.TestSubscriber.OnComplete
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
class ActorRefSourceSpec extends StreamSpec {
|
||||
|
|
@ -33,7 +32,7 @@ class ActorRefSourceSpec extends StreamSpec {
|
|||
ref ! 2
|
||||
s.expectNext(2)
|
||||
ref ! 3
|
||||
s.expectNoMsg(500.millis)
|
||||
s.expectNoMessage(500.millis)
|
||||
}
|
||||
|
||||
"buffer when needed" in {
|
||||
|
|
@ -138,7 +137,7 @@ class ActorRefSourceSpec extends StreamSpec {
|
|||
else
|
||||
s.expectNextOrComplete() match {
|
||||
case Right(`n`) => verifyNext(n + 1)
|
||||
case Right(x) => fail("expected $n, got $x")
|
||||
case Right(x) => fail(s"expected $n, got $x")
|
||||
case Left(_) => // ok, completed
|
||||
}
|
||||
}
|
||||
|
|
@ -159,7 +158,7 @@ class ActorRefSourceSpec extends StreamSpec {
|
|||
else
|
||||
s.expectNextOrComplete() match {
|
||||
case Right(`n`) => verifyNext(n + 1)
|
||||
case Right(x) => fail("expected $n, got $x")
|
||||
case Right(x) => fail(s"expected $n, got $x")
|
||||
case Left(_) => // ok, completed
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,6 +15,7 @@ import akka.stream.javadsl
|
|||
import akka.stream.stage._
|
||||
import akka.stream.testkit._
|
||||
import akka.testkit.TestKit
|
||||
import com.github.ghik.silencer.silent
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
object AttributesSpec {
|
||||
|
|
@ -102,6 +103,7 @@ object AttributesSpec {
|
|||
case class WhateverAttribute(label: String) extends Attribute
|
||||
}
|
||||
|
||||
@silent // tests deprecated APIs
|
||||
class AttributesSpec
|
||||
extends StreamSpec(
|
||||
ConfigFactory
|
||||
|
|
|
|||
|
|
@ -9,10 +9,13 @@ import akka.stream.testkit.StreamSpec
|
|||
import akka.stream.testkit.scaladsl.StreamTestKit._
|
||||
import akka.util.ByteString
|
||||
import akka.stream._
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
|
||||
@silent // tests deprecated APIs
|
||||
class BidiFlowSpec extends StreamSpec {
|
||||
import Attributes._
|
||||
import GraphDSL.Implicits._
|
||||
|
|
|
|||
|
|
@ -85,7 +85,7 @@ class CoupledTerminationFlowSpec extends StreamSpec with ScriptedTest {
|
|||
|
||||
"completed out:Source => complete in:Sink" in {
|
||||
val probe = TestProbe()
|
||||
val f = Flow.fromSinkAndSourceCoupledMat(Sink.onComplete(d => probe.ref ! "done"), Source.empty)(Keep.none) // completes right away, should complete the sink as well
|
||||
val f = Flow.fromSinkAndSourceCoupledMat(Sink.onComplete(_ => probe.ref ! "done"), Source.empty)(Keep.none) // completes right away, should complete the sink as well
|
||||
|
||||
f.runWith(Source.maybe, Sink.ignore) // these do nothing.
|
||||
|
||||
|
|
@ -159,11 +159,11 @@ class CoupledTerminationFlowSpec extends StreamSpec with ScriptedTest {
|
|||
}
|
||||
val assertCompleteAndCancel = () => {
|
||||
probe.expectMsgPF() {
|
||||
case Success(v) => // good
|
||||
case Success(_) => // good
|
||||
case "cancel-received" => // good
|
||||
}
|
||||
probe.expectMsgPF() {
|
||||
case Success(v) => // good
|
||||
case Success(_) => // good
|
||||
case "cancel-received" => // good
|
||||
}
|
||||
}
|
||||
|
|
@ -174,11 +174,11 @@ class CoupledTerminationFlowSpec extends StreamSpec with ScriptedTest {
|
|||
}
|
||||
val assertErrorAndCancel = () => {
|
||||
probe.expectMsgPF() {
|
||||
case Failure(ex) => // good
|
||||
case Failure(_) => // good
|
||||
case "cancel-received" => // good
|
||||
}
|
||||
probe.expectMsgPF() {
|
||||
case Failure(ex) => // good
|
||||
case Failure(_) => // good
|
||||
case "cancel-received" => // good
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -63,8 +63,8 @@ object FlowAskSpec {
|
|||
|
||||
class FailOnAllExcept(n: Int) extends Actor {
|
||||
override def receive: Receive = {
|
||||
case `n` => sender() ! akka.actor.Status.Success(Reply(n))
|
||||
case msg: Int => sender() ! akka.actor.Status.Failure(new Exception(s"Booming for $n!"))
|
||||
case `n` => sender() ! akka.actor.Status.Success(Reply(n))
|
||||
case _: Int => sender() ! akka.actor.Status.Failure(new Exception(s"Booming for $n!"))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -103,8 +103,7 @@ class FlowAskSpec extends StreamSpec {
|
|||
|
||||
"produce asked elements" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 3).ask[Reply](4)(replyOnInts).runWith(Sink.fromSubscriber(c))
|
||||
Source(1 to 3).ask[Reply](4)(replyOnInts).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(Reply(1))
|
||||
|
|
@ -116,8 +115,7 @@ class FlowAskSpec extends StreamSpec {
|
|||
}
|
||||
"produce asked elements (simple ask)" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 3).ask[Reply](replyOnInts).runWith(Sink.fromSubscriber(c))
|
||||
Source(1 to 3).ask[Reply](replyOnInts).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(Reply(1))
|
||||
|
|
@ -129,8 +127,7 @@ class FlowAskSpec extends StreamSpec {
|
|||
}
|
||||
"produce asked elements, when replies are akka.actor.Status.Success" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 3).ask[Reply](4)(statusReplier).runWith(Sink.fromSubscriber(c))
|
||||
Source(1 to 3).ask[Reply](4)(statusReplier).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(Reply(1))
|
||||
|
|
@ -143,8 +140,7 @@ class FlowAskSpec extends StreamSpec {
|
|||
|
||||
"produce future elements in order" in {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 50).ask[Reply](4)(replyRandomDelays).to(Sink.fromSubscriber(c)).run()
|
||||
Source(1 to 50).ask[Reply](4)(replyRandomDelays).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(1000)
|
||||
for (n <- 1 to 50) c.expectNext(Reply(n))
|
||||
|
|
@ -153,9 +149,8 @@ class FlowAskSpec extends StreamSpec {
|
|||
|
||||
"signal ask timeout failure" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
Source(1 to 5)
|
||||
.map(_ + " nope")
|
||||
.map(s => s"$s + nope")
|
||||
.ask[Reply](4)(dontReply)(akka.util.Timeout(10.millis), implicitly[ClassTag[Reply]])
|
||||
.to(Sink.fromSubscriber(c))
|
||||
.run()
|
||||
|
|
@ -166,8 +161,7 @@ class FlowAskSpec extends StreamSpec {
|
|||
"signal ask failure" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
val ref = failsOn1
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5).ask[Reply](4)(ref).to(Sink.fromSubscriber(c)).run()
|
||||
Source(1 to 5).ask[Reply](4)(ref).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectError().getMessage should be("Booming for 1!")
|
||||
|
|
@ -221,9 +215,8 @@ class FlowAskSpec extends StreamSpec {
|
|||
|
||||
"resume after ask failure" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Reply]()
|
||||
implicit val ec = system.dispatcher
|
||||
val ref = failsOn3
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.ask[Reply](4)(ref)
|
||||
.withAttributes(supervisionStrategy(resumingDecider))
|
||||
.to(Sink.fromSubscriber(c))
|
||||
|
|
|
|||
|
|
@ -52,7 +52,7 @@ class FlowBatchSpec extends StreamSpec {
|
|||
for (i <- 1 to 10) {
|
||||
publisher.sendNext(i)
|
||||
}
|
||||
subscriber.expectNoMsg(1.second)
|
||||
subscriber.expectNoMessage(1.second)
|
||||
sub.request(1)
|
||||
subscriber.expectNext(List(10, 9, 8, 7, 6, 5, 4, 3, 2, 1))
|
||||
sub.cancel()
|
||||
|
|
@ -84,19 +84,19 @@ class FlowBatchSpec extends StreamSpec {
|
|||
subscriber.expectNext(1)
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(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)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
sub.request(1)
|
||||
subscriber.expectNext(7)
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
sub.cancel()
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ class FlowBatchWeightedSpec extends StreamSpec {
|
|||
subscriber.expectNext(1)
|
||||
|
||||
publisher.sendNext(3)
|
||||
subscriber.expectNoMsg(1.second)
|
||||
subscriber.expectNoMessage(1.second)
|
||||
|
||||
sub.request(2)
|
||||
subscriber.expectNext(2)
|
||||
|
|
|
|||
|
|
@ -27,7 +27,6 @@ class FlowBufferSpec extends StreamSpec {
|
|||
}
|
||||
|
||||
"pass elements through normally in backpressured mode with buffer size one" in {
|
||||
val futureSink = Sink.head[Seq[Int]]
|
||||
val future =
|
||||
Source(1 to 1000).buffer(1, overflowStrategy = OverflowStrategy.backpressure).grouped(1001).runWith(Sink.head)
|
||||
Await.result(future, 3.seconds) should be(1 to 1000)
|
||||
|
|
@ -83,7 +82,7 @@ class FlowBufferSpec extends StreamSpec {
|
|||
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)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
|
||||
// drain
|
||||
for (i <- 101 to 200) {
|
||||
|
|
@ -92,7 +91,7 @@ class FlowBufferSpec extends StreamSpec {
|
|||
}
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(1.seconds)
|
||||
subscriber.expectNoMessage(1.seconds)
|
||||
|
||||
publisher.sendNext(-1)
|
||||
sub.request(1)
|
||||
|
|
@ -116,7 +115,7 @@ class FlowBufferSpec extends StreamSpec {
|
|||
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)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
|
||||
// drain
|
||||
for (i <- 1 to 99) {
|
||||
|
|
@ -128,7 +127,7 @@ class FlowBufferSpec extends StreamSpec {
|
|||
subscriber.expectNext(200)
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(1.seconds)
|
||||
subscriber.expectNoMessage(1.seconds)
|
||||
|
||||
publisher.sendNext(-1)
|
||||
sub.request(1)
|
||||
|
|
@ -152,7 +151,7 @@ class FlowBufferSpec extends StreamSpec {
|
|||
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)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
|
||||
// drain
|
||||
for (i <- 101 to 150) {
|
||||
|
|
@ -161,7 +160,7 @@ class FlowBufferSpec extends StreamSpec {
|
|||
}
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(1.seconds)
|
||||
subscriber.expectNoMessage(1.seconds)
|
||||
|
||||
publisher.sendNext(-1)
|
||||
sub.request(1)
|
||||
|
|
@ -183,7 +182,7 @@ class FlowBufferSpec extends StreamSpec {
|
|||
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)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
|
||||
// drain
|
||||
for (i <- 1 to 100) {
|
||||
|
|
@ -191,7 +190,7 @@ class FlowBufferSpec extends StreamSpec {
|
|||
}
|
||||
|
||||
subscriber.request(1)
|
||||
subscriber.expectNoMsg(1.seconds)
|
||||
subscriber.expectNoMessage(1.seconds)
|
||||
|
||||
publisher.sendNext(-1)
|
||||
subscriber.requestNext(-1)
|
||||
|
|
@ -241,12 +240,12 @@ class FlowBufferSpec extends StreamSpec {
|
|||
for (i <- 1 to 200) publisher.sendNext(i)
|
||||
|
||||
// The request below is in race otherwise with the onNext(200) above
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
sub.request(1)
|
||||
subscriber.expectNext(200)
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(1.seconds)
|
||||
subscriber.expectNoMessage(1.seconds)
|
||||
|
||||
publisher.sendNext(-1)
|
||||
sub.request(1)
|
||||
|
|
|
|||
|
|
@ -10,10 +10,11 @@ import org.reactivestreams.Publisher
|
|||
|
||||
import scala.collection.immutable.Seq
|
||||
import scala.concurrent.Future
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent // unused vars are used in shouldNot compile tests
|
||||
class FlowCompileSpec extends StreamSpec {
|
||||
|
||||
val intSeq = Source(Seq(1, 2, 3))
|
||||
|
|
|
|||
|
|
@ -131,19 +131,19 @@ class FlowConflateSpec extends StreamSpec {
|
|||
subscriber.expectNext(1)
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(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)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
sub.request(1)
|
||||
subscriber.expectNext(7)
|
||||
|
||||
sub.request(1)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
sub.cancel()
|
||||
|
||||
}
|
||||
|
|
@ -161,7 +161,7 @@ class FlowConflateSpec extends StreamSpec {
|
|||
val sinkProbe = TestSubscriber.probe[Int]()
|
||||
val exceptionLatch = TestLatch()
|
||||
|
||||
val future = Source
|
||||
Source
|
||||
.fromPublisher(sourceProbe)
|
||||
.conflateWithSeed { i =>
|
||||
if (i % 2 == 0) {
|
||||
|
|
@ -243,7 +243,7 @@ class FlowConflateSpec extends StreamSpec {
|
|||
val sinkProbe = TestSubscriber.probe[Vector[Int]]()
|
||||
val saw4Latch = TestLatch()
|
||||
|
||||
val future = Source
|
||||
Source
|
||||
.fromPublisher(sourceProbe)
|
||||
.conflateWithSeed(seed = i => Vector(i))((state, elem) =>
|
||||
if (elem == 2) {
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ class FlowDelaySpec extends StreamSpec {
|
|||
.delay(1.second)
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(10)
|
||||
.expectNoMsg(1800.millis)
|
||||
.expectNoMessage(1800.millis)
|
||||
.expectNext(300.millis, 1)
|
||||
.expectNextN(2 to 10)
|
||||
.expectComplete()
|
||||
|
|
@ -44,10 +44,10 @@ class FlowDelaySpec extends StreamSpec {
|
|||
.delay(300.millis)
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(2)
|
||||
.expectNoMsg(200.millis) //delay
|
||||
.expectNoMessage(200.millis) //delay
|
||||
.expectNext(200.millis, 1) //delayed element
|
||||
.expectNext(100.millis, 2) //buffered element
|
||||
.expectNoMsg(200.millis)
|
||||
.expectNoMessage(200.millis)
|
||||
.request(1)
|
||||
.expectNext(3) //buffered element
|
||||
.expectComplete()
|
||||
|
|
@ -62,10 +62,10 @@ class FlowDelaySpec extends StreamSpec {
|
|||
val pSub = p.expectSubscription()
|
||||
cSub.request(100)
|
||||
pSub.sendNext(1)
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
c.expectNext(1)
|
||||
pSub.sendNext(2)
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
c.expectNext(2)
|
||||
pSub.sendComplete()
|
||||
c.expectComplete()
|
||||
|
|
@ -81,11 +81,11 @@ class FlowDelaySpec extends StreamSpec {
|
|||
cSub.request(100)
|
||||
pSub.sendNext(1)
|
||||
pSub.sendNext(2)
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
pSub.sendNext(3)
|
||||
c.expectNext(1)
|
||||
c.expectNext(2)
|
||||
c.expectNoMsg(150.millis)
|
||||
c.expectNoMessage(150.millis)
|
||||
c.expectNext(3)
|
||||
pSub.sendComplete()
|
||||
c.expectComplete()
|
||||
|
|
@ -127,11 +127,11 @@ class FlowDelaySpec extends StreamSpec {
|
|||
.withAttributes(inputBuffer(1, 1))
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(5)
|
||||
.expectNoMsg(200.millis)
|
||||
.expectNoMessage(200.millis)
|
||||
.expectNext(200.millis, 1)
|
||||
.expectNoMsg(200.millis)
|
||||
.expectNoMessage(200.millis)
|
||||
.expectNext(200.millis, 2)
|
||||
.expectNoMsg(200.millis)
|
||||
.expectNoMessage(200.millis)
|
||||
.expectNext(200.millis, 3)
|
||||
}
|
||||
|
||||
|
|
@ -160,7 +160,7 @@ class FlowDelaySpec extends StreamSpec {
|
|||
cSub.request(20)
|
||||
|
||||
for (i <- 1 to 16) pSub.sendNext(i)
|
||||
c.expectNoMsg(300.millis)
|
||||
c.expectNoMessage(300.millis)
|
||||
pSub.sendNext(17)
|
||||
c.expectNext(100.millis, 1)
|
||||
//fail will terminate despite of non empty internal buffer
|
||||
|
|
@ -178,7 +178,7 @@ class FlowDelaySpec extends StreamSpec {
|
|||
.runWith(Sink.ignore)
|
||||
.pipeTo(testActor)
|
||||
|
||||
expectNoMsg(2.seconds)
|
||||
expectNoMessage(2.seconds)
|
||||
expectMsg(Done)
|
||||
|
||||
// With a buffer large enough to hold all arriving elements, delays don't add up
|
||||
|
|
@ -199,7 +199,7 @@ class FlowDelaySpec extends StreamSpec {
|
|||
.runWith(Sink.ignore)
|
||||
.pipeTo(testActor)
|
||||
|
||||
expectNoMsg(900.millis)
|
||||
expectNoMessage(900.millis)
|
||||
expectMsg(Done)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -36,7 +36,7 @@ class FlowDetacherSpec extends StreamSpec {
|
|||
.detach
|
||||
.runWith(TestSink.probe)
|
||||
.ensureSubscription()
|
||||
.expectNoMsg(500.millis)
|
||||
.expectNoMessage(500.millis)
|
||||
.requestNext() should ===(42)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ class FlowDispatcherSpec extends StreamSpec(s"my-dispatcher = $${akka.test.strea
|
|||
implicit val materializer = ActorMaterializer(settings)
|
||||
|
||||
val probe = TestProbe()
|
||||
val p = Source(List(1, 2, 3)).map(i => { probe.ref ! Thread.currentThread().getName(); i }).to(Sink.ignore).run()
|
||||
Source(List(1, 2, 3)).map(i => { probe.ref ! Thread.currentThread().getName(); i }).to(Sink.ignore).run()
|
||||
probe.receiveN(3).foreach {
|
||||
case s: String => s should startWith(system.name + "-" + dispatcher)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ class FlowDropWithinSpec extends StreamSpec {
|
|||
pSub.sendNext(input.next())
|
||||
}
|
||||
val demand3 = pSub.expectRequest
|
||||
c.expectNoMsg(1500.millis)
|
||||
c.expectNoMessage(1500.millis)
|
||||
(1 to demand3.toInt).foreach { _ =>
|
||||
pSub.sendNext(input.next())
|
||||
}
|
||||
|
|
@ -40,7 +40,7 @@ class FlowDropWithinSpec extends StreamSpec {
|
|||
}
|
||||
pSub.sendComplete()
|
||||
c.expectComplete
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
"deliver completion even before the duration" in {
|
||||
|
|
|
|||
|
|
@ -55,7 +55,7 @@ class FlowExpandSpec extends StreamSpec {
|
|||
publisher.sendNext(-42)
|
||||
|
||||
// The request below is otherwise in race with the above sendNext
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
subscriber.requestNext(-42)
|
||||
|
||||
subscriber.cancel()
|
||||
|
|
@ -75,7 +75,7 @@ class FlowExpandSpec extends StreamSpec {
|
|||
publisher.sendComplete()
|
||||
|
||||
// The request below is otherwise in race with the above sendNext(2) (and completion)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
|
||||
subscriber.requestNext(2)
|
||||
subscriber.expectComplete()
|
||||
|
|
@ -117,7 +117,7 @@ class FlowExpandSpec extends StreamSpec {
|
|||
pending -= 1
|
||||
}
|
||||
|
||||
publisher.expectNoMsg(1.second)
|
||||
publisher.expectNoMessage(1.second)
|
||||
|
||||
subscriber.request(2)
|
||||
subscriber.expectNext(2)
|
||||
|
|
@ -132,7 +132,7 @@ class FlowExpandSpec extends StreamSpec {
|
|||
val (source, sink) =
|
||||
TestSource.probe[Int].expand(i => Iterator.from(0).map(i -> _).take(3)).toMat(TestSink.probe)(Keep.both).run()
|
||||
source.sendNext(1)
|
||||
sink.request(4).expectNext(1 -> 0, 1 -> 1, 1 -> 2).expectNoMsg(100.millis)
|
||||
sink.request(4).expectNext(1 -> 0, 1 -> 1, 1 -> 2).expectNoMessage(100.millis)
|
||||
source.sendNext(2).sendComplete()
|
||||
sink.expectNext(2 -> 0).expectComplete()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -61,7 +61,7 @@ class FlowExtrapolateSpec extends StreamSpec {
|
|||
publisher.sendNext(-42)
|
||||
|
||||
// The request below is otherwise in race with the above sendNext
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
subscriber.requestNext(-42)
|
||||
|
||||
subscriber.cancel()
|
||||
|
|
@ -101,7 +101,7 @@ class FlowExtrapolateSpec extends StreamSpec {
|
|||
publisher.sendComplete()
|
||||
|
||||
// The request below is otherwise in race with the above sendNext(2) (and completion)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscriber.expectNoMessage(500.millis)
|
||||
|
||||
subscriber.requestNext(2)
|
||||
subscriber.expectComplete()
|
||||
|
|
@ -143,7 +143,7 @@ class FlowExtrapolateSpec extends StreamSpec {
|
|||
pending -= 1
|
||||
}
|
||||
|
||||
publisher.expectNoMsg(1.second)
|
||||
publisher.expectNoMessage(1.second)
|
||||
|
||||
subscriber.request(2)
|
||||
subscriber.expectNext(2)
|
||||
|
|
@ -158,7 +158,7 @@ class FlowExtrapolateSpec extends StreamSpec {
|
|||
val (source, sink) =
|
||||
TestSource.probe[Int].expand(i => Iterator.from(0).map(i -> _).take(3)).toMat(TestSink.probe)(Keep.both).run()
|
||||
source.sendNext(1)
|
||||
sink.request(4).expectNext(1 -> 0, 1 -> 1, 1 -> 2).expectNoMsg(100.millis)
|
||||
sink.request(4).expectNext(1 -> 0, 1 -> 1, 1 -> 2).expectNoMessage(100.millis)
|
||||
source.sendNext(2).sendComplete()
|
||||
sink.expectNext(2 -> 0).expectComplete()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -82,9 +82,9 @@ class FlowFlattenMergeSpec extends StreamSpec {
|
|||
|
||||
"bubble up substream exceptions" in assertAllStagesStopped {
|
||||
val ex = new Exception("buh")
|
||||
val result = intercept[TestFailedException] {
|
||||
Source(List(blocked, blocked, Source.failed(ex))).flatMapMerge(10, identity).runWith(Sink.head).futureValue
|
||||
}.cause.get should ===(ex)
|
||||
intercept[TestFailedException] {
|
||||
Source(List(blocked, blocked, Source.failed(ex))).flatMapMerge(10, identity).runWith(Sink.head).futureValue
|
||||
}.cause.get should ===(ex)
|
||||
}
|
||||
|
||||
"bubble up substream materialization exception" in assertAllStagesStopped {
|
||||
|
|
@ -152,7 +152,6 @@ class FlowFlattenMergeSpec extends StreamSpec {
|
|||
|
||||
"cancel substreams when being cancelled" in assertAllStagesStopped {
|
||||
val p1, p2 = TestPublisher.probe[Int]()
|
||||
val ex = new Exception("buh")
|
||||
val sink = Source(List(Source.fromPublisher(p1), Source.fromPublisher(p2)))
|
||||
.flatMapMerge(5, identity)
|
||||
.runWith(TestSink.probe)
|
||||
|
|
@ -168,9 +167,9 @@ class FlowFlattenMergeSpec extends StreamSpec {
|
|||
val p = Source((0 until 100).map(i => src10(10 * i))).flatMapMerge(Int.MaxValue, identity).runWith(TestSink.probe)
|
||||
p.within(1.second) {
|
||||
p.ensureSubscription()
|
||||
p.expectNoMsg()
|
||||
p.expectNoMessage(remainingOrDefault)
|
||||
}
|
||||
val elems = p.within(1.second)((1 to 1000).map(i => p.requestNext()).toSet)
|
||||
val elems = p.within(1.second)((1 to 1000).map(_ => p.requestNext()).toSet)
|
||||
p.expectComplete()
|
||||
elems should ===((0 until 1000).toSet)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -200,7 +200,7 @@ class FlowFoldAsyncSpec extends StreamSpec {
|
|||
"resume when foldAsync throws" in {
|
||||
val c = TestSubscriber.manualProbe[(Int, Int)]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.foldAsync(0 -> 1) {
|
||||
case ((i, res), n) =>
|
||||
if (n == 3) throw new RuntimeException("err4") with NoStackTrace
|
||||
|
|
@ -218,7 +218,7 @@ class FlowFoldAsyncSpec extends StreamSpec {
|
|||
"restart when foldAsync throws" in {
|
||||
val c = TestSubscriber.manualProbe[(Int, Int)]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.foldAsync(0 -> 1) {
|
||||
case ((i, res), n) =>
|
||||
if (n == 3) throw new RuntimeException("err4") with NoStackTrace
|
||||
|
|
@ -235,8 +235,8 @@ class FlowFoldAsyncSpec extends StreamSpec {
|
|||
|
||||
"signal NPE when future is completed with null" in {
|
||||
val c = TestSubscriber.manualProbe[String]()
|
||||
val p = Source(List("a", "b"))
|
||||
.foldAsync("") { (_, elem) =>
|
||||
Source(List("a", "b"))
|
||||
.foldAsync("") { (_, _) =>
|
||||
Future.successful(null.asInstanceOf[String])
|
||||
}
|
||||
.to(Sink.fromSubscriber(c))
|
||||
|
|
@ -248,7 +248,7 @@ class FlowFoldAsyncSpec extends StreamSpec {
|
|||
|
||||
"resume when future is completed with null" in {
|
||||
val c = TestSubscriber.manualProbe[String]()
|
||||
val p = Source(List("a", "b", "c"))
|
||||
Source(List("a", "b", "c"))
|
||||
.foldAsync("") { (str, elem) =>
|
||||
if (elem == "b") Future.successful(null.asInstanceOf[String])
|
||||
else Future.successful(str + elem)
|
||||
|
|
@ -264,7 +264,7 @@ class FlowFoldAsyncSpec extends StreamSpec {
|
|||
|
||||
"restart when future is completed with null" in {
|
||||
val c = TestSubscriber.manualProbe[String]()
|
||||
val p = Source(List("a", "b", "c"))
|
||||
Source(List("a", "b", "c"))
|
||||
.foldAsync("") { (str, elem) =>
|
||||
if (elem == "b") Future.successful(null.asInstanceOf[String])
|
||||
else Future.successful(str + elem)
|
||||
|
|
|
|||
|
|
@ -21,9 +21,9 @@ class FlowFromFutureSpec extends StreamSpec {
|
|||
"A Flow based on a Future" must {
|
||||
"produce one element from already successful Future" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
val p = Source.fromFuture(Future.successful(1)).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
Source.fromFuture(Future.successful(1)).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
sub.request(1)
|
||||
c.expectNext(1)
|
||||
c.expectComplete()
|
||||
|
|
@ -42,11 +42,11 @@ class FlowFromFutureSpec extends StreamSpec {
|
|||
Source.fromFuture(promise.future).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(1)
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
promise.success(1)
|
||||
c.expectNext(1)
|
||||
c.expectComplete()
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"produce one element when Future is completed but not before request" in {
|
||||
|
|
@ -55,7 +55,7 @@ class FlowFromFutureSpec extends StreamSpec {
|
|||
Source.fromFuture(promise.future).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
promise.success(1)
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
sub.request(1)
|
||||
c.expectNext(1)
|
||||
c.expectComplete()
|
||||
|
|
@ -89,9 +89,9 @@ class FlowFromFutureSpec extends StreamSpec {
|
|||
val sub = c.expectSubscription()
|
||||
sub.request(1)
|
||||
sub.cancel()
|
||||
c.expectNoMsg(500.millis)
|
||||
c.expectNoMessage(500.millis)
|
||||
promise.success(1)
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -52,7 +52,7 @@ class FlowGroupBySpec extends StreamSpec {
|
|||
|
||||
def request(demand: Int): Unit = subscription.request(demand)
|
||||
def expectNext(elem: Int): Unit = probe.expectNext(elem)
|
||||
def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max)
|
||||
def expectNoMessage(max: FiniteDuration): Unit = probe.expectNoMessage(max)
|
||||
def expectComplete(): Unit = probe.expectComplete()
|
||||
def expectError(e: Throwable) = probe.expectError(e)
|
||||
def cancel(): Unit = subscription.cancel()
|
||||
|
|
@ -91,16 +91,16 @@ class FlowGroupBySpec extends StreamSpec {
|
|||
"work in the happy case" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(groupCount = 2) {
|
||||
val s1 = StreamPuppet(getSubFlow(1).runWith(Sink.asPublisher(false)))
|
||||
masterSubscriber.expectNoMsg(100.millis)
|
||||
masterSubscriber.expectNoMessage(100.millis)
|
||||
|
||||
s1.expectNoMsg(100.millis)
|
||||
s1.expectNoMessage(100.millis)
|
||||
s1.request(1)
|
||||
s1.expectNext(1)
|
||||
s1.expectNoMsg(100.millis)
|
||||
s1.expectNoMessage(100.millis)
|
||||
|
||||
val s2 = StreamPuppet(getSubFlow(0).runWith(Sink.asPublisher(false)))
|
||||
|
||||
s2.expectNoMsg(100.millis)
|
||||
s2.expectNoMessage(100.millis)
|
||||
s2.request(2)
|
||||
s2.expectNext(2)
|
||||
|
||||
|
|
@ -108,7 +108,7 @@ class FlowGroupBySpec extends StreamSpec {
|
|||
s1.request(1)
|
||||
s2.expectNext(4)
|
||||
|
||||
s2.expectNoMsg(100.millis)
|
||||
s2.expectNoMessage(100.millis)
|
||||
|
||||
s1.expectNext(3)
|
||||
|
||||
|
|
@ -157,7 +157,7 @@ class FlowGroupBySpec extends StreamSpec {
|
|||
substream.request(2)
|
||||
substream.expectNext(2)
|
||||
substream.expectNext(4)
|
||||
substream.expectNoMsg(100.millis)
|
||||
substream.expectNoMessage(100.millis)
|
||||
|
||||
substream.request(2)
|
||||
substream.expectNext(6)
|
||||
|
|
@ -357,7 +357,7 @@ class FlowGroupBySpec extends StreamSpec {
|
|||
|
||||
"emit subscribe before completed" in assertAllStagesStopped {
|
||||
val futureGroupSource =
|
||||
Source.single(0).groupBy(1, elem => "all").prefixAndTail(0).map(_._2).concatSubstreams.runWith(Sink.head)
|
||||
Source.single(0).groupBy(1, _ => "all").prefixAndTail(0).map(_._2).concatSubstreams.runWith(Sink.head)
|
||||
val pub: Publisher[Int] = Await.result(futureGroupSource, 3.seconds).runWith(Sink.asPublisher(false))
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
pub.subscribe(probe)
|
||||
|
|
@ -387,7 +387,7 @@ class FlowGroupBySpec extends StreamSpec {
|
|||
val downstreamSubscription = subscriber.expectSubscription()
|
||||
|
||||
downstreamSubscription.request(300)
|
||||
for (i <- 1 to 300) {
|
||||
for (_ <- 1 to 300) {
|
||||
val byteString = randomByteString(10)
|
||||
upstreamSubscription.expectRequest()
|
||||
upstreamSubscription.sendNext(byteString)
|
||||
|
|
@ -417,9 +417,9 @@ class FlowGroupBySpec extends StreamSpec {
|
|||
|
||||
// Both will attempt to pull upstream
|
||||
substream.request(1)
|
||||
substream.expectNoMsg(100.millis)
|
||||
substream.expectNoMessage(100.millis)
|
||||
downstreamMaster.request(1)
|
||||
downstreamMaster.expectNoMsg(100.millis)
|
||||
downstreamMaster.expectNoMessage(100.millis)
|
||||
|
||||
// Cleanup, not part of the actual test
|
||||
substream.cancel()
|
||||
|
|
@ -612,7 +612,7 @@ class FlowGroupBySpec extends StreamSpec {
|
|||
|
||||
val upstreamSubscription = publisherProbe.expectSubscription()
|
||||
|
||||
for (i <- 1 to 400) {
|
||||
for (_ <- 1 to 400) {
|
||||
val byteString = randomByteString(10)
|
||||
val index = Math.abs(byteString.head % 100)
|
||||
|
||||
|
|
|
|||
|
|
@ -44,16 +44,16 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
|
|||
(1 to demand3).foreach { _ =>
|
||||
pSub.sendNext(input.next())
|
||||
}
|
||||
c.expectNoMsg(300.millis)
|
||||
c.expectNoMessage(300.millis)
|
||||
c.expectNext(((demand1 + demand2 + 1).toInt to (demand1 + demand2 + demand3).toInt).toVector)
|
||||
c.expectNoMsg(300.millis)
|
||||
c.expectNoMessage(300.millis)
|
||||
pSub.expectRequest
|
||||
val last = input.next()
|
||||
pSub.sendNext(last)
|
||||
pSub.sendComplete()
|
||||
c.expectNext(List(last))
|
||||
c.expectComplete
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
"deliver bufferd elements onComplete before the timeout" taggedAs TimingTest in {
|
||||
|
|
@ -63,7 +63,7 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
|
|||
cSub.request(100)
|
||||
c.expectNext((1 to 3).toList)
|
||||
c.expectComplete
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
"buffer groups until requested from downstream" taggedAs TimingTest in {
|
||||
|
|
@ -83,12 +83,12 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
|
|||
(1 to demand2).foreach { _ =>
|
||||
pSub.sendNext(input.next())
|
||||
}
|
||||
c.expectNoMsg(300.millis)
|
||||
c.expectNoMessage(300.millis)
|
||||
cSub.request(1)
|
||||
c.expectNext(((demand1 + 1) to (demand1 + demand2)).toVector)
|
||||
pSub.sendComplete()
|
||||
c.expectComplete
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"drop empty groups" taggedAs TimingTest in {
|
||||
|
|
@ -99,14 +99,14 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
|
|||
val cSub = c.expectSubscription
|
||||
cSub.request(2)
|
||||
pSub.expectRequest
|
||||
c.expectNoMsg(600.millis)
|
||||
c.expectNoMessage(600.millis)
|
||||
pSub.sendNext(1)
|
||||
pSub.sendNext(2)
|
||||
c.expectNext(List(1, 2))
|
||||
// nothing more requested
|
||||
c.expectNoMsg(1100.millis)
|
||||
c.expectNoMessage(1100.millis)
|
||||
cSub.request(3)
|
||||
c.expectNoMsg(600.millis)
|
||||
c.expectNoMessage(600.millis)
|
||||
pSub.sendComplete()
|
||||
c.expectComplete
|
||||
}
|
||||
|
|
@ -129,14 +129,14 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
|
|||
Source.fromPublisher(upstream).groupedWithin(3, 2.second).to(Sink.fromSubscriber(downstream)).run()
|
||||
|
||||
downstream.request(2)
|
||||
downstream.expectNoMsg(1000.millis)
|
||||
downstream.expectNoMessage(1000.millis)
|
||||
|
||||
(1 to 4).foreach(upstream.sendNext)
|
||||
downstream.within(1000.millis) {
|
||||
downstream.expectNext((1 to 3).toVector)
|
||||
}
|
||||
|
||||
downstream.expectNoMsg(1500.millis)
|
||||
downstream.expectNoMessage(1500.millis)
|
||||
|
||||
downstream.within(1000.millis) {
|
||||
downstream.expectNext(List(4))
|
||||
|
|
@ -144,7 +144,7 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
|
|||
|
||||
upstream.sendComplete()
|
||||
downstream.expectComplete()
|
||||
downstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"reset time window when exact max elements reached" taggedAs TimingTest in {
|
||||
|
|
@ -217,14 +217,14 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
|
|||
.run()
|
||||
|
||||
downstream.ensureSubscription()
|
||||
downstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
upstream.sendNext(1)
|
||||
upstream.sendNext(2)
|
||||
upstream.sendNext(3)
|
||||
upstream.sendComplete()
|
||||
downstream.request(1)
|
||||
downstream.expectNext(Vector(1, 2): immutable.Seq[Long])
|
||||
downstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
downstream.request(1)
|
||||
downstream.expectNext(Vector(3): immutable.Seq[Long])
|
||||
downstream.expectComplete()
|
||||
|
|
@ -287,7 +287,7 @@ class FlowGroupedWithinSpec extends StreamSpec with ScriptedTest {
|
|||
upstream.sendNext("22")
|
||||
upstream.sendNext("333")
|
||||
upstream.sendNext("22")
|
||||
downstream.expectNoMsg(50.millis)
|
||||
downstream.expectNoMessage(50.millis)
|
||||
downstream.expectNext(Vector("333", "22", "333", "22"): immutable.Seq[String])
|
||||
upstream.sendComplete()
|
||||
downstream.expectComplete()
|
||||
|
|
|
|||
|
|
@ -26,9 +26,8 @@ class FlowIdleInjectSpec extends StreamSpec {
|
|||
"emit elements periodically after silent periods" in assertAllStagesStopped {
|
||||
val sourceWithIdleGap = Source(1 to 5) ++ Source(6 to 10).initialDelay(2.second)
|
||||
|
||||
val result = Await.result(
|
||||
sourceWithIdleGap.keepAlive(0.6.seconds, () => 0).grouped(1000).runWith(Sink.head),
|
||||
3.seconds) should ===(List(1, 2, 3, 4, 5, 0, 0, 0, 6, 7, 8, 9, 10))
|
||||
Await.result(sourceWithIdleGap.keepAlive(0.6.seconds, () => 0).grouped(1000).runWith(Sink.head), 3.seconds) should ===(
|
||||
List(1, 2, 3, 4, 5, 0, 0, 0, 6, 7, 8, 9, 10))
|
||||
}
|
||||
|
||||
"immediately pull upstream" in {
|
||||
|
|
@ -73,7 +72,7 @@ class FlowIdleInjectSpec extends StreamSpec {
|
|||
Source.fromPublisher(upstream).keepAlive(1.second, () => 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.ensureSubscription()
|
||||
downstream.expectNoMsg(1.5.second)
|
||||
downstream.expectNoMessage(1.5.second)
|
||||
downstream.request(1)
|
||||
downstream.expectNext(0)
|
||||
|
||||
|
|
@ -92,7 +91,7 @@ class FlowIdleInjectSpec extends StreamSpec {
|
|||
downstream.request(10)
|
||||
downstream.expectNextN(1 to 10)
|
||||
|
||||
downstream.expectNoMsg(1.5.second)
|
||||
downstream.expectNoMessage(1.5.second)
|
||||
downstream.request(1)
|
||||
downstream.expectNext(0)
|
||||
|
||||
|
|
@ -107,9 +106,9 @@ class FlowIdleInjectSpec extends StreamSpec {
|
|||
Source.fromPublisher(upstream).keepAlive(1.second, () => 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.ensureSubscription()
|
||||
downstream.expectNoMsg(1.5.second)
|
||||
downstream.expectNoMessage(1.5.second)
|
||||
upstream.sendNext(1)
|
||||
downstream.expectNoMsg(0.5.second)
|
||||
downstream.expectNoMessage(0.5.second)
|
||||
downstream.request(1)
|
||||
downstream.expectNext(1)
|
||||
|
||||
|
|
@ -128,9 +127,9 @@ class FlowIdleInjectSpec extends StreamSpec {
|
|||
downstream.request(10)
|
||||
downstream.expectNextN(1 to 10)
|
||||
|
||||
downstream.expectNoMsg(1.5.second)
|
||||
downstream.expectNoMessage(1.5.second)
|
||||
upstream.sendNext(1)
|
||||
downstream.expectNoMsg(0.5.second)
|
||||
downstream.expectNoMessage(0.5.second)
|
||||
downstream.request(1)
|
||||
downstream.expectNext(1)
|
||||
|
||||
|
|
@ -145,10 +144,10 @@ class FlowIdleInjectSpec extends StreamSpec {
|
|||
Source.fromPublisher(upstream).keepAlive(1.second, () => 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.request(2)
|
||||
downstream.expectNoMsg(500.millis)
|
||||
downstream.expectNoMessage(500.millis)
|
||||
downstream.expectNext(0)
|
||||
|
||||
downstream.expectNoMsg(500.millis)
|
||||
downstream.expectNoMessage(500.millis)
|
||||
downstream.expectNext(0)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ class FlowInitialDelaySpec extends StreamSpec {
|
|||
Source(1 to 10).initialDelay(0.5.second).runWith(Sink.fromSubscriber(probe))
|
||||
|
||||
probe.ensureSubscription()
|
||||
probe.expectNoMsg(1.5.second)
|
||||
probe.expectNoMessage(1.5.second)
|
||||
probe.request(20)
|
||||
probe.expectNextN(1 to 10)
|
||||
|
||||
|
|
|
|||
|
|
@ -36,11 +36,11 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
|
|||
val sub = c.expectSubscription()
|
||||
sub.request(1)
|
||||
c.expectNext(1)
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
sub.request(2)
|
||||
c.expectError().getMessage should be("not two")
|
||||
sub.request(2)
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"produce onError when Source construction throws" in {
|
||||
|
|
@ -51,7 +51,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
|
|||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
c.expectSubscriptionAndError().getMessage should be("no good iterator")
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"produce onError when hasNext throws" in {
|
||||
|
|
@ -65,7 +65,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
|
|||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
c.expectSubscriptionAndError().getMessage should be("no next")
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -88,7 +88,7 @@ abstract class AbstractFlowIteratorSpec extends StreamSpec {
|
|||
val sub = c.expectSubscription()
|
||||
sub.request(1)
|
||||
c.expectNext(1)
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
sub.request(3)
|
||||
c.expectNext(2)
|
||||
c.expectNext(3)
|
||||
|
|
@ -100,7 +100,7 @@ abstract class AbstractFlowIteratorSpec extends StreamSpec {
|
|||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
c.expectSubscriptionAndComplete()
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"produce elements with multiple subscribers" in assertAllStagesStopped {
|
||||
|
|
@ -116,8 +116,8 @@ abstract class AbstractFlowIteratorSpec extends StreamSpec {
|
|||
c1.expectNext(1)
|
||||
c2.expectNext(1)
|
||||
c2.expectNext(2)
|
||||
c1.expectNoMsg(100.millis)
|
||||
c2.expectNoMsg(100.millis)
|
||||
c1.expectNoMessage(100.millis)
|
||||
c2.expectNoMessage(100.millis)
|
||||
sub1.request(2)
|
||||
sub2.request(2)
|
||||
c1.expectNext(2)
|
||||
|
|
@ -136,7 +136,7 @@ abstract class AbstractFlowIteratorSpec extends StreamSpec {
|
|||
val sub1 = c1.expectSubscription()
|
||||
sub1.request(1)
|
||||
c1.expectNext(1)
|
||||
c1.expectNoMsg(100.millis)
|
||||
c1.expectNoMessage(100.millis)
|
||||
p.subscribe(c2)
|
||||
val sub2 = c2.expectSubscription()
|
||||
sub2.request(3)
|
||||
|
|
@ -182,7 +182,7 @@ abstract class AbstractFlowIteratorSpec extends StreamSpec {
|
|||
c.expectNext(1)
|
||||
sub.cancel()
|
||||
sub.request(2)
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -47,7 +47,7 @@ class FlowJoinSpec extends StreamSpec(ConfigFactory.parseString("akka.loglevel=I
|
|||
.buffer((end + 1) / 2, OverflowStrategy.backpressure)
|
||||
.take((end + 1) / 2)
|
||||
|
||||
val mm = flow1.join(flow2).run()
|
||||
flow1.join(flow2).run()
|
||||
|
||||
val sub = probe.expectSubscription()
|
||||
sub.request(1)
|
||||
|
|
|
|||
|
|
@ -80,8 +80,8 @@ class FlowKillSwitchSpec extends StreamSpec {
|
|||
downstream.expectComplete()
|
||||
|
||||
switch.abort(TE("Won't happen"))
|
||||
upstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMsg(100.millis)
|
||||
upstream.expectNoMessage(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -178,12 +178,12 @@ class FlowKillSwitchSpec extends StreamSpec {
|
|||
downstream.expectComplete()
|
||||
|
||||
switch.shutdown()
|
||||
upstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMsg(100.millis)
|
||||
upstream.expectNoMessage(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
|
||||
switch.abort(TE("Abort"))
|
||||
upstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMsg(100.millis)
|
||||
upstream.expectNoMessage(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"ignore subsequent aborts and shutdowns after abort" in assertAllStagesStopped {
|
||||
|
|
@ -200,12 +200,12 @@ class FlowKillSwitchSpec extends StreamSpec {
|
|||
downstream.expectError(TE("Abort"))
|
||||
|
||||
switch.shutdown()
|
||||
upstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMsg(100.millis)
|
||||
upstream.expectNoMessage(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
|
||||
switch.abort(TE("Abort_Late"))
|
||||
upstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMsg(100.millis)
|
||||
upstream.expectNoMessage(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"complete immediately flows materialized after switch shutdown" in assertAllStagesStopped {
|
||||
|
|
@ -262,12 +262,12 @@ class FlowKillSwitchSpec extends StreamSpec {
|
|||
switch1.shutdown()
|
||||
upstream1.expectCancellation()
|
||||
downstream1.expectComplete()
|
||||
upstream2.expectNoMsg(100.millis)
|
||||
downstream2.expectNoMsg(100.millis)
|
||||
upstream2.expectNoMessage(100.millis)
|
||||
downstream2.expectNoMessage(100.millis)
|
||||
|
||||
switch2.abort(TE("Abort"))
|
||||
upstream1.expectNoMsg(100.millis)
|
||||
downstream1.expectNoMsg(100.millis)
|
||||
upstream1.expectNoMessage(100.millis)
|
||||
downstream1.expectNoMessage(100.millis)
|
||||
upstream2.expectCancellation()
|
||||
downstream2.expectError(TE("Abort"))
|
||||
}
|
||||
|
|
@ -289,10 +289,10 @@ class FlowKillSwitchSpec extends StreamSpec {
|
|||
.run()
|
||||
|
||||
downstream.ensureSubscription()
|
||||
downstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
|
||||
switch1.shutdown()
|
||||
downstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
|
||||
switch2.shutdown()
|
||||
downstream.expectComplete()
|
||||
|
|
|
|||
|
|
@ -6,6 +6,8 @@ package akka.stream.scaladsl
|
|||
|
||||
import akka.stream.testkit.StreamSpec
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, StreamLimitReachedException }
|
||||
import akka.util.unused
|
||||
|
||||
import scala.concurrent.Await
|
||||
|
||||
class FlowLimitWeightedSpec extends StreamSpec {
|
||||
|
|
@ -18,7 +20,7 @@ class FlowLimitWeightedSpec extends StreamSpec {
|
|||
"produce empty sequence regardless of cost when source is empty and n = 0" in {
|
||||
val input = Range(0, 0, 1)
|
||||
val n = input.length
|
||||
def costFn(e: Int): Long = 999999L // set to an arbitrarily big value
|
||||
def costFn(@unused e: Int): Long = 999999L // set to an arbitrarily big value
|
||||
val future = Source(input).limitWeighted(n)(costFn).grouped(Integer.MAX_VALUE).runWith(Sink.headOption)
|
||||
val result = Await.result(future, remainingOrDefault)
|
||||
result should be(None)
|
||||
|
|
@ -26,7 +28,7 @@ class FlowLimitWeightedSpec extends StreamSpec {
|
|||
|
||||
"always exhaust a source regardless of n (as long as n > 0) if cost is 0" in {
|
||||
val input = (1 to 15)
|
||||
def costFn(e: Int): Long = 0L
|
||||
def costFn(@unused e: Int): Long = 0L
|
||||
val n = 1 // must not matter since costFn always evaluates to 0
|
||||
val future = Source(input).limitWeighted(n)(costFn).grouped(Integer.MAX_VALUE).runWith(Sink.head)
|
||||
val result = Await.result(future, remainingOrDefault)
|
||||
|
|
@ -35,7 +37,7 @@ class FlowLimitWeightedSpec extends StreamSpec {
|
|||
|
||||
"exhaust source if n equals to input length and cost is 1" in {
|
||||
val input = (1 to 16)
|
||||
def costFn(e: Int): Long = 1L
|
||||
def costFn(@unused e: Int): Long = 1L
|
||||
val n = input.length
|
||||
val future = Source(input).limitWeighted(n)(costFn).grouped(Integer.MAX_VALUE).runWith(Sink.head)
|
||||
val result = Await.result(future, remainingOrDefault)
|
||||
|
|
|
|||
|
|
@ -146,7 +146,7 @@ class FlowLogSpec extends StreamSpec("""
|
|||
"follow supervision strategy when exception thrown" in {
|
||||
val ex = new RuntimeException() with NoStackTrace
|
||||
val future = Source(1 to 5)
|
||||
.log("hi", n => throw ex)
|
||||
.log("hi", _ => throw ex)
|
||||
.withAttributes(supervisionStrategy(resumingDecider))
|
||||
.runWith(Sink.fold(0)(_ + _))
|
||||
Await.result(future, 500.millis) shouldEqual 0
|
||||
|
|
|
|||
|
|
@ -10,7 +10,6 @@ import java.util.concurrent.atomic.AtomicInteger
|
|||
import akka.stream.ActorAttributes.supervisionStrategy
|
||||
import akka.stream.{ ActorAttributes, ActorMaterializer, Supervision }
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit.scaladsl.StreamTestKit._
|
||||
import akka.stream.testkit._
|
||||
|
|
@ -32,12 +31,12 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
"produce future elements" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 3).mapAsync(4)(n => Future(n)).runWith(Sink.fromSubscriber(c))
|
||||
Source(1 to 3).mapAsync(4)(n => Future(n)).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(1)
|
||||
c.expectNext(2)
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
sub.request(2)
|
||||
c.expectNext(3)
|
||||
c.expectComplete()
|
||||
|
|
@ -46,7 +45,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
"produce future elements in order" in {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 50)
|
||||
Source(1 to 50)
|
||||
.mapAsync(4)(
|
||||
n =>
|
||||
if (n % 3 == 0) Future.successful(n)
|
||||
|
|
@ -67,7 +66,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
val probe = TestProbe()
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 20)
|
||||
Source(1 to 20)
|
||||
.mapAsync(8)(n =>
|
||||
Future {
|
||||
probe.ref ! n
|
||||
|
|
@ -76,26 +75,26 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
.to(Sink.fromSubscriber(c))
|
||||
.run()
|
||||
val sub = c.expectSubscription()
|
||||
probe.expectNoMsg(500.millis)
|
||||
probe.expectNoMessage(500.millis)
|
||||
sub.request(1)
|
||||
probe.receiveN(9).toSet should be((1 to 9).toSet)
|
||||
probe.expectNoMsg(500.millis)
|
||||
probe.expectNoMessage(500.millis)
|
||||
sub.request(2)
|
||||
probe.receiveN(2).toSet should be(Set(10, 11))
|
||||
probe.expectNoMsg(500.millis)
|
||||
probe.expectNoMessage(500.millis)
|
||||
sub.request(10)
|
||||
probe.receiveN(9).toSet should be((12 to 20).toSet)
|
||||
probe.expectNoMsg(200.millis)
|
||||
probe.expectNoMessage(200.millis)
|
||||
|
||||
for (n <- 1 to 13) c.expectNext(n)
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
"signal future already failed" in assertAllStagesStopped {
|
||||
val latch = TestLatch(1)
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.mapAsync(4)(
|
||||
n =>
|
||||
if (n == 3) Future.failed[Int](new TE("err1"))
|
||||
|
|
@ -116,7 +115,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
val latch = TestLatch(1)
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.mapAsync(4)(n =>
|
||||
Future {
|
||||
if (n == 3) throw new RuntimeException("err1") with NoStackTrace
|
||||
|
|
@ -169,16 +168,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
val probe =
|
||||
Source.fromIterator(() => input.iterator).mapAsync(5)(p => p.future.map(_.toUpperCase)).runWith(TestSink.probe)
|
||||
|
||||
import TestSubscriber._
|
||||
var gotErrorAlready = false
|
||||
val elementOrErrorOk: PartialFunction[SubscriberEvent, Unit] = {
|
||||
case OnNext("A") => () // is fine
|
||||
case OnNext("B") => () // is fine
|
||||
case OnError(ex) if ex.getMessage == "Boom at C" && !gotErrorAlready =>
|
||||
gotErrorAlready = true // fine, error can over-take elements
|
||||
}
|
||||
probe.request(100)
|
||||
|
||||
val boom = new Exception("Boom at C")
|
||||
|
||||
// placing the future completion signals here is important
|
||||
|
|
@ -199,8 +189,11 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
probe.expectNextOrError() match {
|
||||
case Left(ex) => ex.getMessage should ===("Boom at C") // fine, error can over-take elements
|
||||
case Right(element) => fail(s"Got [$element] yet it caused an exception, should not have happened!")
|
||||
case unexpected => fail(s"unexpected $unexpected")
|
||||
}
|
||||
case unexpected => fail(s"unexpected $unexpected")
|
||||
}
|
||||
case unexpeced => fail(s"unexpected $unexpeced")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -235,7 +228,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
val latch = TestLatch(1)
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.mapAsync(4)(n =>
|
||||
if (n == 3) throw new RuntimeException("err2") with NoStackTrace
|
||||
else {
|
||||
|
|
@ -255,7 +248,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
"resume after future failure" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.mapAsync(4)(n =>
|
||||
Future {
|
||||
if (n == 3) throw new RuntimeException("err3") with NoStackTrace
|
||||
|
|
@ -272,8 +265,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
|
||||
"resume after already failed future" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.mapAsync(4)(n =>
|
||||
if (n == 3) Future.failed(new TE("err3"))
|
||||
else Future.successful(n))
|
||||
|
|
@ -318,7 +310,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
"resume when mapAsync throws" in {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.mapAsync(4)(n =>
|
||||
if (n == 3) throw new RuntimeException("err4") with NoStackTrace
|
||||
else Future(n))
|
||||
|
|
@ -434,7 +426,7 @@ class FlowMapAsyncSpec extends StreamSpec {
|
|||
try {
|
||||
val N = 10000
|
||||
Source(1 to N)
|
||||
.mapAsync(parallelism)(i => deferred())
|
||||
.mapAsync(parallelism)(_ => deferred())
|
||||
.runFold(0)((c, _) => c + 1)
|
||||
.futureValue(Timeout(3.seconds)) should ===(N)
|
||||
} finally {
|
||||
|
|
|
|||
|
|
@ -16,7 +16,6 @@ import akka.testkit.TestLatch
|
|||
import akka.testkit.TestProbe
|
||||
import akka.stream.ActorAttributes.supervisionStrategy
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import scala.concurrent.Promise
|
||||
|
|
@ -35,7 +34,7 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
|
|||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val latch = (1 to 4).map(_ -> TestLatch(1)).toMap
|
||||
val p = Source(1 to 4)
|
||||
Source(1 to 4)
|
||||
.mapAsyncUnordered(4)(n =>
|
||||
Future {
|
||||
Await.ready(latch(n), 5.seconds)
|
||||
|
|
@ -60,7 +59,7 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
|
|||
val probe = TestProbe()
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 20)
|
||||
Source(1 to 20)
|
||||
.mapAsyncUnordered(4)(n =>
|
||||
if (n % 3 == 0) {
|
||||
probe.ref ! n
|
||||
|
|
@ -73,12 +72,12 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
|
|||
.to(Sink.fromSubscriber(c))
|
||||
.run()
|
||||
val sub = c.expectSubscription()
|
||||
c.expectNoMsg(200.millis)
|
||||
probe.expectNoMsg(Duration.Zero)
|
||||
c.expectNoMessage(200.millis)
|
||||
probe.expectNoMessage(Duration.Zero)
|
||||
sub.request(1)
|
||||
var got = Set(c.expectNext())
|
||||
probe.expectMsgAllOf(1, 2, 3, 4, 5)
|
||||
probe.expectNoMsg(500.millis)
|
||||
probe.expectNoMessage(500.millis)
|
||||
sub.request(25)
|
||||
probe.expectMsgAllOf(6 to 20: _*)
|
||||
c.within(3.seconds) {
|
||||
|
|
@ -93,7 +92,7 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
|
|||
val latch = TestLatch(1)
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.mapAsyncUnordered(4)(n =>
|
||||
Future {
|
||||
if (n == 3) throw new RuntimeException("err1") with NoStackTrace
|
||||
|
|
@ -136,7 +135,7 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
|
|||
val latch = TestLatch(1)
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 5)
|
||||
Source(1 to 5)
|
||||
.mapAsyncUnordered(4)(n =>
|
||||
if (n == 3) throw new RuntimeException("err2") with NoStackTrace
|
||||
else {
|
||||
|
|
@ -317,7 +316,7 @@ class FlowMapAsyncUnorderedSpec extends StreamSpec {
|
|||
try {
|
||||
val N = 10000
|
||||
Source(1 to N)
|
||||
.mapAsyncUnordered(parallelism)(i => deferred())
|
||||
.mapAsyncUnordered(parallelism)(_ => deferred())
|
||||
.runFold(0)((c, _) => c + 1)
|
||||
.futureValue(Timeout(3.seconds)) should ===(N)
|
||||
} finally {
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ class FlowMapErrorSpec extends StreamSpec {
|
|||
.map { a =>
|
||||
if (a == 3) throw ex else a
|
||||
}
|
||||
.mapError { case t: Throwable => boom }
|
||||
.mapError { case _: Throwable => boom }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(3)
|
||||
.expectNext(1)
|
||||
|
|
@ -39,7 +39,7 @@ class FlowMapErrorSpec extends StreamSpec {
|
|||
.map { a =>
|
||||
if (a == 2) throw ex else a
|
||||
}
|
||||
.mapError { case t: Exception => throw boom }
|
||||
.mapError { case _: Exception => throw boom }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.requestNext(1)
|
||||
.request(1)
|
||||
|
|
@ -51,7 +51,7 @@ class FlowMapErrorSpec extends StreamSpec {
|
|||
.map { a =>
|
||||
if (a == 2) throw ex else a
|
||||
}
|
||||
.mapError { case t: IndexOutOfBoundsException => boom }
|
||||
.mapError { case _: IndexOutOfBoundsException => boom }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.requestNext(1)
|
||||
.request(1)
|
||||
|
|
@ -61,7 +61,7 @@ class FlowMapErrorSpec extends StreamSpec {
|
|||
"not influence stream when there is no exceptions" in assertAllStagesStopped {
|
||||
Source(1 to 3)
|
||||
.map(identity)
|
||||
.mapError { case t: Throwable => boom }
|
||||
.mapError { case _: Throwable => boom }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(3)
|
||||
.expectNextN(1 to 3)
|
||||
|
|
@ -71,7 +71,7 @@ class FlowMapErrorSpec extends StreamSpec {
|
|||
"finish stream if it's empty" in assertAllStagesStopped {
|
||||
Source.empty
|
||||
.map(identity)
|
||||
.mapError { case t: Throwable => boom }
|
||||
.mapError { case _: Throwable => boom }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(1)
|
||||
.expectComplete()
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ class FlowMonitorSpec extends StreamSpec {
|
|||
}
|
||||
|
||||
"return Finished when stream is cancelled from downstream" in {
|
||||
val ((source, monitor), sink) =
|
||||
val ((_, monitor), sink) =
|
||||
TestSource.probe[Any].monitorMat(Keep.both).toMat(TestSink.probe[Any])(Keep.both).run()
|
||||
sink.cancel()
|
||||
awaitAssert(monitor.state == Finished, 3.seconds)
|
||||
|
|
@ -72,7 +72,7 @@ class FlowMonitorSpec extends StreamSpec {
|
|||
|
||||
"return Failed when stream is abruptly terminated" in {
|
||||
val mat = ActorMaterializer()
|
||||
val (source, monitor) = // notice that `monitor` is like a Keep.both
|
||||
val (_, monitor) = // notice that `monitor` is like a Keep.both
|
||||
TestSource.probe[Any].monitor.to(Sink.ignore).run()(mat)
|
||||
mat.shutdown()
|
||||
|
||||
|
|
|
|||
|
|
@ -29,7 +29,7 @@ class FlowOnCompleteSpec extends StreamSpec with ScriptedTest {
|
|||
val proc = p.expectSubscription
|
||||
proc.expectRequest()
|
||||
proc.sendNext(42)
|
||||
onCompleteProbe.expectNoMsg(100.millis)
|
||||
onCompleteProbe.expectNoMessage(100.millis)
|
||||
proc.sendComplete()
|
||||
onCompleteProbe.expectMsg(Success(Done))
|
||||
}
|
||||
|
|
@ -43,7 +43,7 @@ class FlowOnCompleteSpec extends StreamSpec with ScriptedTest {
|
|||
val ex = new RuntimeException("ex") with NoStackTrace
|
||||
proc.sendError(ex)
|
||||
onCompleteProbe.expectMsg(Failure(ex))
|
||||
onCompleteProbe.expectNoMsg(100.millis)
|
||||
onCompleteProbe.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"invoke callback for an empty stream" in assertAllStagesStopped {
|
||||
|
|
@ -54,7 +54,7 @@ class FlowOnCompleteSpec extends StreamSpec with ScriptedTest {
|
|||
proc.expectRequest()
|
||||
proc.sendComplete()
|
||||
onCompleteProbe.expectMsg(Success(Done))
|
||||
onCompleteProbe.expectNoMsg(100.millis)
|
||||
onCompleteProbe.expectNoMessage(100.millis)
|
||||
}
|
||||
|
||||
"invoke callback after transform and foreach steps " in assertAllStagesStopped {
|
||||
|
|
|
|||
|
|
@ -107,7 +107,7 @@ class FlowOrElseSpec extends AkkaSpec {
|
|||
"complete when both inputs completes without emitting elements, regardless of order" in new OrElseProbedFlow {
|
||||
outProbe.ensureSubscription()
|
||||
inProbe2.sendComplete()
|
||||
outProbe.expectNoMsg(200.millis) // make sure it did not complete here
|
||||
outProbe.expectNoMessage(200.millis) // make sure it did not complete here
|
||||
inProbe1.sendComplete()
|
||||
outProbe.expectComplete()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -148,7 +148,7 @@ class FlowPrefixAndTailSpec extends StreamSpec {
|
|||
"shut down main stage if substream is empty, even when not subscribed" in assertAllStagesStopped {
|
||||
val futureSink = newHeadSink
|
||||
val fut = Source.single(1).prefixAndTail(1).runWith(futureSink)
|
||||
val (takes, tail) = Await.result(fut, 3.seconds)
|
||||
val (takes, _) = Await.result(fut, 3.seconds)
|
||||
takes should be(Seq(1))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,16 +4,19 @@
|
|||
|
||||
package akka.stream.scaladsl
|
||||
|
||||
//#prepend
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Sink
|
||||
|
||||
//#prepend
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.testkit.AkkaSpec
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent // for keeping imports
|
||||
class FlowPrependSpec extends AkkaSpec {
|
||||
|
||||
//#prepend
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Sink
|
||||
|
||||
//#prepend
|
||||
|
||||
val settings = ActorMaterializerSettings(system)
|
||||
|
||||
implicit val materializer = ActorMaterializer(settings)
|
||||
|
|
|
|||
|
|
@ -25,7 +25,7 @@ class FlowRecoverSpec extends StreamSpec {
|
|||
.map { a =>
|
||||
if (a == 3) throw ex else a
|
||||
}
|
||||
.recover { case t: Throwable => 0 }
|
||||
.recover { case _: Throwable => 0 }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.requestNext(1)
|
||||
.requestNext(2)
|
||||
|
|
@ -39,7 +39,7 @@ class FlowRecoverSpec extends StreamSpec {
|
|||
.map { a =>
|
||||
if (a == 2) throw ex else a
|
||||
}
|
||||
.recover { case t: IndexOutOfBoundsException => 0 }
|
||||
.recover { case _: IndexOutOfBoundsException => 0 }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.requestNext(1)
|
||||
.request(1)
|
||||
|
|
@ -49,7 +49,7 @@ class FlowRecoverSpec extends StreamSpec {
|
|||
"not influence stream when there is no exceptions" in assertAllStagesStopped {
|
||||
Source(1 to 3)
|
||||
.map(identity)
|
||||
.recover { case t: Throwable => 0 }
|
||||
.recover { case _: Throwable => 0 }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(3)
|
||||
.expectNextN(1 to 3)
|
||||
|
|
@ -59,7 +59,7 @@ class FlowRecoverSpec extends StreamSpec {
|
|||
"finish stream if it's empty" in assertAllStagesStopped {
|
||||
Source.empty
|
||||
.map(identity)
|
||||
.recover { case t: Throwable => 0 }
|
||||
.recover { case _: Throwable => 0 }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(1)
|
||||
.expectComplete()
|
||||
|
|
|
|||
|
|
@ -10,9 +10,11 @@ import akka.stream.testkit.scaladsl.TestSink
|
|||
import akka.stream._
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit.scaladsl.StreamTestKit._
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
@silent // tests deprecated APIs
|
||||
class FlowRecoverWithSpec extends StreamSpec {
|
||||
|
||||
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1)
|
||||
|
|
@ -27,7 +29,7 @@ class FlowRecoverWithSpec extends StreamSpec {
|
|||
.map { a =>
|
||||
if (a == 3) throw ex else a
|
||||
}
|
||||
.recoverWith { case t: Throwable => Source(List(0, -1)) }
|
||||
.recoverWith { case _: Throwable => Source(List(0, -1)) }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(2)
|
||||
.expectNextN(1 to 2)
|
||||
|
|
@ -43,7 +45,7 @@ class FlowRecoverWithSpec extends StreamSpec {
|
|||
.map { a =>
|
||||
if (a == 3) throw ex else a
|
||||
}
|
||||
.recoverWith { case t: Throwable => Source(List(0, -1)) }
|
||||
.recoverWith { case _: Throwable => Source(List(0, -1)) }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(2)
|
||||
.expectNextN(1 to 2)
|
||||
|
|
@ -57,7 +59,7 @@ class FlowRecoverWithSpec extends StreamSpec {
|
|||
.map { a =>
|
||||
if (a == 2) throw ex else a
|
||||
}
|
||||
.recoverWith { case t: IndexOutOfBoundsException => Source.single(0) }
|
||||
.recoverWith { case _: IndexOutOfBoundsException => Source.single(0) }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(1)
|
||||
.expectNext(1)
|
||||
|
|
@ -70,7 +72,7 @@ class FlowRecoverWithSpec extends StreamSpec {
|
|||
if (a == 3) throw ex else a
|
||||
}
|
||||
src
|
||||
.recoverWith { case t: Throwable => src }
|
||||
.recoverWith { case _: Throwable => src }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(2)
|
||||
.expectNextN(1 to 2)
|
||||
|
|
@ -84,7 +86,7 @@ class FlowRecoverWithSpec extends StreamSpec {
|
|||
"not influence stream when there is no exceptions" in assertAllStagesStopped {
|
||||
Source(1 to 3)
|
||||
.map(identity)
|
||||
.recoverWith { case t: Throwable => Source.single(0) }
|
||||
.recoverWith { case _: Throwable => Source.single(0) }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(3)
|
||||
.expectNextN(1 to 3)
|
||||
|
|
@ -94,14 +96,14 @@ class FlowRecoverWithSpec extends StreamSpec {
|
|||
"finish stream if it's empty" in assertAllStagesStopped {
|
||||
Source.empty
|
||||
.map(identity)
|
||||
.recoverWith { case t: Throwable => Source.single(0) }
|
||||
.recoverWith { case _: Throwable => Source.single(0) }
|
||||
.runWith(TestSink.probe[Int])
|
||||
.request(3)
|
||||
.expectComplete()
|
||||
}
|
||||
|
||||
"switch the second time if alternative source throws exception" in assertAllStagesStopped {
|
||||
val k = Source(1 to 3)
|
||||
Source(1 to 3)
|
||||
.map { a =>
|
||||
if (a == 3) throw new IndexOutOfBoundsException() else a
|
||||
}
|
||||
|
|
|
|||
|
|
@ -226,10 +226,9 @@ class FlowScanAsyncSpec extends StreamSpec {
|
|||
def whenEventualFuture(
|
||||
promises: immutable.Seq[Promise[Int]],
|
||||
zero: Int,
|
||||
decider: Supervision.Decider = Supervision.stoppingDecider)
|
||||
: (TestPublisher.Probe[Int], TestSubscriber.Probe[Int]) = {
|
||||
decider: Supervision.Decider): (TestPublisher.Probe[Int], TestSubscriber.Probe[Int]) = {
|
||||
require(promises.nonEmpty, "must be at least one promise")
|
||||
val promiseScanFlow = Flow[Int].scanAsync(zero) { (accumulator: Int, next: Int) =>
|
||||
val promiseScanFlow = Flow[Int].scanAsync(zero) { (_: Int, next: Int) =>
|
||||
promises(next).future
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -10,7 +10,9 @@ import akka.stream.testkit._
|
|||
import org.scalacheck.Gen
|
||||
import org.scalatest.prop.GeneratorDrivenPropertyChecks
|
||||
import akka.pattern.pipe
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent
|
||||
class FlowSlidingSpec extends StreamSpec with GeneratorDrivenPropertyChecks {
|
||||
import system.dispatcher
|
||||
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16)
|
||||
|
|
|
|||
|
|
@ -13,6 +13,7 @@ import akka.stream.testkit.scaladsl.StreamTestKit._
|
|||
import akka.stream.testkit._
|
||||
import akka.stream._
|
||||
import akka.testkit.TestDuration
|
||||
import com.github.ghik.silencer.silent
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.reactivestreams.{ Publisher, Subscriber }
|
||||
|
||||
|
|
@ -22,7 +23,7 @@ import scala.concurrent.duration._
|
|||
import scala.util.control.NoStackTrace
|
||||
|
||||
object FlowSpec {
|
||||
class Fruit
|
||||
class Fruit extends Serializable
|
||||
class Apple extends Fruit
|
||||
class Orange extends Fruit
|
||||
val fruits = () =>
|
||||
|
|
@ -33,6 +34,7 @@ object FlowSpec {
|
|||
|
||||
}
|
||||
|
||||
@silent // tests type assignments compile
|
||||
class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
||||
import FlowSpec._
|
||||
|
||||
|
|
@ -67,13 +69,13 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
|
|||
new ChainSetup(identity, settings, toPublisher) {
|
||||
upstream.expectRequest(upstreamSubscription, settings.maxInputBufferSize)
|
||||
downstreamSubscription.request(1)
|
||||
upstream.expectNoMsg(100.millis)
|
||||
upstream.expectNoMessage(100.millis)
|
||||
downstreamSubscription.request(2)
|
||||
upstream.expectNoMsg(100.millis)
|
||||
upstream.expectNoMessage(100.millis)
|
||||
upstreamSubscription.sendNext("a")
|
||||
downstream.expectNext("a")
|
||||
upstream.expectRequest(upstreamSubscription, 1)
|
||||
upstream.expectNoMsg(100.millis)
|
||||
upstream.expectNoMessage(100.millis)
|
||||
upstreamSubscription.sendNext("b")
|
||||
upstreamSubscription.sendNext("c")
|
||||
upstreamSubscription.sendNext("d")
|
||||
|
|
@ -150,7 +152,7 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
|
|||
flowOut.subscribe(c1)
|
||||
val sub1 = c1.expectSubscription()
|
||||
sub1.request(3)
|
||||
c1.expectNoMsg(200.millis)
|
||||
c1.expectNoMessage(200.millis)
|
||||
|
||||
val source: Publisher[Int] = Source(List(1, 2, 3)).runWith(Sink.asPublisher(false))
|
||||
source.subscribe(flowIn)
|
||||
|
|
@ -169,7 +171,7 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
|
|||
flowOut.subscribe(c1)
|
||||
val sub1 = c1.expectSubscription()
|
||||
sub1.request(3)
|
||||
c1.expectNoMsg(200.millis)
|
||||
c1.expectNoMessage(200.millis)
|
||||
|
||||
val source: Publisher[Int] = Source(List(1, 2, 3)).runWith(Sink.asPublisher(false))
|
||||
source.subscribe(flowIn)
|
||||
|
|
@ -321,7 +323,7 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
|
|||
upstream.expectRequest(upstreamSubscription, 1)
|
||||
upstreamSubscription.sendNext("element2")
|
||||
|
||||
downstream.expectNoMsg(1.second)
|
||||
downstream.expectNoMessage(1.second)
|
||||
downstream2Subscription.request(1)
|
||||
downstream2.expectNext("firstElement")
|
||||
|
||||
|
|
@ -349,13 +351,13 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
|
|||
upstreamSubscription.expectRequest(1)
|
||||
upstreamSubscription.sendNext("element3")
|
||||
// downstream2 has not requested anything, fan-out buffer 2
|
||||
downstream.expectNoMsg(100.millis.dilated)
|
||||
downstream.expectNoMessage(100.millis.dilated)
|
||||
|
||||
downstream2Subscription.request(2)
|
||||
downstream.expectNext("element3")
|
||||
downstream2.expectNext("element1")
|
||||
downstream2.expectNext("element2")
|
||||
downstream2.expectNoMsg(100.millis.dilated)
|
||||
downstream2.expectNoMessage(100.millis.dilated)
|
||||
|
||||
upstreamSubscription.expectRequest(1)
|
||||
upstreamSubscription.sendNext("element4")
|
||||
|
|
@ -395,7 +397,7 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
|
|||
|
||||
upstreamSubscription.sendNext("a3")
|
||||
downstream.expectNext("a3")
|
||||
downstream2.expectNoMsg(100.millis.dilated) // as nothing was requested yet, fanOutBox needs to cache element in this case
|
||||
downstream2.expectNoMessage(100.millis.dilated) // as nothing was requested yet, fanOutBox needs to cache element in this case
|
||||
|
||||
downstream2Subscription.request(1)
|
||||
downstream2.expectNext("a3")
|
||||
|
|
@ -429,9 +431,9 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
|
|||
upstreamSubscription.sendNext("element3")
|
||||
upstreamSubscription.expectRequest(1)
|
||||
|
||||
downstream.expectNoMsg(200.millis.dilated)
|
||||
downstream2.expectNoMsg(200.millis.dilated)
|
||||
upstream.expectNoMsg(200.millis.dilated)
|
||||
downstream.expectNoMessage(200.millis.dilated)
|
||||
downstream2.expectNoMessage(200.millis.dilated)
|
||||
upstream.expectNoMessage(200.millis.dilated)
|
||||
|
||||
// should unblock fanoutbox
|
||||
downstream2Subscription.cancel()
|
||||
|
|
@ -469,7 +471,7 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
|
|||
downstream.expectNext("a3")
|
||||
downstream.expectComplete()
|
||||
|
||||
downstream2.expectNoMsg(100.millis.dilated) // as nothing was requested yet, fanOutBox needs to cache element in this case
|
||||
downstream2.expectNoMessage(100.millis.dilated) // as nothing was requested yet, fanOutBox needs to cache element in this case
|
||||
|
||||
downstream2Subscription.request(1)
|
||||
downstream2.expectNext("a3")
|
||||
|
|
|
|||
|
|
@ -42,7 +42,7 @@ class FlowSplitAfterSpec extends StreamSpec {
|
|||
|
||||
def request(demand: Int): Unit = subscription.request(demand)
|
||||
def expectNext(elem: Int): Unit = probe.expectNext(elem)
|
||||
def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max)
|
||||
def expectNoMessage(max: FiniteDuration): Unit = probe.expectNoMessage(max)
|
||||
def expectComplete(): Unit = probe.expectComplete()
|
||||
def expectError(e: Throwable) = probe.expectError(e)
|
||||
def cancel(): Unit = subscription.cancel()
|
||||
|
|
@ -77,7 +77,7 @@ class FlowSplitAfterSpec extends StreamSpec {
|
|||
"work in the happy case" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(3, elementCount = 5) {
|
||||
val s1 = StreamPuppet(expectSubFlow().runWith(Sink.asPublisher(false)))
|
||||
masterSubscriber.expectNoMsg(100.millis)
|
||||
masterSubscriber.expectNoMessage(100.millis)
|
||||
|
||||
s1.request(2)
|
||||
s1.expectNext(1)
|
||||
|
|
@ -102,7 +102,7 @@ class FlowSplitAfterSpec extends StreamSpec {
|
|||
"work when first element is split-by" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(splitAfter = 1, elementCount = 3) {
|
||||
val s1 = StreamPuppet(expectSubFlow().runWith(Sink.asPublisher(false)))
|
||||
masterSubscriber.expectNoMsg(100.millis)
|
||||
masterSubscriber.expectNoMessage(100.millis)
|
||||
|
||||
s1.request(3)
|
||||
s1.expectNext(1)
|
||||
|
|
@ -272,7 +272,7 @@ class FlowSplitAfterSpec extends StreamSpec {
|
|||
"work when last element is split-by" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(splitAfter = 3, elementCount = 3) {
|
||||
val s1 = StreamPuppet(expectSubFlow().runWith(Sink.asPublisher(false)))
|
||||
masterSubscriber.expectNoMsg(100.millis)
|
||||
masterSubscriber.expectNoMessage(100.millis)
|
||||
|
||||
s1.request(3)
|
||||
s1.expectNext(1)
|
||||
|
|
|
|||
|
|
@ -34,7 +34,7 @@ class FlowSplitWhenSpec extends StreamSpec {
|
|||
|
||||
def request(demand: Int): Unit = subscription.request(demand)
|
||||
def expectNext(elem: Int): Unit = probe.expectNext(elem)
|
||||
def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max)
|
||||
def expectNoMessage(max: FiniteDuration): Unit = probe.expectNoMessage(max)
|
||||
def expectComplete(): Unit = probe.expectComplete()
|
||||
def expectError(e: Throwable) = probe.expectError(e)
|
||||
def cancel(): Unit = subscription.cancel()
|
||||
|
|
@ -69,7 +69,7 @@ class FlowSplitWhenSpec extends StreamSpec {
|
|||
"work in the happy case" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(elementCount = 4) {
|
||||
val s1 = StreamPuppet(getSubFlow().runWith(Sink.asPublisher(false)))
|
||||
masterSubscriber.expectNoMsg(100.millis)
|
||||
masterSubscriber.expectNoMessage(100.millis)
|
||||
|
||||
s1.request(2)
|
||||
s1.expectNext(1)
|
||||
|
|
@ -81,7 +81,7 @@ class FlowSplitWhenSpec extends StreamSpec {
|
|||
|
||||
s2.request(1)
|
||||
s2.expectNext(3)
|
||||
s2.expectNoMsg(100.millis)
|
||||
s2.expectNoMessage(100.millis)
|
||||
|
||||
s2.request(1)
|
||||
s2.expectNext(4)
|
||||
|
|
@ -160,7 +160,7 @@ class FlowSplitWhenSpec extends StreamSpec {
|
|||
substream.cancel()
|
||||
|
||||
masterStream.expectNext(NotUsed)
|
||||
masterStream.expectNoMsg(100.millis)
|
||||
masterStream.expectNoMessage(100.millis)
|
||||
masterStream.cancel()
|
||||
inputs.expectCancellation()
|
||||
|
||||
|
|
@ -171,7 +171,6 @@ class FlowSplitWhenSpec extends StreamSpec {
|
|||
|
||||
val inputs3 = TestPublisher.probe[Int]()
|
||||
|
||||
val substream3 = TestSubscriber.probe[Int]()
|
||||
val masterStream3 = TestSubscriber.probe[Source[Int, Any]]()
|
||||
|
||||
Source.fromPublisher(inputs3).splitWhen(_ == 2).lift.runWith(Sink.fromSubscriber(masterStream3))
|
||||
|
|
@ -189,15 +188,15 @@ class FlowSplitWhenSpec extends StreamSpec {
|
|||
src2.runWith(Sink.fromSubscriber(substream4))
|
||||
|
||||
substream4.requestNext(2)
|
||||
substream4.expectNoMsg(100.millis)
|
||||
masterStream3.expectNoMsg(100.millis)
|
||||
substream4.expectNoMessage(100.millis)
|
||||
masterStream3.expectNoMessage(100.millis)
|
||||
inputs3.expectRequest()
|
||||
inputs3.expectRequest()
|
||||
inputs3.expectNoMsg(100.millis)
|
||||
inputs3.expectNoMessage(100.millis)
|
||||
|
||||
substream4.cancel()
|
||||
inputs3.expectNoMsg(100.millis)
|
||||
masterStream3.expectNoMsg(100.millis)
|
||||
inputs3.expectNoMessage(100.millis)
|
||||
masterStream3.expectNoMessage(100.millis)
|
||||
|
||||
masterStream3.cancel()
|
||||
inputs3.expectCancellation()
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ class FlowTakeWhileSpec extends StreamSpec {
|
|||
"continue if error" in assertAllStagesStopped {
|
||||
val testException = new Exception("test") with NoStackTrace
|
||||
|
||||
val p = Source(1 to 4)
|
||||
Source(1 to 4)
|
||||
.takeWhile(a => if (a == 3) throw testException else true)
|
||||
.withAttributes(supervisionStrategy(resumingDecider))
|
||||
.runWith(TestSink.probe[Int])
|
||||
|
|
|
|||
|
|
@ -42,20 +42,20 @@ class FlowTakeWithinSpec extends StreamSpec {
|
|||
(1 to demand3).foreach { _ =>
|
||||
pSub.sendNext(input.next())
|
||||
}
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
"deliver buffered elements onComplete before the timeout" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
Source(1 to 3).takeWithin(1.second).to(Sink.fromSubscriber(c)).run()
|
||||
val cSub = c.expectSubscription()
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
cSub.request(100)
|
||||
(1 to 3).foreach { n =>
|
||||
c.expectNext(n)
|
||||
}
|
||||
c.expectComplete()
|
||||
c.expectNoMsg(200.millis)
|
||||
c.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -53,7 +53,7 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
.runWith(TestSink.probe[Int])
|
||||
.request(5)
|
||||
.expectNext(1)
|
||||
.expectNoMsg(100.millis)
|
||||
.expectNoMessage(100.millis)
|
||||
.cancel() // We won't wait 100 days, sorry
|
||||
}
|
||||
|
||||
|
|
@ -76,11 +76,11 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
|
||||
downstream.request(20)
|
||||
upstream.sendNext(1)
|
||||
downstream.expectNoMsg(150.millis)
|
||||
downstream.expectNoMessage(150.millis)
|
||||
downstream.expectNext(1)
|
||||
|
||||
upstream.sendNext(2)
|
||||
downstream.expectNoMsg(150.millis)
|
||||
downstream.expectNoMessage(150.millis)
|
||||
downstream.expectNext(2)
|
||||
|
||||
upstream.sendComplete()
|
||||
|
|
@ -96,7 +96,7 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
upstream.sendNext(1)
|
||||
downstream.expectNext(1)
|
||||
|
||||
downstream.expectNoMsg(300.millis)
|
||||
downstream.expectNoMessage(300.millis)
|
||||
upstream.sendNext(2)
|
||||
downstream.expectNext(2)
|
||||
|
||||
|
|
@ -112,7 +112,7 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
"send elements downstream as soon as time comes" in assertAllStagesStopped {
|
||||
val probe = Source(1 to 10).throttle(2, 750.millis, 0, Shaping).runWith(TestSink.probe[Int]).request(5)
|
||||
probe.receiveWithin(900.millis) should be(Seq(1, 2))
|
||||
probe.expectNoMsg(150.millis).expectNext(3).expectNoMsg(150.millis).expectNext(4).cancel()
|
||||
probe.expectNoMessage(150.millis).expectNext(3).expectNoMessage(150.millis).expectNext(4).cancel()
|
||||
}
|
||||
|
||||
"burst according to its maximum if enough time passed" in assertAllStagesStopped {
|
||||
|
|
@ -126,7 +126,7 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
downstream.receiveWithin(300.millis, 5) should be(1 to 5)
|
||||
|
||||
downstream.request(5)
|
||||
downstream.expectNoMsg(1200.millis)
|
||||
downstream.expectNoMessage(1200.millis)
|
||||
for (i <- 7 to 11) upstream.sendNext(i)
|
||||
downstream.receiveWithin(300.millis, 5) should be(7 to 11)
|
||||
downstream.cancel()
|
||||
|
|
@ -144,9 +144,9 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
|
||||
downstream.request(1)
|
||||
upstream.sendNext(6)
|
||||
downstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
downstream.expectNext(6)
|
||||
downstream.expectNoMsg(500.millis) //wait to receive 2 in burst afterwards
|
||||
downstream.expectNoMessage(500.millis) //wait to receive 2 in burst afterwards
|
||||
downstream.request(5)
|
||||
for (i <- 7 to 10) upstream.sendNext(i)
|
||||
downstream.receiveWithin(100.millis, 2) should be(Seq(7, 8))
|
||||
|
|
@ -189,11 +189,11 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
.runWith(TestSink.probe[ByteString])
|
||||
.request(4)
|
||||
.expectNext(list(0))
|
||||
.expectNoMsg(300.millis)
|
||||
.expectNoMessage(300.millis)
|
||||
.expectNext(list(1))
|
||||
.expectNoMsg(500.millis)
|
||||
.expectNoMessage(500.millis)
|
||||
.expectNext(list(2))
|
||||
.expectNoMsg(700.millis)
|
||||
.expectNoMessage(700.millis)
|
||||
.expectNext(list(3))
|
||||
.expectComplete()
|
||||
}
|
||||
|
|
@ -210,7 +210,7 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
upstream.sendNext(1)
|
||||
downstream.expectNext(1)
|
||||
|
||||
downstream.expectNoMsg(300.millis)
|
||||
downstream.expectNoMessage(300.millis)
|
||||
upstream.sendNext(2)
|
||||
downstream.expectNext(2)
|
||||
|
||||
|
|
@ -226,7 +226,7 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
"send elements downstream as soon as time comes" in assertAllStagesStopped {
|
||||
val probe = Source(1 to 10).throttle(4, 500.millis, 0, _ => 2, Shaping).runWith(TestSink.probe[Int]).request(5)
|
||||
probe.receiveWithin(600.millis) should be(Seq(1, 2))
|
||||
probe.expectNoMsg(100.millis).expectNext(3).expectNoMsg(100.millis).expectNext(4).cancel()
|
||||
probe.expectNoMessage(100.millis).expectNext(3).expectNoMessage(100.millis).expectNext(4).cancel()
|
||||
}
|
||||
|
||||
"burst according to its maximum if enough time passed" in assertAllStagesStopped {
|
||||
|
|
@ -244,10 +244,10 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
|
||||
downstream.request(1)
|
||||
upstream.sendNext(6)
|
||||
downstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
downstream.expectNext(6)
|
||||
downstream.request(5)
|
||||
downstream.expectNoMsg(1200.millis)
|
||||
downstream.expectNoMessage(1200.millis)
|
||||
for (i <- 7 to 11) upstream.sendNext(i)
|
||||
downstream.receiveWithin(300.millis, 5) should be(7 to 11)
|
||||
downstream.cancel()
|
||||
|
|
@ -268,9 +268,9 @@ class FlowThrottleSpec extends StreamSpec {
|
|||
|
||||
downstream.request(1)
|
||||
upstream.sendNext(6)
|
||||
downstream.expectNoMsg(100.millis)
|
||||
downstream.expectNoMessage(100.millis)
|
||||
downstream.expectNext(6)
|
||||
downstream.expectNoMsg(500.millis) //wait to receive 2 in burst afterwards
|
||||
downstream.expectNoMessage(500.millis) //wait to receive 2 in burst afterwards
|
||||
downstream.request(5)
|
||||
for (i <- 7 to 9) upstream.sendNext(i)
|
||||
downstream.receiveWithin(200.millis, 2) should be(Seq(7, 8))
|
||||
|
|
|
|||
|
|
@ -31,8 +31,6 @@ class FlowWatchSpec extends StreamSpec {
|
|||
|
||||
"A Flow with watch" must {
|
||||
|
||||
implicit val timeout = akka.util.Timeout(10.seconds)
|
||||
|
||||
val replyOnInts =
|
||||
system.actorOf(Props(classOf[Replier]).withDispatcher("akka.test.stream-dispatcher"), "replyOnInts")
|
||||
|
||||
|
|
@ -40,8 +38,7 @@ class FlowWatchSpec extends StreamSpec {
|
|||
|
||||
"pass through elements while actor is alive" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
implicit val ec = system.dispatcher
|
||||
val p = Source(1 to 3).watch(replyOnInts).runWith(Sink.fromSubscriber(c))
|
||||
Source(1 to 3).watch(replyOnInts).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(1)
|
||||
|
|
|
|||
|
|
@ -73,7 +73,7 @@ class FlowWatchTerminationSpec extends StreamSpec {
|
|||
"fail future when stream abruptly terminated" in {
|
||||
val mat = ActorMaterializer()
|
||||
|
||||
val (p, future) = TestSource.probe[Int].watchTermination()(Keep.both).to(Sink.ignore).run()(mat)
|
||||
val (_, future) = TestSource.probe[Int].watchTermination()(Keep.both).to(Sink.ignore).run()(mat)
|
||||
mat.shutdown()
|
||||
|
||||
future.failed.futureValue shouldBe an[AbruptTerminationException]
|
||||
|
|
|
|||
|
|
@ -4,18 +4,20 @@
|
|||
|
||||
package akka.stream.scaladsl
|
||||
|
||||
//#zip
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Sink
|
||||
|
||||
//#zip
|
||||
|
||||
import akka.stream.testkit.scaladsl.StreamTestKit._
|
||||
import akka.stream.testkit.{ BaseTwoStreamsSetup, TestSubscriber }
|
||||
import com.github.ghik.silencer.silent
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
@silent // keep unused imports
|
||||
class FlowZipSpec extends BaseTwoStreamsSetup {
|
||||
|
||||
//#zip
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Sink
|
||||
|
||||
//#zip
|
||||
|
||||
override type Outputs = (Int, Int)
|
||||
|
||||
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
|
||||
|
|
|
|||
|
|
@ -4,17 +4,19 @@
|
|||
|
||||
package akka.stream.scaladsl
|
||||
|
||||
//#zip-with-index
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Sink
|
||||
|
||||
//#zip-with-index
|
||||
import akka.stream.testkit.scaladsl.StreamTestKit._
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.stream.testkit.{ StreamSpec, TestSubscriber }
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent // keep unused imports
|
||||
class FlowZipWithIndexSpec extends StreamSpec {
|
||||
|
||||
//#zip-with-index
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Sink
|
||||
|
||||
//#zip-with-index
|
||||
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16)
|
||||
|
||||
implicit val materializer = ActorMaterializer(settings)
|
||||
|
|
|
|||
|
|
@ -4,19 +4,22 @@
|
|||
|
||||
package akka.stream.scaladsl
|
||||
|
||||
//#zip-with
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Sink
|
||||
|
||||
//#zip-with
|
||||
|
||||
import akka.stream.testkit.{ BaseTwoStreamsSetup, TestSubscriber }
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.testkit.EventFilter
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent // keep unused imports
|
||||
class FlowZipWithSpec extends BaseTwoStreamsSetup {
|
||||
|
||||
//#zip-with
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Sink
|
||||
|
||||
//#zip-with
|
||||
|
||||
override type Outputs = Int
|
||||
|
||||
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
|
||||
|
|
@ -60,7 +63,7 @@ class FlowZipWithSpec extends BaseTwoStreamsSetup {
|
|||
probe.expectError() match {
|
||||
case a: java.lang.ArithmeticException => a.getMessage should be("/ by zero")
|
||||
}
|
||||
probe.expectNoMsg(200.millis)
|
||||
probe.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
commonTests()
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import akka.stream._
|
|||
import akka.stream.scaladsl.Framing.FramingException
|
||||
import akka.stream.stage.{ GraphStage, _ }
|
||||
import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber }
|
||||
import akka.util.{ ByteString, ByteStringBuilder }
|
||||
import akka.util.{ unused, ByteString, ByteStringBuilder }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Future
|
||||
|
|
@ -172,18 +172,11 @@ class FramingSpec extends StreamSpec {
|
|||
val fieldOffsets = List(0, 1, 2, 3, 15, 16, 31, 32, 44, 107)
|
||||
|
||||
def encode(payload: ByteString, fieldOffset: Int, fieldLength: Int, byteOrder: ByteOrder): ByteString = {
|
||||
encodeComplexFrame(
|
||||
payload,
|
||||
fieldOffset,
|
||||
fieldLength,
|
||||
byteOrder,
|
||||
ByteString(new Array[Byte](fieldOffset)),
|
||||
ByteString.empty)
|
||||
encodeComplexFrame(payload, fieldLength, byteOrder, ByteString(new Array[Byte](fieldOffset)), ByteString.empty)
|
||||
}
|
||||
|
||||
def encodeComplexFrame(
|
||||
payload: ByteString,
|
||||
fieldOffset: Int,
|
||||
fieldLength: Int,
|
||||
byteOrder: ByteOrder,
|
||||
offset: ByteString,
|
||||
|
|
@ -202,21 +195,21 @@ class FramingSpec extends StreamSpec {
|
|||
import system.dispatcher
|
||||
val resultFutures = for {
|
||||
byteOrder <- byteOrders
|
||||
fieldOffset <- fieldOffsets
|
||||
lengthFieldOffset <- fieldOffsets
|
||||
fieldLength <- fieldLengths
|
||||
} yield {
|
||||
|
||||
val encodedFrames = frameLengths.filter(_ < (1L << (fieldLength * 8))).map { length =>
|
||||
val payload = referenceChunk.take(length)
|
||||
encode(payload, fieldOffset, fieldLength, byteOrder)
|
||||
encode(payload, lengthFieldOffset, fieldLength, byteOrder)
|
||||
}
|
||||
|
||||
Source(encodedFrames)
|
||||
.via(rechunk)
|
||||
.via(Framing.lengthField(fieldLength, fieldOffset, Int.MaxValue, byteOrder))
|
||||
.via(Framing.lengthField(fieldLength, lengthFieldOffset, Int.MaxValue, byteOrder))
|
||||
.grouped(10000)
|
||||
.runWith(Sink.head)
|
||||
.map(result => (result, encodedFrames, (byteOrder, fieldOffset, fieldLength)))
|
||||
.map(result => (result, encodedFrames, (byteOrder, lengthFieldOffset, fieldLength)))
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -253,13 +246,7 @@ class FramingSpec extends StreamSpec {
|
|||
val payload = referenceChunk.take(length)
|
||||
val offsetBytes = offset()
|
||||
val tailBytes = if (offsetBytes.length > 0) new Array[Byte](offsetBytes(0)) else Array.empty[Byte]
|
||||
encodeComplexFrame(
|
||||
payload,
|
||||
fieldOffset,
|
||||
fieldLength,
|
||||
byteOrder,
|
||||
ByteString(offsetBytes),
|
||||
ByteString(tailBytes))
|
||||
encodeComplexFrame(payload, fieldLength, byteOrder, ByteString(offsetBytes), ByteString(tailBytes))
|
||||
}
|
||||
|
||||
Source(encodedFrames)
|
||||
|
|
@ -394,7 +381,7 @@ class FramingSpec extends StreamSpec {
|
|||
"fail the stage on computeFrameSize values less than minimum chunk size" in {
|
||||
implicit val bo = java.nio.ByteOrder.LITTLE_ENDIAN
|
||||
|
||||
def computeFrameSize(arr: Array[Byte], l: Int): Int = 3
|
||||
def computeFrameSize(@unused arr: Array[Byte], @unused l: Int): Int = 3
|
||||
|
||||
// A 4-byte message containing only an Int specifying the length of the payload
|
||||
val bs = ByteString.newBuilder.putInt(4).result()
|
||||
|
|
|
|||
|
|
@ -37,12 +37,12 @@ class FutureFlattenSourceSpec extends StreamSpec {
|
|||
"emit no elements before the future of source successful" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
val sourcePromise = Promise[Source[Int, String]]()
|
||||
val p = Source.fromFutureSource(sourcePromise.future).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
Source.fromFutureSource(sourcePromise.future).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
import scala.concurrent.duration._
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
sub.request(3)
|
||||
c.expectNoMsg(100.millis)
|
||||
c.expectNoMessage(100.millis)
|
||||
sourcePromise.success(underlying)
|
||||
c.expectNext(1)
|
||||
c.expectNext(2)
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import akka.stream.ActorMaterializer
|
|||
import akka.stream.ActorMaterializerSettings
|
||||
import akka.stream.testkit._
|
||||
import akka.stream._
|
||||
import com.github.ghik.silencer.silent
|
||||
import org.reactivestreams.Subscriber
|
||||
|
||||
object GraphFlowSpec {
|
||||
|
|
@ -36,6 +37,7 @@ object GraphFlowSpec {
|
|||
val stdResult = Set(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)
|
||||
}
|
||||
|
||||
@silent
|
||||
class GraphFlowSpec extends StreamSpec {
|
||||
|
||||
import GraphFlowSpec._
|
||||
|
|
@ -290,7 +292,7 @@ class GraphFlowSpec extends StreamSpec {
|
|||
SinkShape(flow.in)
|
||||
})
|
||||
|
||||
val (m1, m2, m3) = RunnableGraph
|
||||
val (m1, _, m3) = RunnableGraph
|
||||
.fromGraph(GraphDSL.create(source, flow, sink)(Tuple3.apply) { implicit b => (src, f, snk) =>
|
||||
import GraphDSL.Implicits._
|
||||
src.out.map(_.toInt) ~> f.in
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ class GraphBalanceSpec extends StreamSpec {
|
|||
|
||||
sub1.request(1)
|
||||
c1.expectNext(1)
|
||||
c1.expectNoMsg(100.millis)
|
||||
c1.expectNoMessage(100.millis)
|
||||
|
||||
sub2.request(2)
|
||||
c2.expectNext(2)
|
||||
|
|
@ -63,14 +63,14 @@ class GraphBalanceSpec extends StreamSpec {
|
|||
|
||||
val sub1 = s1.expectSubscription()
|
||||
sub1.request(1)
|
||||
s1.expectNoMsg(200.millis)
|
||||
s1.expectNoMessage(200.millis)
|
||||
|
||||
val s2 = TestSubscriber.manualProbe[Int]()
|
||||
p2.subscribe(s2)
|
||||
val sub2 = s2.expectSubscription()
|
||||
|
||||
// still no demand from s2
|
||||
s1.expectNoMsg(200.millis)
|
||||
s1.expectNoMessage(200.millis)
|
||||
|
||||
sub2.request(2)
|
||||
s1.expectNext(1)
|
||||
|
|
@ -107,7 +107,7 @@ class GraphBalanceSpec extends StreamSpec {
|
|||
val sub3 = s3.expectSubscription()
|
||||
|
||||
sub2.request(2)
|
||||
s1.expectNoMsg(200.millis)
|
||||
s1.expectNoMessage(200.millis)
|
||||
sub3.cancel()
|
||||
|
||||
s1.expectNext(1)
|
||||
|
|
|
|||
|
|
@ -40,10 +40,10 @@ class GraphBroadcastSpec extends StreamSpec {
|
|||
sub1.request(1)
|
||||
sub2.request(2)
|
||||
c1.expectNext(1)
|
||||
c1.expectNoMsg(100.millis)
|
||||
c1.expectNoMessage(100.millis)
|
||||
c2.expectNext(1)
|
||||
c2.expectNext(2)
|
||||
c2.expectNoMsg(100.millis)
|
||||
c2.expectNoMessage(100.millis)
|
||||
sub1.request(3)
|
||||
c1.expectNext(2)
|
||||
c1.expectNext(3)
|
||||
|
|
|
|||
|
|
@ -8,12 +8,14 @@ import akka.stream.impl.fusing.GraphStages
|
|||
import akka.stream._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.stage._
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
object GraphDSLCompileSpec {
|
||||
class Fruit
|
||||
class Apple extends Fruit
|
||||
}
|
||||
|
||||
@silent // tests deprecated APIs
|
||||
class GraphDSLCompileSpec extends StreamSpec {
|
||||
import GraphDSLCompileSpec._
|
||||
|
||||
|
|
@ -236,7 +238,7 @@ class GraphDSLCompileSpec extends StreamSpec {
|
|||
"throw an error if some ports are not connected" in {
|
||||
intercept[IllegalStateException] {
|
||||
GraphDSL.create() { implicit b =>
|
||||
val s = b.add(Source.empty[Int])
|
||||
b.add(Source.empty[Int])
|
||||
val op = b.add(Flow[Int].map(_ + 1))
|
||||
val sink = b.add(Sink.foreach[Int](println))
|
||||
|
||||
|
|
|
|||
|
|
@ -180,7 +180,7 @@ class GraphMatValueSpec extends StreamSpec {
|
|||
ClosedShape
|
||||
})
|
||||
|
||||
val result = g.run()
|
||||
g.run()
|
||||
}
|
||||
|
||||
"ignore materialized values for a graph with no materialized values exposed, but keep side-effects" in {
|
||||
|
|
@ -199,7 +199,7 @@ class GraphMatValueSpec extends StreamSpec {
|
|||
ClosedShape
|
||||
})
|
||||
|
||||
var result = g.run()
|
||||
g.run()
|
||||
|
||||
expectMsg("side effect!")
|
||||
|
||||
|
|
|
|||
|
|
@ -133,7 +133,7 @@ class GraphMergeLatestSpec extends TwoStreamsSetup {
|
|||
val up2 = TestSource.probe[Int]
|
||||
val probe = TestSubscriber.manualProbe[List[Int]]()
|
||||
|
||||
val (in1, in2) = RunnableGraph
|
||||
val (in1, _) = RunnableGraph
|
||||
.fromGraph(GraphDSL.create(up1, up2)((_, _)) { implicit b => (s1, s2) =>
|
||||
val m = b.add(MergeLatest[Int](2, true))
|
||||
|
||||
|
|
@ -144,7 +144,7 @@ class GraphMergeLatestSpec extends TwoStreamsSetup {
|
|||
})
|
||||
.run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
probe.expectSubscription()
|
||||
|
||||
in1.sendComplete()
|
||||
probe.expectComplete()
|
||||
|
|
|
|||
|
|
@ -6,10 +6,12 @@ package akka.stream.scaladsl
|
|||
|
||||
import akka.stream.testkit.TwoStreamsSetup
|
||||
import akka.stream._
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
@silent // stream usage
|
||||
class GraphMergePreferredSpec extends TwoStreamsSetup {
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
|
|
@ -76,7 +78,7 @@ class GraphMergePreferredSpec extends TwoStreamsSetup {
|
|||
val s = Source(0 to 3)
|
||||
|
||||
(the[IllegalArgumentException] thrownBy {
|
||||
val g = RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
val merge = b.add(MergePreferred[Int](1))
|
||||
|
||||
s ~> merge.preferred
|
||||
|
|
@ -93,7 +95,7 @@ class GraphMergePreferredSpec extends TwoStreamsSetup {
|
|||
val s = Source(0 to 3)
|
||||
|
||||
(the[IllegalArgumentException] thrownBy {
|
||||
val g = RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
val merge = b.add(MergePreferred[Int](1))
|
||||
|
||||
s ~> merge.preferred
|
||||
|
|
|
|||
|
|
@ -6,10 +6,12 @@ package akka.stream.scaladsl
|
|||
|
||||
import akka.stream._
|
||||
import akka.stream.testkit.TwoStreamsSetup
|
||||
import com.github.ghik.silencer.silent
|
||||
import org.scalacheck.Gen
|
||||
import org.scalatest.prop.GeneratorDrivenPropertyChecks
|
||||
import org.scalacheck.Shrink
|
||||
|
||||
@silent // tests deprecated apis
|
||||
class GraphMergeSortedSpec extends TwoStreamsSetup with GeneratorDrivenPropertyChecks {
|
||||
|
||||
override type Outputs = Int
|
||||
|
|
|
|||
|
|
@ -104,7 +104,7 @@ class GraphPartialSpec extends StreamSpec {
|
|||
}
|
||||
|
||||
"be able to expose the ports of imported graphs" in {
|
||||
val p = GraphDSL.create(Flow[Int].map(_ + 1)) { implicit b => flow =>
|
||||
val p = GraphDSL.create(Flow[Int].map(_ + 1)) { _ => flow =>
|
||||
FlowShape(flow.in, flow.out)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -91,7 +91,7 @@ class GraphPartitionSpec extends StreamSpec {
|
|||
.run()
|
||||
|
||||
c1.request(1)
|
||||
c1.expectNoMsg(1.seconds)
|
||||
c1.expectNoMessage(1.seconds)
|
||||
c2.request(1)
|
||||
c2.expectNext(6)
|
||||
c1.expectNext(3)
|
||||
|
|
@ -197,7 +197,7 @@ class GraphPartitionSpec extends StreamSpec {
|
|||
.run()
|
||||
|
||||
c1.request(1)
|
||||
c1.expectNoMsg(1.second)
|
||||
c1.expectNoMessage(1.second)
|
||||
c2.request(1)
|
||||
c2.expectNext(6)
|
||||
c1.expectComplete()
|
||||
|
|
@ -302,7 +302,7 @@ class GraphPartitionSpec extends StreamSpec {
|
|||
val even = TestSubscriber.probe[Int]()
|
||||
Source(1 to 2).divertTo(Sink.fromSubscriber(odd), _ % 2 != 0).to(Sink.fromSubscriber(even)).run()
|
||||
even.request(1)
|
||||
even.expectNoMsg(1.second)
|
||||
even.expectNoMessage(1.second)
|
||||
odd.request(1)
|
||||
odd.expectNext(1)
|
||||
even.expectNext(2)
|
||||
|
|
|
|||
|
|
@ -101,7 +101,7 @@ class GraphStageTimersSpec extends StreamSpec {
|
|||
driver ! TestSingleTimer
|
||||
expectMsg(Tick(1))
|
||||
}
|
||||
expectNoMsg(1.second)
|
||||
expectNoMessage(1.second)
|
||||
}
|
||||
|
||||
driver.stopStage()
|
||||
|
|
@ -118,7 +118,7 @@ class GraphStageTimersSpec extends StreamSpec {
|
|||
within(1.second) {
|
||||
expectMsg(Tick(2))
|
||||
}
|
||||
expectNoMsg(1.second)
|
||||
expectNoMessage(1.second)
|
||||
}
|
||||
|
||||
driver.stopStage()
|
||||
|
|
@ -134,7 +134,7 @@ class GraphStageTimersSpec extends StreamSpec {
|
|||
within(300.millis, 1.second) {
|
||||
expectMsg(Tick(1))
|
||||
}
|
||||
expectNoMsg(1.second)
|
||||
expectNoMessage(1.second)
|
||||
|
||||
driver.stopStage()
|
||||
}
|
||||
|
|
@ -147,7 +147,7 @@ class GraphStageTimersSpec extends StreamSpec {
|
|||
case t: Tick => t
|
||||
}
|
||||
(seq should have).length(5)
|
||||
expectNoMsg(1.second)
|
||||
expectNoMessage(1.second)
|
||||
|
||||
driver.stopStage()
|
||||
}
|
||||
|
|
@ -188,7 +188,7 @@ class GraphStageTimersSpec extends StreamSpec {
|
|||
downstream.expectNext(2)
|
||||
downstream.expectNext(3)
|
||||
|
||||
downstream.expectNoMsg(1.second)
|
||||
downstream.expectNoMessage(1.second)
|
||||
|
||||
upstream.sendComplete()
|
||||
downstream.expectComplete()
|
||||
|
|
|
|||
|
|
@ -37,10 +37,10 @@ class GraphUnzipSpec extends StreamSpec {
|
|||
sub1.request(1)
|
||||
sub2.request(2)
|
||||
c1.expectNext(1 * 2)
|
||||
c1.expectNoMsg(100.millis)
|
||||
c1.expectNoMessage(100.millis)
|
||||
c2.expectNext("a")
|
||||
c2.expectNext("b")
|
||||
c2.expectNoMsg(100.millis)
|
||||
c2.expectNoMessage(100.millis)
|
||||
sub1.request(3)
|
||||
c1.expectNext(2 * 2)
|
||||
c1.expectNext(3 * 2)
|
||||
|
|
|
|||
|
|
@ -9,9 +9,11 @@ import akka.stream.testkit.TestSubscriber.Probe
|
|||
import akka.stream.testkit.scaladsl.StreamTestKit._
|
||||
import akka.stream.testkit._
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.testkit.EventFilter
|
||||
import akka.util.unused
|
||||
|
||||
class GraphUnzipWithSpec extends StreamSpec {
|
||||
|
||||
|
|
@ -26,13 +28,13 @@ class GraphUnzipWithSpec extends StreamSpec {
|
|||
type LeftOutput = Int
|
||||
type RightOutput = String
|
||||
|
||||
abstract class Fixture(b: GraphDSL.Builder[_]) {
|
||||
abstract class Fixture(@unused b: GraphDSL.Builder[_]) {
|
||||
def in: Inlet[Int]
|
||||
def left: Outlet[LeftOutput]
|
||||
def right: Outlet[RightOutput]
|
||||
}
|
||||
|
||||
val f: (Int => (Int, String)) = b => (b + b, b + "+" + b)
|
||||
val f: (Int => (Int, String)) = b => (b + b, s"$b + $b")
|
||||
|
||||
def fixture(b: GraphDSL.Builder[_]): Fixture = new Fixture(b) {
|
||||
val unzip = b.add(UnzipWith[Int, Int, String](f))
|
||||
|
|
@ -119,26 +121,26 @@ class GraphUnzipWithSpec extends StreamSpec {
|
|||
|
||||
leftProbe.expectNext(2)
|
||||
leftProbe.expectNext(4)
|
||||
leftProbe.expectNoMsg(100.millis)
|
||||
leftProbe.expectNoMessage(100.millis)
|
||||
|
||||
rightProbe.expectNext("1+1")
|
||||
rightProbe.expectNoMsg(100.millis)
|
||||
rightProbe.expectNext("1 + 1")
|
||||
rightProbe.expectNoMessage(100.millis)
|
||||
|
||||
leftSubscription.request(1)
|
||||
rightSubscription.request(2)
|
||||
|
||||
leftProbe.expectNext(6)
|
||||
leftProbe.expectNoMsg(100.millis)
|
||||
leftProbe.expectNoMessage(100.millis)
|
||||
|
||||
rightProbe.expectNext("2+2")
|
||||
rightProbe.expectNext("3+3")
|
||||
rightProbe.expectNoMsg(100.millis)
|
||||
rightProbe.expectNext("2 + 2")
|
||||
rightProbe.expectNext("3 + 3")
|
||||
rightProbe.expectNoMessage(100.millis)
|
||||
|
||||
leftSubscription.request(1)
|
||||
rightSubscription.request(1)
|
||||
|
||||
leftProbe.expectNext(8)
|
||||
rightProbe.expectNext("4+4")
|
||||
rightProbe.expectNext("4 + 4")
|
||||
|
||||
leftProbe.expectComplete()
|
||||
rightProbe.expectComplete()
|
||||
|
|
@ -146,50 +148,55 @@ class GraphUnzipWithSpec extends StreamSpec {
|
|||
|
||||
"work in the sad case" in {
|
||||
val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1)
|
||||
val mat = ActorMaterializer(settings)
|
||||
|
||||
val leftProbe = TestSubscriber.manualProbe[LeftOutput]()
|
||||
val rightProbe = TestSubscriber.manualProbe[RightOutput]()
|
||||
try {
|
||||
val leftProbe = TestSubscriber.manualProbe[LeftOutput]()
|
||||
val rightProbe = TestSubscriber.manualProbe[RightOutput]()
|
||||
|
||||
RunnableGraph
|
||||
.fromGraph(GraphDSL.create() { implicit b =>
|
||||
val unzip = b.add(UnzipWith[Int, Int, String]((b: Int) => (1 / b, 1 + "/" + b)))
|
||||
RunnableGraph
|
||||
.fromGraph(GraphDSL.create() { implicit b =>
|
||||
val unzip = b.add(UnzipWith[Int, Int, String]((b: Int) => (1 / b, s"1 / $b")))
|
||||
|
||||
Source(-2 to 2) ~> unzip.in
|
||||
Source(-2 to 2) ~> unzip.in
|
||||
|
||||
unzip.out0 ~> Sink.fromSubscriber(leftProbe)
|
||||
unzip.out1 ~> Sink.fromSubscriber(rightProbe)
|
||||
unzip.out0 ~> Sink.fromSubscriber(leftProbe)
|
||||
unzip.out1 ~> Sink.fromSubscriber(rightProbe)
|
||||
|
||||
ClosedShape
|
||||
})
|
||||
.run()
|
||||
ClosedShape
|
||||
})
|
||||
.run()(mat)
|
||||
|
||||
val leftSubscription = leftProbe.expectSubscription()
|
||||
val rightSubscription = rightProbe.expectSubscription()
|
||||
val leftSubscription = leftProbe.expectSubscription()
|
||||
val rightSubscription = rightProbe.expectSubscription()
|
||||
|
||||
def requestFromBoth(): Unit = {
|
||||
leftSubscription.request(1)
|
||||
rightSubscription.request(1)
|
||||
}
|
||||
def requestFromBoth(): Unit = {
|
||||
leftSubscription.request(1)
|
||||
rightSubscription.request(1)
|
||||
}
|
||||
|
||||
requestFromBoth()
|
||||
leftProbe.expectNext(1 / -2)
|
||||
rightProbe.expectNext("1/-2")
|
||||
|
||||
requestFromBoth()
|
||||
leftProbe.expectNext(1 / -1)
|
||||
rightProbe.expectNext("1/-1")
|
||||
|
||||
EventFilter[ArithmeticException](occurrences = 1).intercept {
|
||||
requestFromBoth()
|
||||
}
|
||||
leftProbe.expectNext(1 / -2)
|
||||
rightProbe.expectNext("1 / -2")
|
||||
|
||||
leftProbe.expectError() match {
|
||||
case a: java.lang.ArithmeticException => a.getMessage should be("/ by zero")
|
||||
}
|
||||
rightProbe.expectError()
|
||||
requestFromBoth()
|
||||
leftProbe.expectNext(1 / -1)
|
||||
rightProbe.expectNext("1 / -1")
|
||||
|
||||
leftProbe.expectNoMsg(100.millis)
|
||||
rightProbe.expectNoMsg(100.millis)
|
||||
EventFilter[ArithmeticException](occurrences = 1).intercept {
|
||||
requestFromBoth()
|
||||
}
|
||||
|
||||
leftProbe.expectError() match {
|
||||
case a: java.lang.ArithmeticException => a.getMessage should be("/ by zero")
|
||||
}
|
||||
rightProbe.expectError()
|
||||
|
||||
leftProbe.expectNoMessage(100.millis)
|
||||
rightProbe.expectNoMessage(100.millis)
|
||||
} finally {
|
||||
mat.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
"unzipWith expanded Person.unapply (3 outputs)" in {
|
||||
|
|
|
|||
|
|
@ -97,7 +97,7 @@ class GraphZipLatestWithSpec extends TwoStreamsSetup {
|
|||
case a: java.lang.ArithmeticException =>
|
||||
a.getMessage should be("/ by zero")
|
||||
}
|
||||
probe.expectNoMsg(200.millis)
|
||||
probe.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
commonTests()
|
||||
|
|
@ -133,7 +133,7 @@ class GraphZipLatestWithSpec extends TwoStreamsSetup {
|
|||
subscriber1.expectSubscriptionAndError(TestException)
|
||||
|
||||
val subscriber2 = setup(nonemptyPublisher(1 to 4), soonToFailPublisher)
|
||||
val subscription2 = subscriber2.expectSubscriptionAndError(TestException)
|
||||
subscriber2.expectSubscriptionAndError(TestException)
|
||||
}
|
||||
|
||||
"zipLatestWith a ETA expanded Person.apply (3 inputs)" in {
|
||||
|
|
@ -193,9 +193,7 @@ class GraphZipLatestWithSpec extends TwoStreamsSetup {
|
|||
v19: Int,
|
||||
v20: String,
|
||||
v21: Int,
|
||||
v22: String) =>
|
||||
v1 + v2 + v3 + v4 + v5 + v6 + v7 + v8 + v9 + v10 +
|
||||
v11 + v12 + v13 + v14 + v15 + v16 + v17 + v18 + v19 + v20 + v21 + v22
|
||||
v22: String) => s"$v1$v2$v3$v4$v5$v6$v7$v8$v9$v10$v11$v12$v13$v14$v15$v16$v17$v18$v19$v20$v21$v22"
|
||||
|
||||
// odd input ports will be Int, even input ports will be String
|
||||
val zip = b.add(ZipLatestWith(sum22))
|
||||
|
|
|
|||
|
|
@ -187,7 +187,7 @@ class GraphZipNSpec extends TwoStreamsSetup {
|
|||
|
||||
upstream1.sendNext(1)
|
||||
upstream1.sendComplete()
|
||||
downstream.expectNoMsg(500.millis)
|
||||
downstream.expectNoMessage(500.millis)
|
||||
|
||||
upstream2.sendNext(2)
|
||||
upstream2.sendComplete()
|
||||
|
|
@ -226,7 +226,7 @@ class GraphZipNSpec extends TwoStreamsSetup {
|
|||
subscriber1.expectSubscriptionAndError(TestException)
|
||||
|
||||
val subscriber2 = setup(nonemptyPublisher(1 to 4), soonToFailPublisher)
|
||||
val subscription2 = subscriber2.expectSubscriptionAndError(TestException)
|
||||
subscriber2.expectSubscriptionAndError(TestException)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -186,7 +186,7 @@ class GraphZipSpec extends TwoStreamsSetup {
|
|||
|
||||
upstream1.sendNext(1)
|
||||
upstream1.sendComplete()
|
||||
downstream.expectNoMsg(500.millis)
|
||||
downstream.expectNoMessage(500.millis)
|
||||
|
||||
upstream2.sendNext("A")
|
||||
upstream2.sendComplete()
|
||||
|
|
@ -225,7 +225,7 @@ class GraphZipSpec extends TwoStreamsSetup {
|
|||
subscriber1.expectSubscriptionAndError(TestException)
|
||||
|
||||
val subscriber2 = setup(nonemptyPublisher(1 to 4), soonToFailPublisher)
|
||||
val subscription2 = subscriber2.expectSubscriptionAndError(TestException)
|
||||
subscriber2.expectSubscriptionAndError(TestException)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -81,7 +81,7 @@ class GraphZipWithNSpec extends TwoStreamsSetup {
|
|||
probe.expectError() match {
|
||||
case a: java.lang.ArithmeticException => a.getMessage should be("/ by zero")
|
||||
}
|
||||
probe.expectNoMsg(200.millis)
|
||||
probe.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
commonTests()
|
||||
|
|
@ -115,7 +115,7 @@ class GraphZipWithNSpec extends TwoStreamsSetup {
|
|||
subscriber1.expectSubscriptionAndError(TestException)
|
||||
|
||||
val subscriber2 = setup(nonemptyPublisher(1 to 4), soonToFailPublisher)
|
||||
val subscription2 = subscriber2.expectSubscriptionAndError(TestException)
|
||||
subscriber2.expectSubscriptionAndError(TestException)
|
||||
}
|
||||
|
||||
"work with 3 inputs" in {
|
||||
|
|
|
|||
|
|
@ -80,7 +80,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
probe.expectError() match {
|
||||
case a: java.lang.ArithmeticException => a.getMessage should be("/ by zero")
|
||||
}
|
||||
probe.expectNoMsg(200.millis)
|
||||
probe.expectNoMessage(200.millis)
|
||||
}
|
||||
|
||||
commonTests()
|
||||
|
|
@ -114,7 +114,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
subscriber1.expectSubscriptionAndError(TestException)
|
||||
|
||||
val subscriber2 = setup(nonemptyPublisher(1 to 4), soonToFailPublisher)
|
||||
val subscription2 = subscriber2.expectSubscriptionAndError(TestException)
|
||||
subscriber2.expectSubscriptionAndError(TestException)
|
||||
}
|
||||
|
||||
"zipWith a ETA expanded Person.apply (3 inputs)" in {
|
||||
|
|
@ -171,10 +171,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
v19: Int,
|
||||
v20: String,
|
||||
v21: Int,
|
||||
v22: String) =>
|
||||
v1 + v2 + v3 + v4 + v5 + v6 + v7 + v8 + v9 + v10 +
|
||||
v11 + v12 + v13 + v14 + v15 + v16 + v17 + v18 +
|
||||
v19 + v20 + v21 + v22
|
||||
v22: String) => s"$v1$v2$v3$v4$v5$v6$v7$v8$v9$v10$v11$v12$v13$v14$v15$v16$v17$v18$v19$v20$v21$v22"
|
||||
|
||||
// odd input ports will be Int, even input ports will be String
|
||||
val zip = b.add(ZipWith(sum22))
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue