diff --git a/akka-actor/src/main/scala-2.13-/akka/util/ccompat/package.scala b/akka-actor/src/main/scala-2.13-/akka/util/ccompat/package.scala index be055fa19b..c1c614ef1d 100644 --- a/akka-actor/src/main/scala-2.13-/akka/util/ccompat/package.scala +++ b/akka-actor/src/main/scala-2.13-/akka/util/ccompat/package.scala @@ -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 } diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/SourceTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/SourceTest.java index ff8ce69619..a9435e8f6e 100644 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/SourceTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/SourceTest.java @@ -326,7 +326,7 @@ public class SourceTest extends StreamTest { List 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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala index 56e3e94251..b9d3ccf945 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala @@ -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 { diff --git a/akka-stream-tests/src/test/scala/akka/stream/DslFactoriesConsistencySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/DslFactoriesConsistencySpec.scala index 09560cf2d6..b116e62290 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/DslFactoriesConsistencySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/DslFactoriesConsistencySpec.scala @@ -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 } /** diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/FanoutProcessorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/FanoutProcessorSpec.scala index 6ba72c1692..53f23facaf 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/FanoutProcessorSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/FanoutProcessorSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala index ec1154bb5c..6b75bc7c17 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala @@ -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[_]] } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/GraphStageLogicSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/GraphStageLogicSpec.scala index 586de5a319..f617871138 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/GraphStageLogicSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/GraphStageLogicSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/ResizableMultiReaderRingBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/ResizableMultiReaderRingBufferSpec.scala index c4f748e76e..5f19288e2a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/ResizableMultiReaderRingBufferSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/ResizableMultiReaderRingBufferSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala index aa617791f3..602d55ca82 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalBuilderSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalBuilderSpec.scala index 2e956dedec..d2b30d515d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalBuilderSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalBuilderSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala index f32684db0c..739c46f68a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala @@ -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")) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala index d5d2e3f9da..96e29ae139 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala index 006de4ae45..9c26724785 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala @@ -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) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpec.scala index f4cab0ecc4..9e03fc6b65 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/InterpreterSpec.scala @@ -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 = diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/KeepGoingStageSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/KeepGoingStageSpec.scala index 18e7573a3f..e1dfaa0c57 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/KeepGoingStageSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/KeepGoingStageSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/LifecycleInterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/LifecycleInterpreterSpec.scala index ac27519ce3..03094774a4 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/LifecycleInterpreterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/LifecycleInterpreterSpec.scala @@ -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 = diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/ByteStringParserSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/ByteStringParserSpec.scala index 4381cbb4c3..371fcdd55f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/ByteStringParserSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/ByteStringParserSpec.scala @@ -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] diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala index b6ea723e62..8025321b08 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala index b1ee9a0632..42fec4feee 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala index 177352b3c4..f1ffdc953a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSourceSpec.scala index e4d203ce80..9ebaa841f0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSourceSpec.scala @@ -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 { diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala index eb8960a6c0..776444b8ee 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala index 45b455e0b6..b17bc551c3 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala @@ -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 } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala index 2321a0feb5..f406f81860 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala index c4dd401d5b..8635881b52 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala @@ -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) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CoderSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CoderSpec.scala index 33c8b4659c..7b38e423cd 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CoderSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CoderSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CompressionTestingTools.scala b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CompressionTestingTools.scala index e5b4fe2c9b..a4588e02f2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/compression/CompressionTestingTools.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/compression/CompressionTestingTools.scala @@ -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] = diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala index 2dbf9182bf..cc62888697 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSourceSpec.scala index 62304b4b0d..f70fb9ec1f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSourceSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala index 226e48efa6..62e9ed82d8 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala @@ -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 } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala index e9cad8272d..d3c1cf39ed 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala index 2ed5deea20..d8675b103b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala @@ -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._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala index 3e5c511088..f9184c55a5 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala @@ -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 } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAskSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAskSpec.scala index 8adab4d634..47199985d9 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAskSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowAskSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala index 9009973a50..1cc7fe2a4b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala @@ -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() } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala index 663b7fc7ba..264f831977 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala index 8015d60706..e3f173100d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowCompileSpec.scala index b42b963267..a78be9ff4f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowCompileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowCompileSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala index 5e0f70ef1c..6099e0474d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala @@ -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) { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDelaySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDelaySpec.scala index 0b1e16c2f2..a22231ac12 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDelaySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDelaySpec.scala @@ -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) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDetacherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDetacherSpec.scala index ed3777e110..72d6957ac7 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDetacherSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDetacherSpec.scala @@ -36,7 +36,7 @@ class FlowDetacherSpec extends StreamSpec { .detach .runWith(TestSink.probe) .ensureSubscription() - .expectNoMsg(500.millis) + .expectNoMessage(500.millis) .requestNext() should ===(42) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala index 4c40d22127..0d0419c859 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala @@ -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) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWithinSpec.scala index 5d7664099d..c67d5c21cc 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWithinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropWithinSpec.scala @@ -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 { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala index c39ae7c43c..dce965a9f2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala @@ -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() } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala index 18fef88f98..641b75ec2f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala @@ -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() } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala index bfdb122e19..32505693fe 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala @@ -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) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldAsyncSpec.scala index acd3056a71..8d6edf30b6 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldAsyncSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFoldAsyncSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFromFutureSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFromFutureSpec.scala index fb64ad6e58..28201dc72b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFromFutureSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFromFutureSpec.scala @@ -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) } } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala index 4e8c0979d3..826b390c17 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedWithinSpec.scala index d88ad35514..19e43e8f3c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedWithinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedWithinSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala index eff38b2c92..55ae51d9d9 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala @@ -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) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala index d8214f33c2..16295a828e 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala index d0a60c0837..c8678d85bb 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala @@ -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) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala index 1aa0b81c89..d2f6900e46 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowKillSwitchSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowKillSwitchSpec.scala index fe53d8eaec..d875c77b1a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowKillSwitchSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowKillSwitchSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala index cf25630ae2..0597562e2a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala index c97868c290..2ca473348c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala index 29e745ec2f..043e69bd0c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncSpec.scala @@ -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 { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala index 390622eb6c..02ef828d06 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapAsyncUnorderedSpec.scala @@ -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 { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala index 15edd66e2f..358d2ee914 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala index 194fbed015..6d843cf3a3 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala index 74352d6d3b..3d13c2abaa 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala @@ -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 { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOrElseSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOrElseSpec.scala index 7e3c864a92..80e8c38254 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOrElseSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOrElseSpec.scala @@ -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() } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala index d6d93d6332..fadb955ee6 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala @@ -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)) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrependSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrependSpec.scala index 6811627af3..32bfd69853 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrependSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrependSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala index 5dc1aab051..3cd81a4077 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverWithSpec.scala index 5747ac118b..9f7d60f7fb 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverWithSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverWithSpec.scala @@ -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 } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanAsyncSpec.scala index 1839244803..7b12c49112 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanAsyncSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanAsyncSpec.scala @@ -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 } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSlidingSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSlidingSpec.scala index 08f6c48d40..a9a2d29e37 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSlidingSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSlidingSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala index bc9cfba861..5b91b440af 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala @@ -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") diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala index 66e5395fa7..a6664688c0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala index cc0dddc8b0..6f925970f7 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala index 481969ac5f..6e2e44b4d7 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWhileSpec.scala @@ -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]) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWithinSpec.scala index 5debd09f4c..dd0615b91c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWithinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeWithinSpec.scala @@ -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) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala index 40854e5bf7..da0b9a142f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchSpec.scala index 47a2823558..ee790be392 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala index d1b87285e8..f58c16cbca 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala @@ -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] diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipSpec.scala index ed80fd20f0..a6e1632404 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipSpec.scala @@ -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]) = { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipWithIndexSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipWithIndexSpec.scala index 6dc412ca9c..10daa87f40 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipWithIndexSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipWithIndexSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipWithSpec.scala index 6266f1d347..fa4b6a5d1a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipWithSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowZipWithSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FramingSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FramingSpec.scala index a5577e54a5..574c7211d5 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FramingSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FramingSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FutureFlattenSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FutureFlattenSourceSpec.scala index 42de445607..01f193a0ec 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FutureFlattenSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FutureFlattenSourceSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala index 7bc003ecd2..d4e8e8450a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala index be3c510f69..73e4270012 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala index 457705575a..78a76966ab 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala index 66dc157c95..362ad5249a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala index 577562ee9b..2aced0d47b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala @@ -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!") diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeLatestSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeLatestSpec.scala index 643ee7715c..74c151aa19 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeLatestSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeLatestSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePreferredSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePreferredSpec.scala index 700f027c53..4bfd7a93cb 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePreferredSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergePreferredSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSortedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSortedSpec.scala index d7dde48f33..94f6b74090 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSortedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSortedSpec.scala @@ -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 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala index b8bbc9240b..959c348375 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala @@ -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) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala index 0c1147e422..f901222067 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala index b48cfed8f1..97eccd224c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala @@ -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() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala index d16a3760bd..2cfc21adee 100755 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala @@ -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) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala index dd82b3c737..376da8ef09 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala @@ -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 { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestWithSpec.scala index 72218bc27e..1f064d02c3 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestWithSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipLatestWithSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipNSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipNSpec.scala index 39d1a841b5..d37e1b74eb 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipNSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipNSpec.scala @@ -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) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala index ed480acaff..047f545873 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala @@ -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) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithNSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithNSpec.scala index 8149337a69..68b85c448a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithNSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithNSpec.scala @@ -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 { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala index 7ba3a7134a..cba4679385 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala @@ -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)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HubSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HubSpec.scala index 3e5a228311..0a7891e8a1 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HubSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HubSpec.scala @@ -82,7 +82,7 @@ class HubSpec extends StreamSpec { expectMsg(1) expectMsg(2) expectMsg(3) - expectNoMsg(100.millis) + expectNoMessage(100.millis) // One element consumed (it was requested), demand 0 remains at producer downstream.expectNext(1) @@ -96,7 +96,7 @@ class HubSpec extends StreamSpec { expectMsg(4) expectMsg(5) - expectNoMsg(100.millis) + expectNoMessage(100.millis) // Two additional elements have been sent: // - 3, 4, 5 are pending @@ -108,7 +108,7 @@ class HubSpec extends StreamSpec { sub.request(1) downstream.expectNext(3) - expectNoMsg(100.millis) + expectNoMessage(100.millis) sub.request(1) downstream.expectNext(4) @@ -329,8 +329,8 @@ class HubSpec extends StreamSpec { downstream1.expectNext(1, 2, 3, 4) downstream2.expectNext(1, 2, 3, 4, 5, 6, 7, 8) - downstream1.expectNoMsg(100.millis) - downstream2.expectNoMsg(100.millis) + downstream1.expectNoMessage(100.millis) + downstream2.expectNoMessage(100.millis) upstream.sendError(TE("Failed")) @@ -410,7 +410,7 @@ class HubSpec extends StreamSpec { "work in the happy case with one stream" in assertAllStagesStopped { val source = - Source(1 to 10).runWith(PartitionHub.sink((size, elem) => 0, startAfterNrOfConsumers = 0, bufferSize = 8)) + Source(1 to 10).runWith(PartitionHub.sink((_, _) => 0, startAfterNrOfConsumers = 0, bufferSize = 8)) source.runWith(Sink.seq).futureValue should ===(1 to 10) } @@ -429,7 +429,7 @@ class HubSpec extends StreamSpec { val source = Source(0 until 10).runWith(PartitionHub.statefulSink(() => { var n = 0L - (info, elem) => { + (info, _) => { n += 1 info.consumerIdByIdx((n % info.size).toInt) } @@ -465,7 +465,7 @@ class HubSpec extends StreamSpec { "be able to use as fastest consumer router" in assertAllStagesStopped { val source = Source(0 until 1000).runWith( PartitionHub.statefulSink( - () => (info, elem) => info.consumerIds.toVector.minBy(id => info.queueSize(id)), + () => (info, _) => info.consumerIds.toVector.minBy(id => info.queueSize(id)), startAfterNrOfConsumers = 2, bufferSize = 4)) val result1 = source.runWith(Sink.seq) @@ -501,7 +501,7 @@ class HubSpec extends StreamSpec { testSource.sendNext(7) probe1.expectNext(5) probe1.expectNext(7) - probe0.expectNoMsg(10.millis) + probe0.expectNoMessage(10.millis) probe0.request(10) probe0.expectNext(6) @@ -513,8 +513,7 @@ class HubSpec extends StreamSpec { "route unevenly" in assertAllStagesStopped { val (testSource, hub) = TestSource .probe[Int] - .toMat(PartitionHub.sink((size, elem) => (elem % 3) % 2, startAfterNrOfConsumers = 2, bufferSize = 8))( - Keep.both) + .toMat(PartitionHub.sink((_, elem) => (elem % 3) % 2, startAfterNrOfConsumers = 2, bufferSize = 8))(Keep.both) .run() val probe0 = hub.runWith(TestSink.probe[Int]) val probe1 = hub.runWith(TestSink.probe[Int]) @@ -547,7 +546,7 @@ class HubSpec extends StreamSpec { "backpressure" in assertAllStagesStopped { val (testSource, hub) = TestSource .probe[Int] - .toMat(PartitionHub.sink((size, elem) => 0, startAfterNrOfConsumers = 2, bufferSize = 4))(Keep.both) + .toMat(PartitionHub.sink((_, _) => 0, startAfterNrOfConsumers = 2, bufferSize = 4))(Keep.both) .run() val probe0 = hub.runWith(TestSink.probe[Int]) val probe1 = hub.runWith(TestSink.probe[Int]) @@ -611,8 +610,8 @@ class HubSpec extends StreamSpec { downstream1.expectNext(0, 2, 4, 6) downstream2.expectNext(1, 3, 5, 7, 9, 11, 13, 15) - downstream1.expectNoMsg(100.millis) - downstream2.expectNoMsg(100.millis) + downstream1.expectNoMessage(100.millis) + downstream2.expectNoMessage(100.millis) upstream.sendError(TE("Failed")) @@ -633,7 +632,7 @@ class HubSpec extends StreamSpec { "remember completion for materialisations after completion" in { val (sourceProbe, source) = - TestSource.probe[Unit].toMat(PartitionHub.sink((size, elem) => 0, startAfterNrOfConsumers = 0))(Keep.both).run() + TestSource.probe[Unit].toMat(PartitionHub.sink((_, _) => 0, startAfterNrOfConsumers = 0))(Keep.both).run() val sinkProbe = source.runWith(TestSink.probe[Unit]) sourceProbe.sendComplete() @@ -651,7 +650,7 @@ class HubSpec extends StreamSpec { "properly signal error to consumers arriving after producer finished" in assertAllStagesStopped { val source = - Source.failed[Int](TE("Fail!")).runWith(PartitionHub.sink((size, elem) => 0, startAfterNrOfConsumers = 0)) + Source.failed[Int](TE("Fail!")).runWith(PartitionHub.sink((_, _) => 0, startAfterNrOfConsumers = 0)) // Wait enough so the Hub gets the failure. This is racy, but this is fine because both // cases should work in the end Thread.sleep(10) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/JsonFramingSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/JsonFramingSpec.scala index 5b6948d894..d177cfe56b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/JsonFramingSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/JsonFramingSpec.scala @@ -118,7 +118,7 @@ class JsonFramingSpec extends AkkaSpec { val input = TestPublisher.probe[ByteString]() val output = TestSubscriber.probe[String]() - val result = Source + Source .fromPublisher(input) .via(JsonFraming.objectScanner(Int.MaxValue)) .map(_.utf8String) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazilyAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazilyAsyncSpec.scala index 2fc4dc91c4..ac1d9d7ad7 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazilyAsyncSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazilyAsyncSpec.scala @@ -37,7 +37,7 @@ class LazilyAsyncSpec extends StreamSpec with DefaultTimeout with ScalaFutures { val probe = TestSubscriber.probe[Int]() val constructed = new AtomicBoolean(false) - val result = Source + Source .lazilyAsync { () => constructed.set(true); Future(42) } @@ -83,7 +83,7 @@ class LazilyAsyncSpec extends StreamSpec with DefaultTimeout with ScalaFutures { "propagate failed future from factory" in assertAllStagesStopped { val probe = TestSubscriber.probe[Int]() val failure = new RuntimeException("too bad") - val materialization = Source + Source .lazilyAsync { () => Future.failed(failure) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala index a2bc0c9bdc..5e377f9e91 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala @@ -46,7 +46,7 @@ class LazyFlowSpec extends StreamSpec { sourceSub.expectRequest(1) sourceSub.sendNext(0) sourceSub.expectRequest(1) - sourceProbe.expectNoMsg(200.millis) + sourceProbe.expectNoMessage(200.millis) p.success(Flow[Int]) flowProbe.request(99) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala index a119124f7a..23e5e0512f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala @@ -43,7 +43,7 @@ class LazySinkSpec extends StreamSpec { sourceSub.expectRequest(1) sourceSub.sendNext(0) sourceSub.expectRequest(1) - sourceProbe.expectNoMsg(200.millis) + sourceProbe.expectNoMessage(200.millis) a[TimeoutException] shouldBe thrownBy { Await.result(futureProbe, remainingOrDefault) } p.success(TestSink.probe[Int]) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySourceSpec.scala index dff9bfe24a..2556b40a69 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySourceSpec.scala @@ -33,7 +33,7 @@ class LazySourceSpec extends StreamSpec with DefaultTimeout with ScalaFutures { "never construct the source when there was no demand" in assertAllStagesStopped { val probe = TestSubscriber.probe[Int]() val constructed = new AtomicBoolean(false) - val result = Source + Source .fromGraph(LazySource { () => constructed.set(true); Source(List(1, 2, 3)) }) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala index d021e45c69..3e9b7bfadd 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala @@ -29,7 +29,7 @@ class MaybeSourceSpec extends StreamSpec with DefaultTimeout { val subs = c.expectSubscription() subs.request(1000) - c.expectNoMsg(300.millis) + c.expectNoMessage(300.millis) subs.cancel() f.future.futureValue shouldEqual None @@ -94,7 +94,7 @@ class MaybeSourceSpec extends StreamSpec with DefaultTimeout { val c = TestSubscriber.manualProbe[Int]() neverPub.subscribe(c) - val subs = c.expectSubscription() + c.expectSubscription() mat.shutdown() f.future.failed.futureValue shouldBe an[AbruptStageTerminationException] diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala index 41c009c7e6..e87dcdb038 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala @@ -305,7 +305,7 @@ class QueueSourceSpec extends StreamSpec { } "some elements not yet delivered to stage" in { - val (queue, probe) = + val (queue, _) = Source.queue[Unit](10, OverflowStrategy.fail).toMat(TestSink.probe)(Keep.both).run() intercept[StreamDetachedException] { Await.result((1 to 15).map(_ => queue.offer(())).last, 3.seconds) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala index d68a20ba4c..0e476e8b98 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala @@ -7,7 +7,6 @@ package akka.stream.scaladsl import java.util.stream.Collectors import akka.actor.ActorSystem -import akka.dispatch.Dispatchers import akka.stream._ import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.impl.StreamSupervisor.Children diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachAsyncSpec.scala index a1c9d0e266..e93791e869 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachAsyncSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachAsyncSpec.scala @@ -25,7 +25,7 @@ class SinkForeachAsyncSpec extends StreamSpec { "A foreachAsync" must { "handle empty source" in assertAllStagesStopped { import system.dispatcher - val p = Source(List.empty[Int]).runWith(Sink.foreachAsync(3)(a => Future {})) + val p = Source(List.empty[Int]).runWith(Sink.foreachAsync(3)(_ => Future {})) Await.result(p, remainingOrDefault) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala index 831bfa7ff0..54d34944e8 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkForeachParallelSpec.scala @@ -12,11 +12,13 @@ import akka.stream.Supervision._ import akka.stream.testkit.StreamSpec import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.testkit.{ TestLatch, TestProbe } +import com.github.ghik.silencer.silent import scala.concurrent.Await import scala.concurrent.duration._ import scala.util.control.NoStackTrace +@silent // tests deprecated APIs class SinkForeachParallelSpec extends StreamSpec { implicit val materializer = ActorMaterializer() @@ -58,7 +60,7 @@ class SinkForeachParallelSpec extends StreamSpec { Await.ready(latch(n), 5.seconds) })) probe.expectMsgAllOf(1, 2, 3, 4) - probe.expectNoMsg(200.millis) + probe.expectNoMessage(200.millis) assert(!p.isCompleted) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala index a75c9c9c87..1d821cfa80 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala @@ -15,12 +15,14 @@ import akka.stream.testkit.Utils._ import akka.stream.testkit._ import akka.stream.testkit.scaladsl.TestSink import akka.testkit.DefaultTimeout +import com.github.ghik.silencer.silent import org.reactivestreams.Publisher import org.scalatest.concurrent.ScalaFutures import scala.concurrent.{ Await, Future } import scala.concurrent.duration._ +@silent // tests deprecated APIs class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures { import GraphDSL.Implicits._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala index 94b8fe04f1..c138827db1 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala @@ -6,10 +6,11 @@ package akka.stream.scaladsl import akka.testkit.DefaultTimeout import org.scalatest.time.{ Millis, Span } + import scala.concurrent.{ Await, Future } import scala.concurrent.duration._ - import akka.stream.testkit.Utils.TE +import com.github.ghik.silencer.silent //#imports import akka.stream._ @@ -23,6 +24,7 @@ import java.util.stream.BaseStream import akka.stream.testkit.scaladsl.TestSink +@silent // tests assigning to typed val class SourceSpec extends StreamSpec with DefaultTimeout { implicit val materializer = ActorMaterializer() @@ -107,7 +109,7 @@ class SourceSpec extends StreamSpec with DefaultTimeout { val sub = out.expectSubscription() sub.request(10) - val subs = for (i <- 0 to 4) { + for (i <- 0 to 4) { val s = probes(i).expectSubscription() s.expectRequest() s.sendNext(i) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StageActorRefSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StageActorRefSpec.scala index d2c38c8bf3..ec3d65a41e 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StageActorRefSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StageActorRefSpec.scala @@ -139,7 +139,7 @@ class StageActorRefSpec extends StreamSpec with ImplicitSender { stageRef ! Add(40) val filter = EventFilter.custom { - case e: Logging.Warning => true + case _: Logging.Warning => true case _ => false } system.eventStream.publish(TestEvent.Mute(filter)) @@ -197,19 +197,20 @@ object StageActorRefSpec { def behavior(m: (ActorRef, Any)): Unit = { m match { - case (sender, Add(n)) => sum += n - case (sender, PullNow) => pull(in) + case (_, Add(n)) => sum += n + case (_, PullNow) => pull(in) case (sender, CallInitStageActorRef) => sender ! getStageActor(behavior).ref - case (sender, BecomeStringEcho) => + case (_, BecomeStringEcho) => getStageActor { case (theSender, msg) => theSender ! msg.toString } - case (sender, StopNow) => + case (_, StopNow) => p.trySuccess(sum) completeStage() case (sender, AddAndTell(n)) => sum += n sender ! sum + case _ => throw new RuntimeException("unexpected: " + m) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamRefsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamRefsSpec.scala index 7563829ea3..1cf5e3c897 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamRefsSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/StreamRefsSpec.scala @@ -28,7 +28,6 @@ object StreamRefsSpec { } class DataSourceActor(probe: ActorRef) extends Actor with ActorLogging { - import context.dispatcher implicit val mat = ActorMaterializer() def receive = { @@ -45,7 +44,7 @@ object StreamRefsSpec { case "give-infinite" => val source: Source[String, NotUsed] = Source.fromIterator(() => Iterator.from(1)).map("ping-" + _) - val (r: NotUsed, ref: SourceRef[String]) = source.toMat(StreamRefs.sourceRef())(Keep.both).run() + val (_: NotUsed, ref: SourceRef[String]) = source.toMat(StreamRefs.sourceRef())(Keep.both).run() sender() ! ref @@ -399,7 +398,7 @@ class StreamRefsSpec(config: Config) extends AkkaSpec(config) with ImplicitSende val p2: TestPublisher.Probe[String] = TestSource.probe[String].to(sinkRef).run() p1.ensureSubscription() - val req = p1.expectRequest() + p1.expectRequest() // will be cancelled immediately, since it's 2nd: p2.ensureSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala index b1a02da92a..7693c9b24b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala @@ -40,8 +40,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends StreamSpec(conf) { "timeout and cancel substream publishers when no-one subscribes to them after some time (time them out)" in assertAllStagesStopped { val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]() val publisherProbe = TestPublisher.probe[Int]() - val publisher = - Source.fromPublisher(publisherProbe).groupBy(3, _ % 3).lift(_ % 3).runWith(Sink.fromSubscriber(subscriber)) + Source.fromPublisher(publisherProbe).groupBy(3, _ % 3).lift(_ % 3).runWith(Sink.fromSubscriber(subscriber)) val downstreamSubscription = subscriber.expectSubscription() downstreamSubscription.request(100) @@ -81,8 +80,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends StreamSpec(conf) { "timeout and stop groupBy parent actor if none of the substreams are actually consumed" in assertAllStagesStopped { val publisherProbe = TestPublisher.probe[Int]() val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]() - val publisher = - Source.fromPublisher(publisherProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.fromSubscriber(subscriber)) + Source.fromPublisher(publisherProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.fromSubscriber(subscriber)) val downstreamSubscription = subscriber.expectSubscription() downstreamSubscription.request(100) @@ -92,15 +90,14 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends StreamSpec(conf) { publisherProbe.sendNext(3) publisherProbe.sendComplete() - val (_, s1) = subscriber.expectNext() - val (_, s2) = subscriber.expectNext() + val (_, _) = subscriber.expectNext() + val (_, _) = subscriber.expectNext() } "not timeout and cancel substream publishers when they have been subscribed to" in { val publisherProbe = TestPublisher.probe[Int]() val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]() - val publisher = - Source.fromPublisher(publisherProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.fromSubscriber(subscriber)) + Source.fromPublisher(publisherProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.fromSubscriber(subscriber)) val downstreamSubscription = subscriber.expectSubscription() downstreamSubscription.request(100) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TakeLastSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TakeLastSinkSpec.scala index 297403b6e7..cd35c201d5 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TakeLastSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TakeLastSinkSpec.scala @@ -6,10 +6,12 @@ package akka.stream.scaladsl import akka.stream.testkit.{ StreamSpec, TestPublisher } import akka.stream.{ AbruptTerminationException, ActorMaterializer, ActorMaterializerSettings } +import com.github.ghik.silencer.silent import scala.collection.immutable import scala.concurrent.{ Await, Future } +@silent class TakeLastSinkSpec extends StreamSpec { val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala index c7c306ba2c..ec140268a8 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala @@ -21,10 +21,10 @@ class TickSourceSpec extends StreamSpec { val sub = c.expectSubscription() sub.request(2) c.expectNext("tick") - c.expectNoMsg(200.millis) + c.expectNoMessage(200.millis) c.expectNext("tick") sub.cancel() - c.expectNoMsg(200.millis) + c.expectNoMessage(200.millis) } "drop ticks when not requested" taggedAs TimingTest in { @@ -33,15 +33,15 @@ class TickSourceSpec extends StreamSpec { val sub = c.expectSubscription() sub.request(2) c.expectNext("tick") - c.expectNoMsg(200.millis) + c.expectNoMessage(200.millis) c.expectNext("tick") - c.expectNoMsg(1400.millis) + c.expectNoMessage(1400.millis) sub.request(2) c.expectNext("tick") - c.expectNoMsg(200.millis) + c.expectNoMessage(200.millis) c.expectNext("tick") sub.cancel() - c.expectNoMsg(200.millis) + c.expectNoMessage(200.millis) } "reject multiple subscribers, but keep the first" taggedAs TimingTest in { @@ -54,7 +54,7 @@ class TickSourceSpec extends StreamSpec { c2.expectSubscriptionAndError() sub1.request(1) c1.expectNext("tick") - c1.expectNoMsg(200.millis) + c1.expectNoMessage(200.millis) sub1.request(2) c1.expectNext("tick") sub1.cancel() @@ -77,9 +77,9 @@ class TickSourceSpec extends StreamSpec { val sub = c.expectSubscription() sub.request(1000) c.expectNext(1) - c.expectNoMsg(200.millis) + c.expectNoMessage(200.millis) c.expectNext(2) - c.expectNoMsg(200.millis) + c.expectNoMessage(200.millis) sub.cancel() } @@ -89,9 +89,9 @@ class TickSourceSpec extends StreamSpec { val cancellable = tickSource.to(Sink.fromSubscriber(c)).run() val sub = c.expectSubscription() sub.request(2) - c.expectNoMsg(600.millis) + c.expectNoMessage(600.millis) c.expectNext("tick") - c.expectNoMsg(200.millis) + c.expectNoMessage(200.millis) c.expectNext("tick") cancellable.cancel() awaitCond(cancellable.isCancelled) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala index aca4595c34..f404053eb2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala @@ -8,7 +8,6 @@ import java.util.concurrent.atomic.AtomicInteger import akka.Done import akka.actor.ActorSystem -import akka.dispatch.Dispatchers import akka.stream.impl.StreamSupervisor.Children import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.testkit.Utils._ @@ -82,7 +81,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { probe.request(1) resource.created.futureValue - probe.expectNoMsg(200.millis) + probe.expectNoMessage(200.millis) createPromise.success(Done) values.foreach { n => @@ -253,7 +252,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { Source .unfoldResourceAsync[Int, Iterator[Int]]( () => Future.successful(List(1, 2, 3).iterator), - reader => throw TE("read-error"), + _ => throw TE("read-error"), _ => throw new TE("close-error")) .withAttributes(ActorAttributes.supervisionStrategy(Supervision.restartingDecider)) .runWith(Sink.fromSubscriber(out)) @@ -267,7 +266,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { Source .unfoldResourceAsync[Int, Iterator[Int]]( () => Future.successful(List(1, 2, 3).iterator), - reader => throw TE("read-error"), + _ => throw TE("read-error"), _ => Future.failed(new TE("close-error"))) .withAttributes(ActorAttributes.supervisionStrategy(Supervision.restartingDecider)) .runWith(Sink.fromSubscriber(out)) @@ -284,7 +283,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { () => if (startCounter.incrementAndGet() < 2) Future.successful(List(1, 2, 3).iterator) else throw TE("start-error"), - reader => throw TE("read-error"), + _ => throw TE("read-error"), _ => Future.successful(Done)) .withAttributes(ActorAttributes.supervisionStrategy(Supervision.restartingDecider)) .runWith(Sink.fromSubscriber(out)) @@ -301,7 +300,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { () => if (startCounter.incrementAndGet() < 2) Future.successful(List(1, 2, 3).iterator) else Future.failed(TE("start-error")), - reader => throw TE("read-error"), + _ => throw TE("read-error"), _ => Future.successful(Done)) .withAttributes(ActorAttributes.supervisionStrategy(Supervision.restartingDecider)) .runWith(Sink.fromSubscriber(out)) @@ -314,7 +313,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig) val materializer = ActorMaterializer()(sys) try { - val p = Source + Source .unfoldResourceAsync[String, Unit]( () => Promise[Unit].future, // never complete _ => ???, diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala index 9d5b3ff28e..a462949cd5 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala @@ -10,7 +10,6 @@ import java.nio.file.Files import java.util.concurrent.atomic.AtomicInteger import akka.actor.ActorSystem -import akka.dispatch.Dispatchers import akka.stream.ActorAttributes._ import akka.stream.Supervision._ import akka.stream.impl.StreamSupervisor.Children @@ -67,7 +66,7 @@ class UnfoldResourceSourceSpec extends StreamSpec(UnboundedMailboxConfig) { c.expectNext() should ===(chunks.next()) sub.request(1) c.expectNext() should ===(chunks.next()) - c.expectNoMsg(300.millis) + c.expectNoMessage(300.millis) while (chunks.hasNext) { sub.request(1) @@ -112,7 +111,7 @@ class UnfoldResourceSourceSpec extends StreamSpec(UnboundedMailboxConfig) { p.subscribe(c) val sub = c.expectSubscription() - (0 to 19).foreach(i => { + (0 to 19).foreach(_ => { sub.request(1) c.expectNext() should ===(manyLinesArray(0)) }) @@ -140,7 +139,7 @@ class UnfoldResourceSourceSpec extends StreamSpec(UnboundedMailboxConfig) { p.subscribe(c) val sub = c.expectSubscription() - (0 to 121).foreach(i => { + (0 to 121).foreach(_ => { sub.request(1) c.expectNext().utf8String should ===(nextChunk().toString) }) diff --git a/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala index 606c9198ce..7dc193c5bf 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala @@ -9,8 +9,6 @@ import akka.stream.scaladsl.{ Flow, GraphDSL, Keep, Merge, Partition, Sink, Sour import akka.stream.testkit.scaladsl.TestSink import akka.stream.testkit.StreamSpec -import scala.concurrent.duration._ - class MaterializerStateSpec extends StreamSpec { "The MaterializerSnapshotting" must { diff --git a/project/AkkaDisciplinePlugin.scala b/project/AkkaDisciplinePlugin.scala index 4c01759254..6a973bc57e 100644 --- a/project/AkkaDisciplinePlugin.scala +++ b/project/AkkaDisciplinePlugin.scala @@ -25,7 +25,6 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport { "akka-bench-jmh", "akka-bench-jmh-typed", "akka-persistence-tck", - "akka-stream-tests", "akka-stream-tests-tck") val strictProjects = Set("akka-discovery", "akka-protobuf", "akka-coordination")