Remove warnings from akka-stream-tests (#27114)

This commit is contained in:
Christopher Batey 2019-06-13 12:06:02 +01:00 committed by Patrik Nordwall
parent 03588c3036
commit 231f0d6bb1
122 changed files with 546 additions and 551 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -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[_]]
}
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -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] =

View file

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

View file

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

View file

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

View file

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

View file

@ -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._

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -36,7 +36,7 @@ class FlowDetacherSpec extends StreamSpec {
.detach
.runWith(TestSink.probe)
.ensureSubscription()
.expectNoMsg(500.millis)
.expectNoMessage(500.millis)
.requestNext() should ===(42)
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -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]) = {

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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