Skip to content

Commit

Permalink
Allow to compile akka-http-tests with Scala 3 with 3.0-migration
Browse files Browse the repository at this point in the history
  • Loading branch information
jchyb committed Apr 14, 2022
1 parent 493695f commit 983d026
Show file tree
Hide file tree
Showing 28 changed files with 77 additions and 53 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -244,7 +244,7 @@ class AkkaHttpServerLatencyMultiNodeSpec extends MultiNodeSpec(AkkaHttpServerLat
runOn(loadGenerator) {
info(s"${id} => running: $cmd")
import akka.pattern.ask
implicit val timeout = Timeout(30.minutes) // we don't want to timeout here
implicit val timeout: Timeout = Timeout(30.minutes) // we don't want to timeout here

val res = (loadGeneratorActor ? LoadGenCommand(cmd)).mapTo[LoadGenResults]
val results = Await.result(res, timeout.duration)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import akka.testkit.TestEvent._
import scala.concurrent.duration._
import akka.remote.testconductor.RoleName
import akka.actor.RootActorPath
import akka.event.{ Logging, LoggingAdapter }
import akka.event.{ Logging, LogSource, LoggingAdapter }

/**
* Configure the role names and participants of the test, including configuration settings.
Expand Down Expand Up @@ -261,13 +261,13 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
this(config.myself, ActorSystem(MultiNodeSpec.getCallerName(classOf[MultiNodeSpec]), ConfigFactory.load(config.config)),
config.roles, config.deployments)

val log: LoggingAdapter = Logging(system, this.getClass)
val log: LoggingAdapter = Logging(system, this.getClass)(LogSource.fromClass)

/**
* Enrich `.await()` onto all Awaitables, using remaining duration from the innermost
* enclosing `within` block or QueryTimeout.
*/
implicit def awaitHelper[T](w: Awaitable[T]) = new AwaitHelper(w)
implicit def awaitHelper[T](w: Awaitable[T]): AwaitHelper[T] = new AwaitHelper(w)
class AwaitHelper[T](w: Awaitable[T]) {
def await: T = Await.result(w, remainingOr(testConductor.Settings.QueryTimeout.duration))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ import scala.concurrent.duration._
/**
* Support trait allowing trivially recording perf metrics from [[MultiNodeSpec]]s
*/
private[akka] trait PerfFlamesSupport { _: MultiNodeSpec =>
private[akka] trait PerfFlamesSupport { multiNodeSpec: MultiNodeSpec =>

/**
* Runs `perf-java-flames` script on given node (JVM process).
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import scala.concurrent.duration._
class CustomMediaTypesSpec extends AkkaSpec with ScalaFutures
with Directives with RequestBuilding {

implicit val mat = ActorMaterializer()
implicit val mat: ActorMaterializer = ActorMaterializer()

"Http" should {
"find media types in a set if they differ in casing" in {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ import akka.http.scaladsl.model._
import akka.testkit.AkkaSpec

class FormDataSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer()
implicit val materializer: ActorMaterializer = ActorMaterializer()
import system.dispatcher

val formData = FormData(Map("surname" -> "Smith", "age" -> "42"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ object TestSingleRequest extends App {
akka.log-dead-letters = off
akka.stream.materializer.debug.fuzzing-mode = off
""")
implicit val system = ActorSystem("ServerTest", testConf)
implicit val materializer = ActorMaterializer()
implicit val system: ActorSystem = ActorSystem("ServerTest", testConf)
implicit val materializer: ActorMaterializer = ActorMaterializer()
import system.dispatcher

val url = StdIn.readLine("url? ")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,8 +70,8 @@ Consetetur sadipscing elitr, sed diam nonumy eirmod tempor invidunt ut labore et
voluptua. At vero eos et accusam et justo duo dolores et ea rebum. Stet clita kasd gubergren, no sea takimata sanctus
est Lorem ipsum dolor sit amet. Lorem ipsum dolor sit amet, consetetur sadipscing elitr, sed diam nonumy e""".replace("\r\n", "\n")

implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = ActorMaterializer()
implicit val system: ActorSystem = ActorSystem(getClass.getSimpleName)
implicit val materializer: ActorMaterializer = ActorMaterializer()

override def afterAll() = TestKit.shutdownActorSystem(system)
}
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,11 @@ import akka.http.scaladsl.unmarshalling.FromEntityUnmarshaller
import spray.json.{ JsValue, PrettyPrinter, JsonPrinter, DefaultJsonProtocol }

import scala.collection.immutable.ListMap
import spray.json.RootJsonFormat

class SprayJsonSupportSpec extends JsonSupportSpec {
object EmployeeJsonProtocol extends DefaultJsonProtocol {
implicit val employeeFormat = jsonFormat5(Employee.apply)
implicit val employeeFormat: RootJsonFormat[Employee] = jsonFormat5(Employee.apply)
}
import EmployeeJsonProtocol._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,13 +9,14 @@ import akka.http.scaladsl.model.StatusCodes._
import akka.http.scaladsl.model.headers.Accept
import akka.http.scaladsl.model.{ ContentTypes, MediaRanges, MediaTypes }
import akka.http.scaladsl.server.{ Route, RoutingSpec }
import spray.json.RootJsonFormat

class FromStatusCodeAndXYZMarshallerSpec extends RoutingSpec {
case class ErrorInfo(errorMessage: String)
// a somewhat arbitrary ErrorInfo marshaller that can either return a text or an application/json response
implicit val errorInfoMarshaller: ToEntityMarshaller[ErrorInfo] = {
import spray.json.DefaultJsonProtocol._
implicit val errorInfoFormat = jsonFormat1(ErrorInfo.apply _)
implicit val errorInfoFormat: RootJsonFormat[ErrorInfo] = jsonFormat1(ErrorInfo.apply _)
Marshaller.oneOf(
Marshaller.StringMarshaller.compose[ErrorInfo](_.errorMessage),
SprayJsonSupport.sprayJsonMarshaller(errorInfoFormat)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@ import org.scalatest.freespec.AnyFreeSpec
import org.scalatest.matchers.should.Matchers

class MarshallingSpec extends AnyFreeSpec with Matchers with BeforeAndAfterAll with MultipartMarshallers with MarshallingTestUtils {
implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = ActorMaterializer()
implicit val system: ActorSystem = ActorSystem(getClass.getSimpleName)
implicit val materializer: ActorMaterializer = ActorMaterializer()
import system.dispatcher

override val testConfig = ConfigFactory.load()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import akka.http.scaladsl.model
import model.HttpMethods._
import model.StatusCodes
import akka.testkit.EventFilter
import akka.http.scaladsl.server.util.ConstructFromTuple

object BasicRouteSpecs {
private[http] def defaultExnHandler500Error(message: String) = {
Expand Down Expand Up @@ -148,7 +149,7 @@ class BasicRouteSpecs extends RoutingSpec {
"extract one argument" in {
case class MyNumber(i: Int)

val abcPath = path("abc" / IntNumber).as(MyNumber)(echoComplete)
val abcPath = path("abc" / IntNumber).as(ConstructFromTuple.instance1(MyNumber))(echoComplete)

Get("/abc/5") ~> abcPath ~> check {
responseAs[String] shouldEqual "MyNumber(5)"
Expand All @@ -157,7 +158,7 @@ class BasicRouteSpecs extends RoutingSpec {
"extract two arguments" in {
case class Person(name: String, age: Int)

val personPath = path("person" / Segment / IntNumber).as(Person)(echoComplete)
val personPath = path("person" / Segment / IntNumber).as(ConstructFromTuple.instance2(Person))(echoComplete)

Get("/person/john/38") ~> personPath ~> check {
responseAs[String] shouldEqual "Person(john,38)"
Expand All @@ -168,7 +169,7 @@ class BasicRouteSpecs extends RoutingSpec {
require(i > 10)
}

val abcPath = path("abc" / IntNumber).as(MyValidNumber)(echoComplete)
val abcPath = path("abc" / IntNumber).as(ConstructFromTuple.instance1(MyValidNumber))(echoComplete)

Get("/abc/5") ~> abcPath ~> check {
rejection shouldBe a[ValidationRejection]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,9 @@ object ConnectionTestApp {
}
""")

implicit val system = ActorSystem("ConnectionTest", testConf)
implicit val system: ActorSystem = ActorSystem("ConnectionTest", testConf)
import system.dispatcher
implicit val materializer = ActorMaterializer()
implicit val materializer: ActorMaterializer = ActorMaterializer()

val clientFlow = Http().superPool[Int]()

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import scala.concurrent.duration._
import scala.util.{ Failure, Success, Try }
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
import akka.event.LogSource

abstract class DontLeakActorsOnFailingConnectionSpecs(poolImplementation: String)
extends AnyWordSpecLike with Matchers with BeforeAndAfterAll with WithLogCapturing {
Expand All @@ -37,10 +38,10 @@ abstract class DontLeakActorsOnFailingConnectionSpecs(poolImplementation: String

http.host-connection-pool.base-connection-backoff = 0 ms
}""").withFallback(ConfigFactory.load())
implicit val system = ActorSystem("DontLeakActorsOnFailingConnectionSpecs-" + poolImplementation, config)
implicit val materializer = ActorMaterializer()
implicit val system: ActorSystem = ActorSystem("DontLeakActorsOnFailingConnectionSpecs-" + poolImplementation, config)
implicit val materializer: ActorMaterializer = ActorMaterializer()

val log = Logging(system, getClass)
val log = Logging(system, getClass)(LogSource.fromClass)

"Http.superPool" should {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,10 @@ import akka.stream.scaladsl._
import akka.testkit._
import akka.util.ByteString
import org.scalatest.concurrent.ScalaFutures
import spray.json.{ RootJsonFormat, RootJsonWriter }

class EntityStreamingSpec extends RoutingSpec with ScalaFutures {
implicit override val patience = PatienceConfig(5.seconds.dilated(system), 200.millis)
implicit override val patience: PatienceConfig = PatienceConfig(5.seconds.dilated(system), 200.millis)

//#models
case class Tweet(uid: Int, txt: String)
Expand All @@ -35,8 +36,8 @@ class EntityStreamingSpec extends RoutingSpec with ScalaFutures {
extends akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport
with spray.json.DefaultJsonProtocol {

implicit val tweetFormat = jsonFormat2(Tweet.apply)
implicit val measurementFormat = jsonFormat2(Measurement.apply)
implicit val tweetFormat: RootJsonFormat[Tweet] = jsonFormat2(Tweet.apply)
implicit val measurementFormat: RootJsonFormat[Measurement] = jsonFormat2(Measurement.apply)
}

"spray-json-response-streaming" in {
Expand Down Expand Up @@ -278,7 +279,14 @@ class EntityStreamingSpec extends RoutingSpec with ScalaFutures {
.runFold(0) { (cnt, _) => cnt + 1 }

complete {
measurementsSubmitted.map(n => Map("msg" -> s"""Total metrics received: $n"""))
implicit val marshaller = // Scala 3 workaround for missing implicit conversion
Marshaller.futureMarshaller(
Marshaller.liftMarshaller(
sprayJsonMarshaller(mapFormat(StringJsonFormat, StringJsonFormat))
)
)

measurementsSubmitted.map((n: Int) => Map("msg" -> s"""Total metrics received: $n"""))
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,12 +41,12 @@ class SizeLimitSpec extends AnyWordSpec with Matchers with RequestBuilding with
akka.http.server.parsing.max-content-length = $maxContentLength
akka.http.routing.decode-max-size = $decodeMaxSize
""")
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
implicit val system: ActorSystem = ActorSystem(getClass.getSimpleName, testConf)
import system.dispatcher
implicit val materializer = ActorMaterializer()
implicit val materializer: ActorMaterializer = ActorMaterializer()
val random = new scala.util.Random(42)

implicit val defaultPatience = PatienceConfig(timeout = Span(2, Seconds), interval = Span(5, Millis))
implicit val defaultPatience: PatienceConfig = PatienceConfig(timeout = Span(2, Seconds), interval = Span(5, Millis))

"a normal route" should {
val route = path("noDirective") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,12 @@ object TcpLeakApp extends App {
akka.loglevel = DEBUG
akka.log-dead-letters = on
akka.io.tcp.trace-logging = on""")
implicit val system = ActorSystem("ServerTest", testConf)
implicit val fm = ActorMaterializer()
implicit val system: ActorSystem = ActorSystem("ServerTest", testConf)
implicit val fm: ActorMaterializer = ActorMaterializer()

import system.dispatcher

val tcpFlow = Tcp().outgoingConnection(new InetSocketAddress("127.0.0.1", 1234)).named("TCP-outgoingConnection")
val tcpFlow = Tcp(system).outgoingConnection(new InetSocketAddress("127.0.0.1", 1234)).named("TCP-outgoingConnection")
List
.fill(100)(
Source
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@ import akka.http.scaladsl.common.EntityStreamingSupport
import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
import scala.io.StdIn
import akka.http.scaladsl.common.JsonEntityStreamingSupport
import spray.json.RootJsonFormat

object TestServer extends App {
val testConf: Config = ConfigFactory.parseString("""
Expand All @@ -26,16 +28,16 @@ object TestServer extends App {
akka.stream.materializer.debug.fuzzing-mode = off
""")

implicit val system = ActorSystem("ServerTest", testConf)
implicit val system: ActorSystem = ActorSystem("ServerTest", testConf)
implicit val ec: ExecutionContext = system.dispatcher
implicit val materializer = ActorMaterializer()
implicit val materializer: ActorMaterializer = ActorMaterializer()

import spray.json.DefaultJsonProtocol._
import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport._
final case class Tweet(message: String)
implicit val tweetFormat = jsonFormat1(Tweet)
implicit val tweetFormat: RootJsonFormat[Tweet] = jsonFormat1(Tweet)

implicit val jsonStreaming = EntityStreamingSupport.json()
implicit val jsonStreaming: JsonEntityStreamingSupport = EntityStreamingSupport.json()

import ScalaXmlSupport._
import Directives._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import scala.concurrent.duration._

class CodingDirectivesSpec extends RoutingSpec with Inside {

implicit val routeTestTimeout = RouteTestTimeout(3.seconds.dilated)
implicit val routeTestTimeout: RouteTestTimeout = RouteTestTimeout(3.seconds.dilated)

val echoRequestContent: Route = { ctx => ctx.complete(ctx.request.entity.dataBytes.utf8String) }

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import akka.testkit._
class FileAndResourceDirectivesSpec extends RoutingSpec with Inspectors with Inside {

// operations touch files, can be randomly hit by slowness
implicit val routeTestTimeout = RouteTestTimeout(3.seconds.dilated)
implicit val routeTestTimeout: RouteTestTimeout = RouteTestTimeout(3.seconds.dilated)

// need to serve from the src directory, when sbt copies the resource directory over to the
// target directory it will resolve symlinks in the process
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import scala.concurrent.duration._
class FileUploadDirectivesSpec extends RoutingSpec with Eventually {

// tests touches filesystem, so reqs may take longer than the default of 1.second to complete
implicit val routeTimeout = RouteTestTimeout(6.seconds.dilated)
implicit val routeTimeout: RouteTestTimeout = RouteTestTimeout(6.seconds.dilated)

"the storeUploadedFile directive" should {
val data = s"<int>${"42" * 1000000}</int>" // ~2MB of data
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,11 @@ import akka.http.scaladsl.unmarshalling.Unmarshaller.HexInt
import akka.http.scaladsl.model._
import akka.http.scaladsl.model.MediaTypes._
import akka.http.impl.util.BenchUtils
import akka.http.scaladsl.unmarshalling.FromEntityUnmarshaller
import scala.xml.NodeSeq

class FormFieldDirectivesSpec extends RoutingSpec {
implicit val nodeSeqUnmarshaller =
implicit val nodeSeqUnmarshaller: FromEntityUnmarshaller[NodeSeq] =
ScalaXmlSupport.nodeSeqUnmarshaller(`text/xml`, `text/html`, `text/plain`)

val nodeSeq: xml.NodeSeq = <b>yes</b>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -225,7 +225,7 @@ class MarshallingDirectivesSpec extends RoutingSpec with Inside {
"The marshalling infrastructure for JSON" should {
import spray.json._
case class Foo(name: String)
implicit val fooFormat = jsonFormat1(Foo)
implicit val fooFormat: RootJsonFormat[Foo] = jsonFormat1(Foo)
val foo = Foo("Hällö")

"render JSON with UTF-8 encoding if no `Accept-Charset` request header is present" in {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import org.scalatest.Inside
import akka.http.scaladsl.unmarshalling.Unmarshaller, Unmarshaller._
import akka.http.scaladsl.model.StatusCodes
import org.scalatest.freespec.AnyFreeSpec
import akka.http.scaladsl.server.util.ConstructFromTuple

class ParameterDirectivesSpec extends AnyFreeSpec with GenericRoutingSpec with Inside {
"when used with 'as[Int]' the parameter directive should" - {
Expand Down Expand Up @@ -273,7 +274,7 @@ class ParameterDirectivesSpec extends AnyFreeSpec with GenericRoutingSpec with I
"extract a parameter value as Case Class" in {
case class Color(red: Int, green: Int, blue: Int)
Get("/?red=90&green=50&blue=0") ~> {
parameters("red".as[Int], "green".as[Int], "blue".as[Int]).as(Color) { color =>
parameters("red".as[Int], "green".as[Int], "blue".as[Int]).as(ConstructFromTuple.instance3(Color)) { color =>
complete(s"${color.red} ${color.green} ${color.blue}")
}
} ~> check { responseAs[String] shouldEqual "90 50 0" }
Expand All @@ -285,7 +286,7 @@ class ParameterDirectivesSpec extends AnyFreeSpec with GenericRoutingSpec with I
require(0 <= blue && blue <= 255)
}
Get("/?red=500&green=0&blue=0") ~> {
parameters("red".as[Int], "green".as[Int], "blue".as[Int]).as(Color) { color =>
parameters("red".as[Int], "green".as[Int], "blue".as[Int]).as(ConstructFromTuple.instance3(Color)) { color =>
complete(s"${color.red} ${color.green} ${color.blue}")
}
} ~> check {
Expand All @@ -299,7 +300,7 @@ class ParameterDirectivesSpec extends AnyFreeSpec with GenericRoutingSpec with I
require(0 <= blue && blue <= 255)
}
Get("/?red=0&green=0&blue=0") ~> {
parameters("red".as[Int], "green".as[Int], "blue".as[Int]).as(Color) { _ =>
parameters("red".as[Int], "green".as[Int], "blue".as[Int]).as(ConstructFromTuple.instance3(Color)) { _ =>
throw new IllegalArgumentException
}
} ~> check {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,14 @@ class RouteDirectivesSpec extends AnyWordSpec with GenericRoutingSpec {
case AlreadyRegistered =>
import spray.json.DefaultJsonProtocol._
import SprayJsonSupport._
StatusCodes.BadRequest -> Map("error" -> "User already Registered")
implicit val marshaller = // Scala 3 workaround for missing implicit conversion
Marshaller.fromStatusCodeAndValue[StatusCodes.ClientError, Map[String, String]](
a => a,
sprayJsonMarshaller(mapFormat(StringJsonFormat, StringJsonFormat)
)
)

(StatusCodes.BadRequest -> Map("error" -> "User already Registered"))
}
}
}
Expand Down
Loading

0 comments on commit 983d026

Please sign in to comment.