remove akka.stream.testkit.AkkaSpec, fixes #19879

This commit is contained in:
Roland Kuhn 2016-02-25 14:27:45 +01:00
parent 06b4d54960
commit 45e97e8467
183 changed files with 265 additions and 346 deletions

View file

@ -13,7 +13,6 @@ import scala.concurrent.Await
import akka.util.Timeout
object FSMActorSpec {
val timeout = Timeout(2 seconds)
class Latches(implicit system: ActorSystem) {
val unlockedLatch = TestLatch()
@ -103,6 +102,8 @@ object FSMActorSpec {
class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with ImplicitSender {
import FSMActorSpec._
val timeout = Timeout(2 seconds)
"An FSM Actor" must {
"unlock the lock" in {

View file

@ -14,7 +14,7 @@ import scala.util.Failure
import language.postfixOps
class AskSpec extends AkkaSpec with ScalaFutures {
class AskSpec extends AkkaSpec {
"The “ask” pattern" must {
"send request to actor and wrap the answer in Future" in {
@ -230,4 +230,3 @@ class AskSpec extends AkkaSpec with ScalaFutures {
}
}

View file

@ -18,7 +18,7 @@ object PromiseRefSpec {
case object SecondMessage
}
class PromiseRefSpec extends AkkaSpec with ImplicitSender with ScalaFutures {
class PromiseRefSpec extends AkkaSpec with ImplicitSender {
import PromiseRefSpec._
import akka.pattern._
@ -71,4 +71,3 @@ class PromiseRefSpec extends AkkaSpec with ImplicitSender with ScalaFutures {
}
}
}

View file

@ -14,7 +14,7 @@ object ExplicitAskSpec {
case class Response(sentFrom: ActorRef)
}
class ExplicitAskSpec extends AkkaSpec with ScalaFutures {
class ExplicitAskSpec extends AkkaSpec {
import ExplicitAskSpec._
"The “ask” pattern with explicit sender" must {
@ -47,4 +47,3 @@ class ExplicitAskSpec extends AkkaSpec with ScalaFutures {
}
}

View file

@ -5,7 +5,7 @@ package akka.util
import org.scalatest.WordSpec
import org.scalatest.Matchers
import org.scalautils.ConversionCheckedTripleEquals
import org.scalactic.ConversionCheckedTripleEquals
object TypedMultiMapSpec {
trait AbstractKey { type Type }
@ -69,4 +69,4 @@ class TypedMultiMapSpec extends WordSpec with Matchers with ConversionCheckedTri
}
}
}

View file

@ -7,14 +7,13 @@ import akka.actor.{ ActorRef, PoisonPill }
import akka.contrib.circuitbreaker.CircuitBreakerProxy._
import akka.testkit.{ AkkaSpec, TestProbe }
import akka.util.Timeout
import org.scalatest.concurrent.ScalaFutures
import org.scalatest.time.{ Millis, Second, Span }
import org.scalatest.{ GivenWhenThen, Matchers }
import scala.concurrent.duration._
import scala.language.postfixOps
class CircuitBreakerProxySpec extends AkkaSpec() with GivenWhenThen with Matchers with ScalaFutures {
class CircuitBreakerProxySpec extends AkkaSpec() with GivenWhenThen {
val baseCircuitBreakerPropsBuilder =
CircuitBreakerPropsBuilder(
@ -439,7 +438,6 @@ class CircuitBreakerProxySpec extends AkkaSpec() with GivenWhenThen with Matcher
}
"Ask Extension" should {
implicit val patienceConfig = PatienceConfig(timeout = Span(1, Second), interval = Span(100, Millis))
import Implicits.askWithCircuitBreaker
import scala.concurrent.ExecutionContext.Implicits.global
@ -488,7 +486,6 @@ class CircuitBreakerProxySpec extends AkkaSpec() with GivenWhenThen with Matcher
}
"Future Extension" should {
implicit val patienceConfig = PatienceConfig(timeout = Span(1, Second), interval = Span(100, Millis))
import Implicits.futureExtensions
import akka.pattern.ask

View file

@ -4,7 +4,7 @@
package docs.http.scaladsl
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
class MarshalSpec extends AkkaSpec {

View file

@ -9,7 +9,7 @@ import akka.http.scaladsl.model._
//#import-model
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.util.ByteString
import akka.http.scaladsl.model.headers.BasicHttpCredentials

View file

@ -5,7 +5,7 @@
package docs.http.scaladsl
import akka.stream.{ Materializer, ActorMaterializer }
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
class UnmarshalSpec extends AkkaSpec {

View file

@ -8,7 +8,7 @@ import akka.actor.Props
import akka.stream.ActorMaterializer
import akka.stream.actor.ActorPublisher
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
object ActorPublisherDocSpec {

View file

@ -15,7 +15,7 @@ import akka.stream.actor.ActorSubscriberMessage
import akka.stream.actor.MaxInFlightRequestStrategy
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
object ActorSubscriberDocSpec {
//#worker-pool

View file

@ -3,7 +3,7 @@
*/
package docs.stream
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.scaladsl._
import akka.stream._
import akka.util.ByteString
@ -143,7 +143,7 @@ object BidiFlowDocSpec {
})
}
class BidiFlowDocSpec extends AkkaSpec with ConversionCheckedTripleEquals {
class BidiFlowDocSpec extends AkkaSpec {
import BidiFlowDocSpec._
implicit val materializer = ActorMaterializer()

View file

@ -7,7 +7,7 @@ import akka.NotUsed
import akka.stream._
import akka.stream.scaladsl.Tcp.OutgoingConnection
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.util.ByteString
import scala.concurrent.{ Future, Promise }

View file

@ -7,7 +7,7 @@ import akka.NotUsed
import akka.actor.Cancellable
import akka.stream.{ ClosedShape, FlowShape }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import scala.concurrent.{ Promise, Future }

View file

@ -8,7 +8,7 @@ import akka.stream.ActorMaterializer
import akka.stream.ActorMaterializerSettings
import akka.stream.Supervision
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.Attributes
import akka.stream.ActorAttributes
import scala.concurrent.duration._

View file

@ -6,7 +6,7 @@ package docs.stream
import akka.NotUsed
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import scala.collection.immutable
import scala.concurrent.{ Future, Await }

View file

@ -3,7 +3,7 @@ package docs.stream
import akka.NotUsed
import akka.stream.FlowShape
import akka.stream.scaladsl.{ GraphDSL, Merge, Balance, Source, Flow }
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
class FlowParallelismDocSpec extends AkkaSpec {

View file

@ -2,7 +2,7 @@ package docs.stream
import akka.stream.{ ClosedShape, OverflowStrategy, ActorMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
class GraphCyclesSpec extends AkkaSpec {

View file

@ -8,8 +8,8 @@ import akka.stream.scaladsl.{ Keep, Sink, Flow, Source }
import akka.stream.stage._
import akka.stream._
import akka.stream.testkit.{ TestPublisher, TestSubscriber, AkkaSpec }
import akka.testkit.TestLatch
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
import akka.testkit.{ AkkaSpec, TestLatch }
import scala.collection.mutable
import scala.concurrent.{ Promise, Await, Future }
@ -519,4 +519,4 @@ class GraphStageDocSpec extends AkkaSpec {
sub.cancel()
}
}
}

View file

@ -6,7 +6,7 @@ package docs.stream
import akka.NotUsed
import scala.concurrent.duration._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.scaladsl._
import akka.stream.ActorMaterializer
import scala.concurrent.Future

View file

@ -4,7 +4,7 @@
package docs.stream
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
class MigrationsScala extends AkkaSpec {

View file

@ -12,7 +12,7 @@ import scala.math._
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.collection.immutable
import akka.testkit.TestLatch
import akka.testkit.{ AkkaSpec, TestLatch }
class RateTransformationDocSpec extends AkkaSpec {

View file

@ -8,6 +8,7 @@ import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.testkit._
import org.reactivestreams.Processor
import akka.testkit.AkkaSpec
class ReactiveStreamsDocSpec extends AkkaSpec {
import TwitterStreamQuickstartDocSpec._

View file

@ -3,7 +3,7 @@ package docs.stream
import akka.NotUsed
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
class StreamBuffersRateSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer()

View file

@ -6,7 +6,7 @@ package docs.stream
import akka.actor.ActorRef
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._

View file

@ -10,7 +10,7 @@ import akka.stream.testkit.scaladsl._
import scala.util._
import scala.concurrent.duration._
import scala.concurrent._
import akka.testkit.TestProbe
import akka.testkit.{ AkkaSpec, TestProbe }
import akka.pattern
class StreamTestKitDocSpec extends AkkaSpec {

View file

@ -15,7 +15,7 @@ import scala.concurrent.Future
//#imports
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
object TwitterStreamQuickstartDocSpec {
//#model

View file

@ -8,8 +8,7 @@ import scala.concurrent.Future
import org.scalatest.concurrent.ScalaFutures
import scala.concurrent.duration._
class RecipeSeq extends RecipeSpec with ScalaFutures {
implicit val patience = PatienceConfig(3.seconds)
class RecipeSeq extends RecipeSpec {
"Draining to a strict sequence" must {

View file

@ -1,7 +1,7 @@
package docs.stream.cookbook
import akka.stream.ActorMaterializer
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
trait RecipeSpec extends AkkaSpec {

View file

@ -10,6 +10,7 @@ import akka.stream.scaladsl.{ FileIO, Sink, Source }
import akka.stream.testkit.Utils._
import akka.stream.testkit._
import akka.util.ByteString
import akka.testkit.AkkaSpec
import scala.concurrent.Future

View file

@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicReference
import akka.stream._
import akka.stream.scaladsl.Tcp._
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.testkit.TestProbe
import akka.util.ByteString
import docs.utils.TestUtils

View file

@ -1,13 +1,13 @@
package akka.http.impl.engine.client
import javax.net.ssl.SSLContext
import akka.http.scaladsl.{ ConnectionContext, Http }
import akka.http.scaladsl.model.{ HttpResponse, HttpRequest }
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.testkit.{ TestSubscriber, TestPublisher, AkkaSpec, TestUtils, Utils }
import akka.stream.testkit.{ TestSubscriber, TestPublisher, TestUtils, Utils }
import akka.http.scaladsl.model.headers
import akka.testkit.AkkaSpec
class ClientCancellationSpec extends AkkaSpec("""
akka.loglevel = DEBUG

View file

@ -9,7 +9,6 @@ import java.nio.ByteBuffer
import java.nio.channels.{ SocketChannel, ServerSocketChannel }
import java.util.concurrent.atomic.AtomicInteger
import akka.http.impl.settings.ConnectionPoolSettingsImpl
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NonFatal
@ -20,10 +19,11 @@ import akka.http.impl.util.{ SingletonException, StreamUtils }
import akka.http.scaladsl.settings.{ ClientConnectionSettings, ConnectionPoolSettings, ServerSettings }
import akka.stream.{ ActorMaterializer }
import akka.stream.TLSProtocol._
import akka.stream.testkit.{ TestPublisher, TestSubscriber, AkkaSpec }
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
import akka.stream.scaladsl._
import akka.http.scaladsl.model.headers._
import akka.http.scaladsl.model._
import akka.testkit.AkkaSpec
class ConnectionPoolSpec extends AkkaSpec("""
akka.loggers = []

View file

@ -10,15 +10,14 @@ import scala.concurrent.Await
import scala.concurrent.duration._
import akka.stream.{ ActorMaterializerSettings, FlowShape, ActorMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.http.scaladsl.{ Http, TestUtils }
import akka.http.scaladsl.model._
import akka.stream.testkit.Utils
import org.scalatest.concurrent.ScalaFutures
class HighLevelOutgoingConnectionSpec extends AkkaSpec with ScalaFutures {
class HighLevelOutgoingConnectionSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system).withFuzzing(true))
implicit val patience = PatienceConfig(1.second)
"The connection-level client implementation" should {

View file

@ -6,7 +6,7 @@ package akka.http.impl.engine.client
import akka.actor.ActorSystem
import akka.http.scaladsl.settings.{ ClientConnectionSettings, ConnectionPoolSettings, ServerSettings }
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import com.typesafe.config.ConfigFactory
class HttpConfigurationSpec extends AkkaSpec {

View file

@ -11,7 +11,7 @@ import org.scalatest.concurrent.ScalaFutures
import akka.http.scaladsl.settings.ClientConnectionSettings
import akka.util.ByteString
import akka.event.NoLogging
import akka.stream.{ClosedShape, ActorMaterializer}
import akka.stream.{ ClosedShape, ActorMaterializer }
import akka.stream.TLSProtocol._
import akka.stream.testkit._
import akka.stream.scaladsl._
@ -20,6 +20,7 @@ import akka.http.scaladsl.model.HttpMethods._
import akka.http.scaladsl.model._
import akka.http.scaladsl.model.headers._
import akka.http.impl.util._
import akka.testkit.AkkaSpec
class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF") with Inside {
implicit val materializer = ActorMaterializer()
@ -179,7 +180,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
}
}
"proceed to next response once previous response's entity has been drained" in new TestSetup with ScalaFutures {
"proceed to next response once previous response's entity has been drained" in new TestSetup {
def twice(action: => Unit): Unit = { action; action }
twice {
@ -203,11 +204,10 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|""")
val whenComplete = expectResponse().entity.dataBytes.runWith(Sink.ignore)
whenComplete.futureValue should be (akka.Done)
whenComplete.futureValue should be(akka.Done)
}
}
"handle several requests on one persistent connection" which {
"has a first response that was chunked" in new TestSetup {
requestsSub.sendNext(HttpRequest())

View file

@ -10,8 +10,9 @@ import akka.http.scaladsl.model._
import akka.http.scaladsl.settings.ParserSettings
import akka.stream.{ ActorMaterializer, Attributes }
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.testkit.{ TestSubscriber, TestPublisher, AkkaSpec }
import akka.stream.testkit.{ TestSubscriber, TestPublisher }
import akka.util.ByteString
import akka.testkit.AkkaSpec
class PrepareResponseSpec extends AkkaSpec {

View file

@ -9,7 +9,7 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.concurrent.ScalaFutures
import akka.stream.{ Server, Client, ActorMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.http.impl.util._
import akka.http.scaladsl.{ ConnectionContext, Http }
import akka.http.scaladsl.model.{ StatusCodes, HttpResponse, HttpRequest }
@ -21,7 +21,7 @@ class TlsEndpointVerificationSpec extends AkkaSpec("""
akka.loglevel = INFO
akka.io.tcp.trace-logging = off
akka.http.parsing.tls-session-info-header = on
""") with ScalaFutures {
""") {
implicit val materializer = ActorMaterializer()
/*
@ -30,7 +30,6 @@ class TlsEndpointVerificationSpec extends AkkaSpec("""
val includeTestsHittingActualWebsites = false
val timeout = Timeout(Span(3, Seconds))
implicit val patience = PatienceConfig(Span(10, Seconds))
"The client implementation" should {
"not accept certificates signed by unknown CA" in {

View file

@ -22,6 +22,7 @@ import headers._
import HttpEntity._
import MediaTypes._
import HttpMethods._
import akka.testkit.AkkaSpec
class HttpServerSpec extends AkkaSpec(
"""akka.loggers = []
@ -354,7 +355,7 @@ class HttpServerSpec extends AkkaSpec(
}
}
"proceed to next request once previous request's entity has beed drained" in new TestSetup with ScalaFutures {
"proceed to next request once previous request's entity has beed drained" in new TestSetup {
def twice(action: => Unit): Unit = { action; action }
twice {
@ -369,7 +370,7 @@ class HttpServerSpec extends AkkaSpec(
|""")
val whenComplete = expectRequest().entity.dataBytes.runWith(Sink.ignore)
whenComplete.futureValue should be (akka.Done)
whenComplete.futureValue should be(akka.Done)
}
}

View file

@ -21,12 +21,11 @@ import java.net.InetSocketAddress
import akka.stream.impl.fusing.GraphStages
import akka.util.ByteString
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.EventFilter
import akka.testkit.{ AkkaSpec, EventFilter }
class WebSocketIntegrationSpec extends AkkaSpec("akka.stream.materializer.debug.fuzzing-mode=off")
with ScalaFutures with ConversionCheckedTripleEquals with Eventually {
with Eventually {
implicit val patience = PatienceConfig(3.seconds)
implicit val materializer = ActorMaterializer()
"A WebSocket server" must {

View file

@ -4,18 +4,17 @@
package akka.http.impl.util
import java.util.concurrent.atomic.AtomicInteger
import akka.NotUsed
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Flow, Keep, Sink, Source }
import akka.stream.testkit.Utils._
import akka.stream.testkit.{ AkkaSpec, _ }
import akka.stream.testkit._
import org.scalactic.ConversionCheckedTripleEquals
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.testkit.AkkaSpec
class One2OneBidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
class One2OneBidiFlowSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer()
"A One2OneBidiFlow" must {

View file

@ -7,7 +7,7 @@ import akka.http.javadsl.model._
import org.scalatest.concurrent.ScalaFutures
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
class ConnectHttpSpec extends WordSpec with Matchers with BeforeAndAfterAll with ScalaFutures {
class ConnectHttpSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val httpContext = ConnectionContext.noEncryption()
val httpsContext = ConnectionContext.https(null)
@ -72,4 +72,4 @@ class ConnectHttpSpec extends WordSpec with Matchers with BeforeAndAfterAll with
ex.getMessage should include("non https scheme!")
}
}
}
}

View file

@ -25,7 +25,7 @@ import org.scalatest.{ WordSpec, Matchers, BeforeAndAfterAll }
import org.scalatest.concurrent.ScalaFutures
import scala.util.Try
class HttpExtensionApiSpec extends WordSpec with Matchers with BeforeAndAfterAll with ScalaFutures {
class HttpExtensionApiSpec extends WordSpec with Matchers with BeforeAndAfterAll {
// tries to cover all surface area of javadsl.Http

View file

@ -5,7 +5,6 @@
package akka.http.scaladsl
import scala.concurrent.duration._
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
import org.scalatest.concurrent.ScalaFutures
import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
@ -13,16 +12,14 @@ import akka.http.scaladsl.unmarshalling.Unmarshal
import akka.http.scaladsl.marshalling.Marshal
import akka.http.scaladsl.model._
import scala.concurrent.Await
import akka.testkit.AkkaSpec
class FormDataSpec extends WordSpec with Matchers with ScalaFutures with BeforeAndAfterAll {
implicit val system = ActorSystem(getClass.getSimpleName)
class FormDataSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer()
import system.dispatcher
val formData = FormData(Map("surname" -> "Smith", "age" -> "42"))
implicit val patience = PatienceConfig(3.seconds)
"The FormData infrastructure" should {
"properly round-trip the fields of www-urlencoded forms" in {
Marshal(formData).to[HttpEntity]
@ -38,7 +35,4 @@ class FormDataSpec extends WordSpec with Matchers with ScalaFutures with BeforeA
}
}
override def afterAll() = {
Await.result(system.terminate(), 10.seconds)
}
}

View file

@ -1,105 +0,0 @@
/*
* Copyright (C) 2009-2016 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.stream.testkit
import akka.testkit.TestKit
import akka.event.LoggingAdapter
import scala.concurrent.duration.FiniteDuration
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfterAll
import akka.dispatch.Dispatchers
import akka.actor.ActorSystem
import org.scalatest.Matchers
import com.typesafe.config.Config
import akka.event.Logging
import scala.concurrent.Future
import org.scalatest.WordSpecLike
import akka.testkit.TestEvent.Mute
import akka.testkit.DeadLettersFilter
import scala.concurrent.duration._
object AkkaSpec { // FIXME: remove once going back to project dependencies
val testConf: Config = ConfigFactory.parseString("""
akka {
loglevel = "WARNING"
stdout-loglevel = "WARNING"
actor {
default-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = 8
parallelism-factor = 2.0
parallelism-max = 8
}
}
}
}
""")
def mapToConfig(map: Map[String, Any]): Config = {
import scala.collection.JavaConverters._
ConfigFactory.parseMap(map.asJava)
}
def getCallerName(clazz: Class[_]): String = {
val s = (Thread.currentThread.getStackTrace map (_.getClassName) drop 1)
.dropWhile(_ matches "(java.lang.Thread|.*AkkaSpec.?$)")
val reduced = s.lastIndexWhere(_ == clazz.getName) match {
case -1 s
case z s drop (z + 1)
}
reduced.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_")
}
}
abstract class AkkaSpec(_system: ActorSystem)
extends TestKit(_system) with WordSpecLike with Matchers with BeforeAndAfterAll with WatchedByCoroner {
def this(config: Config) = this(ActorSystem(AkkaSpec.getCallerName(getClass),
ConfigFactory.load(config.withFallback(AkkaSpec.testConf))))
def this(s: String) = this(ConfigFactory.parseString(s))
def this(configMap: Map[String, _]) = this(AkkaSpec.mapToConfig(configMap))
def this() = this(ActorSystem(AkkaSpec.getCallerName(getClass), AkkaSpec.testConf))
val log: LoggingAdapter = Logging(system, this.getClass)
override val invokeBeforeAllAndAfterAllEvenIfNoTestsAreExpected = true
final override def beforeAll {
startCoroner
atStartup()
}
final override def afterAll {
beforeTermination()
shutdown()
afterTermination()
stopCoroner()
}
protected def atStartup() {}
protected def beforeTermination() {}
protected def afterTermination() {}
def spawn(dispatcherId: String = Dispatchers.DefaultDispatcherId)(body: Unit): Unit =
Future(body)(system.dispatchers.lookup(dispatcherId))
override def expectedTestDuration: FiniteDuration = 60.seconds
def muteDeadLetters(messageClasses: Class[_]*)(sys: ActorSystem = system): Unit =
if (!sys.log.isDebugEnabled) {
def mute(clazz: Class[_]): Unit =
sys.eventStream.publish(Mute(DeadLettersFilter(clazz)(occurrences = Int.MaxValue)))
if (messageClasses.isEmpty) mute(classOf[AnyRef])
else messageClasses foreach mute
}
}

View file

@ -9,6 +9,7 @@ import org.reactivestreams.Publisher
import scala.collection.immutable
import scala.util.control.NoStackTrace
import akka.stream.testkit.Utils._
import akka.testkit.AkkaSpec
abstract class BaseTwoStreamsSetup extends AkkaSpec {

View file

@ -6,8 +6,8 @@ package akka.stream.testkit
import akka.stream._
import akka.stream.scaladsl.Source
import akka.stream.testkit.scaladsl.TestSink
import scala.concurrent.duration._
import akka.testkit.AkkaSpec
class StreamTestKitSpec extends AkkaSpec {

View file

@ -9,6 +9,7 @@ import akka.stream.testkit.TestSubscriber._
import akka.stream.testkit.Utils._
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
import org.reactivestreams.Subscription
import akka.testkit.AkkaSpec
class TestPublisherSubscriberSpec extends AkkaSpec {

View file

@ -8,7 +8,7 @@ import scala.concurrent.duration._
import akka.actor.ActorSystem
import akka.actor.ActorSystemImpl
import org.testng.annotations.AfterClass
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.event.Logging
import akka.testkit.TestEvent
import akka.testkit.EventFilter

View file

@ -4,7 +4,6 @@
package akka.stream.tck
import java.io.{ File, FileWriter }
import akka.actor.ActorSystem
import akka.event.Logging
import akka.stream.scaladsl.FileIO
@ -15,6 +14,7 @@ import akka.testkit.{ EventFilter, TestEvent }
import akka.util.ByteString
import org.reactivestreams.Publisher
import org.testng.annotations.{ AfterClass, BeforeClass }
import akka.testkit.AkkaSpec
class FilePublisherTest extends AkkaPublisherVerification[ByteString] {

View file

@ -5,7 +5,7 @@ import akka.actor.Props;
import akka.stream.StreamTest;
import akka.stream.javadsl.AkkaJUnitActorSystemResource;
import akka.stream.javadsl.Source;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.testkit.JavaTestKit;
import org.junit.ClassRule;
import org.junit.Test;

View file

@ -6,7 +6,7 @@ import akka.stream.StreamTest;
import akka.stream.javadsl.AkkaJUnitActorSystemResource;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.testkit.JavaTestKit;
import org.junit.ClassRule;
import org.junit.Test;

View file

@ -9,7 +9,7 @@ import akka.stream.javadsl.AkkaJUnitActorSystemResource;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.javadsl.StreamConverters;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.stream.testkit.Utils;
import akka.util.ByteString;
import org.junit.ClassRule;

View file

@ -6,7 +6,7 @@ package akka.stream.javadsl;
import org.junit.rules.ExternalResource;
import akka.actor.ActorSystem;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.testkit.JavaTestKit;
import com.typesafe.config.Config;

View file

@ -14,7 +14,7 @@ import org.junit.Test;
import akka.stream.Attributes;
import akka.stream.StreamTest;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
public class AttributesTest extends StreamTest {

View file

@ -19,7 +19,7 @@ import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import akka.japi.Pair;
import akka.stream.*;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.stream.javadsl.GraphDSL.Builder;
import akka.japi.function.*;
import akka.util.ByteString;

View file

@ -11,7 +11,7 @@ import akka.stream.*;
import akka.stream.javadsl.GraphDSL.Builder;
import akka.stream.stage.*;
import akka.japi.function.*;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.testkit.JavaTestKit;
import akka.testkit.TestProbe;

View file

@ -15,7 +15,7 @@ import akka.stream.*;
import akka.stream.impl.ConstantFun;
import akka.stream.javadsl.GraphDSL.Builder;
import akka.stream.stage.*;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.stream.testkit.TestPublisher;
import akka.testkit.JavaTestKit;
import org.junit.ClassRule;

View file

@ -5,7 +5,7 @@ package akka.stream.javadsl;
import akka.NotUsed;
import akka.stream.StreamTest;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.util.ByteString;
import org.junit.ClassRule;
import org.junit.Test;

View file

@ -21,7 +21,7 @@ import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import akka.japi.function.Function2;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.testkit.JavaTestKit;
import static org.junit.Assert.*;

View file

@ -17,7 +17,7 @@ import akka.japi.pf.PFBuilder;
import akka.stream.*;
import akka.stream.impl.ConstantFun;
import akka.stream.stage.*;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.stream.testkit.TestPublisher;
import akka.testkit.JavaTestKit;
import org.junit.ClassRule;

View file

@ -25,7 +25,7 @@ import scala.runtime.BoxedUnit;
import akka.stream.*;
import akka.stream.javadsl.Tcp.*;
import akka.japi.function.*;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import akka.stream.testkit.TestUtils;
import akka.util.ByteString;
import akka.testkit.JavaTestKit;

View file

@ -8,7 +8,7 @@ import akka.stream.StreamTest;
import akka.stream.javadsl.AkkaJUnitActorSystemResource;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.testkit.AkkaSpec;
import akka.testkit.AkkaSpec;
import org.junit.ClassRule;
import org.junit.Ignore;

View file

@ -3,8 +3,7 @@ package akka.stream
import akka.actor.{ ActorSystem, Props }
import akka.stream.impl.{ StreamSupervisor, ActorMaterializerImpl }
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.testkit.AkkaSpec
import akka.testkit.{ TestActor, ImplicitSender }
import akka.testkit.{ AkkaSpec, TestActor, ImplicitSender }
import scala.concurrent.Await
import scala.concurrent.duration._

View file

@ -4,7 +4,7 @@
package akka.stream
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import org.scalactic.ConversionCheckedTripleEquals
import akka.stream.Attributes._
import akka.stream.Fusing.FusedGraph
@ -15,11 +15,10 @@ import scala.concurrent.duration._
import akka.stream.impl.fusing.GraphInterpreter
import akka.event.BusLogging
class FusingSpec extends AkkaSpec with ScalaFutures with ConversionCheckedTripleEquals {
class FusingSpec extends AkkaSpec {
final val Debug = false
implicit val materializer = ActorMaterializer()
implicit val patience = PatienceConfig(1.second)
def graph(async: Boolean) =
Source.unfold(1)(x Some(x -> x)).filter(_ % 2 == 1)

View file

@ -9,7 +9,7 @@ import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.TestEvent.Mute
import akka.testkit.{ EventFilter, ImplicitSender, TestProbe }
import akka.testkit.{ AkkaSpec, EventFilter, ImplicitSender, TestProbe }
import scala.annotation.tailrec
import scala.concurrent.duration._

View file

@ -7,7 +7,7 @@ import akka.actor.{ Actor, ActorRef, Props }
import akka.routing.{ ActorRefRoutee, RoundRobinRoutingLogic, Router }
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import org.reactivestreams.Subscription

View file

@ -10,6 +10,7 @@ import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.TestProbe
import org.reactivestreams.{ Publisher, Subscriber }
import akka.testkit.AkkaSpec
class FlowTimedSpec extends AkkaSpec with ScriptedTest {

View file

@ -3,7 +3,7 @@
*/
package akka.stream.impl
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.ActorMaterializerSettings
class FixedBufferSpec extends AkkaSpec {

View file

@ -4,7 +4,7 @@
package akka.stream.impl
import akka.stream.stage.GraphStageLogic.{ EagerTerminateOutput, EagerTerminateInput }
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream._
import akka.stream.Fusing.aggressive
import akka.stream.scaladsl._
@ -16,7 +16,7 @@ import org.scalactic.ConversionCheckedTripleEquals
import org.scalatest.concurrent.ScalaFutures
import scala.concurrent.duration.Duration
class GraphStageLogicSpec extends AkkaSpec with GraphInterpreterSpecKit with ConversionCheckedTripleEquals with ScalaFutures {
class GraphStageLogicSpec extends AkkaSpec with GraphInterpreterSpecKit {
implicit val materializer = ActorMaterializer()

View file

@ -4,7 +4,7 @@
package akka.stream.impl
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import org.reactivestreams.{ Subscription, Subscriber, Publisher }
import akka.stream._

View file

@ -5,15 +5,14 @@
package akka.stream.impl
import java.util.concurrent.TimeoutException
import akka.Done
import akka.stream.scaladsl._
import akka.stream.testkit.Utils._
import akka.stream.testkit.{ AkkaSpec, TestPublisher, TestSubscriber }
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
import akka.stream.{ ActorMaterializer, ClosedShape }
import scala.concurrent.duration._
import scala.concurrent.{ Await, Future }
import akka.testkit.AkkaSpec
class TimeoutsSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer()

View file

@ -4,16 +4,15 @@
package akka.stream.impl.fusing
import java.util.concurrent.CountDownLatch
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler }
import akka.stream.testkit.Utils._
import akka.stream.testkit.{ AkkaSpec, TestPublisher, TestSubscriber }
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
import akka.testkit.EventFilter
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.testkit.AkkaSpec
class ActorGraphInterpreterSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer()

View file

@ -2,7 +2,7 @@ package akka.stream.impl.fusing
import akka.stream.testkit.Utils.TE
import akka.testkit.EventFilter
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
class GraphInterpreterFailureModesSpec extends AkkaSpec with GraphInterpreterSpecKit {

View file

@ -3,7 +3,7 @@
*/
package akka.stream.impl.fusing
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.testkit.Utils._
class GraphInterpreterPortsSpec extends AkkaSpec with GraphInterpreterSpecKit {

View file

@ -6,7 +6,7 @@ package akka.stream.impl.fusing
import akka.NotUsed
import akka.stream.{ OverflowStrategy, Attributes }
import akka.stream.stage.AbstractStage.PushPullGraphStage
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.scaladsl.{ Merge, Broadcast, Balance, Zip }
import GraphInterpreter._

View file

@ -8,7 +8,7 @@ import akka.stream._
import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, Failed, GraphAssembly, UpstreamBoundaryStageLogic }
import akka.stream.stage.AbstractStage.PushPullGraphStage
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler, _ }
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.testkit.Utils.TE
import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly

View file

@ -5,7 +5,7 @@ package akka.stream.impl.fusing
import akka.stream.impl.ConstantFun
import akka.stream.stage._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.testkit.EventFilter
import akka.stream.Supervision

View file

@ -4,8 +4,8 @@
package akka.stream.impl.fusing
import akka.stream.impl.ConstantFun
import akka.stream.{ Supervision }
import akka.stream.testkit.AkkaSpec
import akka.stream.Supervision
import akka.testkit.AkkaSpec
class InterpreterStressSpec extends AkkaSpec with GraphInterpreterSpecKit {
import Supervision.stoppingDecider

View file

@ -10,7 +10,7 @@ import akka.stream.stage.PushPullStage
import akka.stream.stage.Stage
import akka.stream.stage.TerminationDirective
import akka.stream.stage.SyncDirective
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
object InterpreterSupervisionSpec {
val TE = new Exception("TEST") with NoStackTrace {

View file

@ -3,7 +3,7 @@
*/
package akka.stream.impl.fusing
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.util.ByteString
import akka.stream.stage._
import akka.stream.Supervision

View file

@ -7,7 +7,7 @@ import akka.actor.{ NoSerializationVerificationNeeded, ActorRef }
import akka.stream.scaladsl.{ Keep, Source }
import akka.stream.{ Attributes, Inlet, SinkShape, ActorMaterializer }
import akka.stream.stage.{ InHandler, AsyncCallback, GraphStageLogic, GraphStageWithMaterializedValue }
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.testkit.Utils._
import scala.concurrent.{ Await, Promise, Future }

View file

@ -4,7 +4,7 @@
package akka.stream.impl.fusing
import akka.stream.stage._
import akka.stream.testkit.AkkaSpec
import akka.testkit.AkkaSpec
import akka.stream.testkit.Utils.TE
import scala.concurrent.duration._

View file

@ -5,7 +5,6 @@ package akka.stream.io
import java.io.File
import java.nio.file.StandardOpenOption
import akka.actor.ActorSystem
import akka.stream.impl.ActorMaterializerImpl
import akka.stream.impl.StreamSupervisor
@ -18,10 +17,10 @@ import akka.stream.ActorMaterializerSettings
import akka.stream.ActorAttributes
import akka.util.ByteString
import akka.util.Timeout
import scala.collection.mutable.ListBuffer
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.testkit.AkkaSpec
class FileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
@ -155,4 +154,3 @@ class FileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
}
}

View file

@ -6,7 +6,6 @@ package akka.stream.io
import java.io.File
import java.io.FileWriter
import java.util.Random
import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
import akka.stream.ActorMaterializerSettings
@ -22,9 +21,9 @@ import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.TestSink
import akka.util.ByteString
import akka.util.Timeout
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.testkit.AkkaSpec
object FileSourceSpec {
final case class Settings(chunkSize: Int, readAhead: Int)

View file

@ -5,7 +5,6 @@ package akka.stream.io
import java.io.{ IOException, InputStream }
import java.util.concurrent.TimeoutException
import akka.actor.ActorSystem
import akka.stream._
import akka.stream.Attributes.inputBuffer
@ -15,14 +14,14 @@ import akka.stream.impl.{ ActorMaterializerImpl, StreamSupervisor }
import akka.stream.scaladsl.{ Keep, Source, StreamConverters }
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.TestSource
import akka.stream.testkit.{ AkkaSpec, GraphStageMessages, TestSinkStage }
import akka.stream.testkit.{ GraphStageMessages, TestSinkStage }
import akka.testkit.TestProbe
import akka.util.ByteString
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.concurrent.{ Await, Future }
import scala.util.control.NoStackTrace
import akka.testkit.AkkaSpec
class InputStreamSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
import system.dispatcher

View file

@ -12,11 +12,12 @@ import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
import akka.util.ByteString
import akka.testkit.AkkaSpec
import org.scalatest.concurrent.ScalaFutures
import scala.concurrent.Await
import scala.concurrent.duration._
class InputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) with ScalaFutures {
class InputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
implicit val materializer = ActorMaterializer(settings)

View file

@ -9,7 +9,7 @@ import akka.stream.scaladsl.{ Source, StreamConverters }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
import akka.testkit.TestProbe
import akka.testkit.{ AkkaSpec, TestProbe }
import akka.util.ByteString
import scala.concurrent.Await

View file

@ -5,7 +5,6 @@ package akka.stream.io
import java.io.IOException
import java.util.concurrent.TimeoutException
import akka.actor.ActorSystem
import akka.stream._
import akka.stream.Attributes.inputBuffer
@ -18,11 +17,11 @@ import akka.stream.testkit._
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.TestProbe
import akka.util.ByteString
import scala.concurrent.duration.Duration.Zero
import scala.concurrent.duration._
import scala.concurrent.{ Await, Future }
import scala.util.Random
import akka.testkit.AkkaSpec
class OutputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
import system.dispatcher

View file

@ -19,6 +19,7 @@ import scala.concurrent.{ Promise, Await }
import scala.concurrent.duration._
import java.net.BindException
import akka.testkit.EventFilter
import akka.testkit.AkkaSpec
class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.timeout = 2s") with TcpHelper {
var demand = 0L

View file

@ -20,7 +20,7 @@ import akka.stream.scaladsl._
import akka.stream.stage._
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.EventFilter
import akka.testkit.{ AkkaSpec, EventFilter }
import akka.util.ByteString
import javax.net.ssl._

View file

@ -12,6 +12,7 @@ import akka.stream.testkit.scaladsl._
import org.scalatest.concurrent.ScalaFutures
import org.scalactic.ConversionCheckedTripleEquals
import scala.concurrent.duration._
import akka.testkit.AkkaSpec
object ActorRefBackpressureSinkSpec {
val initMessage = "start"
@ -47,10 +48,9 @@ object ActorRefBackpressureSinkSpec {
}
class ActorRefBackpressureSinkSpec extends AkkaSpec with ScalaFutures with ConversionCheckedTripleEquals {
class ActorRefBackpressureSinkSpec extends AkkaSpec {
import ActorRefBackpressureSinkSpec._
implicit val mat = ActorMaterializer()
implicit val patience = PatienceConfig(2.second)
def createActor[T](c: Class[T]) =
system.actorOf(Props(c, testActor).withDispatcher("akka.test.stream-dispatcher"))

View file

@ -10,6 +10,7 @@ import akka.stream.testkit.scaladsl._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.testkit.AkkaSpec
object ActorRefSinkSpec {
case class Fw(ref: ActorRef) extends Actor {

View file

@ -10,6 +10,7 @@ import akka.stream.testkit.scaladsl._
import akka.stream.testkit.Utils._
import akka.actor.PoisonPill
import akka.actor.Status
import akka.testkit.AkkaSpec
class ActorRefSourceSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer()

View file

@ -16,6 +16,7 @@ import akka.stream.impl.SinkModule
import akka.stream.impl.StreamLayout.Module
import org.scalatest.concurrent.ScalaFutures
import akka.stream.impl.SinkholeSubscriber
import akka.testkit.AkkaSpec
object AttributesSpec {
@ -37,7 +38,7 @@ object AttributesSpec {
}
class AttributesSpec extends AkkaSpec with ScalaFutures {
class AttributesSpec extends AkkaSpec {
import AttributesSpec._
val settings = ActorMaterializerSettings(system)

View file

@ -4,7 +4,6 @@
package akka.stream.scaladsl
import akka.NotUsed
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.Utils._
import org.scalactic.ConversionCheckedTripleEquals
import akka.util.ByteString
@ -12,8 +11,9 @@ import akka.stream._
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.collection.immutable
import akka.testkit.AkkaSpec
class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
class BidiFlowSpec extends AkkaSpec {
import Attributes._
import GraphDSL.Implicits._

View file

@ -6,9 +6,10 @@ package akka.stream.scaladsl
import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
import akka.stream.ActorMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, TestSubscriber }
import akka.stream.testkit.TestSubscriber
import org.reactivestreams.Subscriber
import org.scalatest.Matchers
import akka.testkit.AkkaSpec
class FlowAppendSpec extends AkkaSpec with River {

View file

@ -8,6 +8,7 @@ import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.stream.{ OverflowStrategy, ActorMaterializer, ActorMaterializerSettings }
import akka.stream.testkit._
import akka.testkit.AkkaSpec
class FlowBatchSpec extends AkkaSpec {

View file

@ -6,6 +6,7 @@ package akka.stream.scaladsl
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
import akka.stream.testkit._
import scala.concurrent.duration._
import akka.testkit.AkkaSpec
class FlowBatchWeightedSpec extends AkkaSpec {

View file

@ -10,6 +10,7 @@ import akka.stream.{ BufferOverflowException, ActorMaterializer, ActorMaterializ
import akka.stream.testkit._
import akka.stream.testkit.scaladsl._
import akka.stream.testkit.Utils._
import akka.testkit.AkkaSpec
class FlowBufferSpec extends AkkaSpec {

Some files were not shown because too many files have changed in this diff Show more