!str #15121 MaterializerSettings now uses a config object
Resolves #15121
This commit is contained in:
parent
a25d0fcde9
commit
cc4fd5ca2c
76 changed files with 597 additions and 476 deletions
|
|
@ -6,6 +6,7 @@ package akka.http.model.japi;
|
|||
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.http.HttpExt;
|
||||
import akka.stream.MaterializerSettings;
|
||||
|
||||
public final class Http {
|
||||
private Http(){}
|
||||
|
|
@ -18,4 +19,8 @@ public final class Http {
|
|||
public static Object bind(String host, int port) {
|
||||
return Accessors$.MODULE$.Bind(host, port);
|
||||
}
|
||||
/** Create a Bind message to send to the Http Manager */
|
||||
public static Object bind(String host, int port, MaterializerSettings materializerSettings) {
|
||||
return Accessors$.MODULE$.Bind(host, port, materializerSettings);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -166,4 +166,4 @@ akka.http {
|
|||
# Fully qualified config path which holds the dispatcher configuration
|
||||
# to be used for the HttpManager.
|
||||
manager-dispatcher = "akka.actor.default-dispatcher"
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,13 +31,13 @@ object Http extends ExtensionKey[HttpExt] {
|
|||
localAddress: Option[InetSocketAddress],
|
||||
options: immutable.Traversable[Inet.SocketOption],
|
||||
settings: Option[ClientConnectionSettings],
|
||||
materializerSettings: MaterializerSettings) extends SetupOutgoingChannel
|
||||
materializerSettings: Option[MaterializerSettings]) extends SetupOutgoingChannel
|
||||
object Connect {
|
||||
def apply(host: String, port: Int = 80,
|
||||
localAddress: Option[InetSocketAddress] = None,
|
||||
options: immutable.Traversable[Inet.SocketOption] = Nil,
|
||||
settings: Option[ClientConnectionSettings] = None,
|
||||
materializerSettings: MaterializerSettings = MaterializerSettings()): Connect =
|
||||
materializerSettings: Option[MaterializerSettings] = None): Connect =
|
||||
apply(new InetSocketAddress(host, port), localAddress, options, settings, materializerSettings)
|
||||
}
|
||||
|
||||
|
|
@ -100,12 +100,12 @@ object Http extends ExtensionKey[HttpExt] {
|
|||
backlog: Int,
|
||||
options: immutable.Traversable[Inet.SocketOption],
|
||||
serverSettings: Option[ServerSettings],
|
||||
materializerSettings: MaterializerSettings)
|
||||
materializerSettings: Option[MaterializerSettings])
|
||||
object Bind {
|
||||
def apply(interface: String, port: Int = 80, backlog: Int = 100,
|
||||
options: immutable.Traversable[Inet.SocketOption] = Nil,
|
||||
serverSettings: Option[ServerSettings] = None,
|
||||
materializerSettings: MaterializerSettings = MaterializerSettings()): Bind =
|
||||
materializerSettings: Option[MaterializerSettings] = None): Bind =
|
||||
apply(new InetSocketAddress(interface, port), backlog, options, serverSettings, materializerSettings)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,17 +4,18 @@
|
|||
|
||||
package akka.http
|
||||
|
||||
import scala.util.{ Failure, Success }
|
||||
import scala.concurrent.duration._
|
||||
import akka.io.IO
|
||||
import akka.util.Timeout
|
||||
import akka.stream.io.StreamTcp
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.http.client._
|
||||
import akka.actor._
|
||||
import akka.http.client._
|
||||
import akka.http.server.{ HttpServerPipeline, ServerSettings }
|
||||
import akka.io.IO
|
||||
import akka.pattern.ask
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.io.StreamTcp
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.util.Timeout
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.{ Failure, Success }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -27,18 +28,19 @@ private[http] class HttpManager(httpSettings: HttpExt#Settings) extends Actor wi
|
|||
private[this] var clientPipelines = Map.empty[ClientConnectionSettings, HttpClientPipeline]
|
||||
|
||||
def receive = {
|
||||
case connect @ Http.Connect(remoteAddress, localAddress, options, settings, materializerSettings) ⇒
|
||||
case connect @ Http.Connect(remoteAddress, localAddress, options, clientConnectionSettings, materializerSettings) ⇒
|
||||
log.debug("Attempting connection to {}", remoteAddress)
|
||||
val commander = sender()
|
||||
val effectiveSettings = ClientConnectionSettings(settings)
|
||||
val tcpConnect = StreamTcp.Connect(materializerSettings, remoteAddress, localAddress, options,
|
||||
val effectiveSettings = ClientConnectionSettings(clientConnectionSettings)
|
||||
|
||||
val tcpConnect = StreamTcp.Connect(remoteAddress, localAddress, materializerSettings, options,
|
||||
effectiveSettings.connectingTimeout, effectiveSettings.idleTimeout)
|
||||
val askTimeout = Timeout(effectiveSettings.connectingTimeout + 5.seconds) // FIXME: how can we improve this?
|
||||
val tcpConnectionFuture = IO(StreamTcp)(context.system).ask(tcpConnect)(askTimeout)
|
||||
tcpConnectionFuture onComplete {
|
||||
case Success(tcpConn: StreamTcp.OutgoingTcpConnection) ⇒
|
||||
val pipeline = clientPipelines.getOrElse(effectiveSettings, {
|
||||
val pl = new HttpClientPipeline(effectiveSettings, FlowMaterializer(materializerSettings), log)
|
||||
val pl = new HttpClientPipeline(effectiveSettings, FlowMaterializer(), log)
|
||||
clientPipelines = clientPipelines.updated(effectiveSettings, pl)
|
||||
pl
|
||||
})
|
||||
|
|
@ -51,17 +53,17 @@ private[http] class HttpManager(httpSettings: HttpExt#Settings) extends Actor wi
|
|||
case x ⇒ throw new IllegalStateException("Unexpected response to `Connect` from StreamTcp: " + x)
|
||||
}
|
||||
|
||||
case Http.Bind(endpoint, backlog, options, settings, materializerSettings) ⇒
|
||||
case Http.Bind(endpoint, backlog, options, serverSettings, materializerSettings) ⇒
|
||||
log.debug("Binding to {}", endpoint)
|
||||
val commander = sender()
|
||||
val effectiveSettings = ServerSettings(settings)
|
||||
val tcpBind = StreamTcp.Bind(materializerSettings, endpoint, backlog, options)
|
||||
val effectiveSettings = ServerSettings(serverSettings)
|
||||
val tcpBind = StreamTcp.Bind(endpoint, materializerSettings, backlog, options)
|
||||
val askTimeout = Timeout(effectiveSettings.bindTimeout + 5.seconds) // FIXME: how can we improve this?
|
||||
val tcpServerBindingFuture = IO(StreamTcp)(context.system).ask(tcpBind)(askTimeout)
|
||||
tcpServerBindingFuture onComplete {
|
||||
case Success(StreamTcp.TcpServerBinding(localAddress, connectionStream)) ⇒
|
||||
log.info("Bound to {}", endpoint)
|
||||
implicit val materializer = FlowMaterializer(materializerSettings)
|
||||
implicit val materializer = FlowMaterializer()
|
||||
val httpServerPipeline = new HttpServerPipeline(effectiveSettings, materializer, log)
|
||||
val httpConnectionStream = Flow(connectionStream)
|
||||
.map(httpServerPipeline)
|
||||
|
|
|
|||
|
|
@ -4,9 +4,8 @@
|
|||
|
||||
package akka.http.model.japi
|
||||
|
||||
import akka.http.{ HttpExt, model }
|
||||
import akka.actor.ActorSystem
|
||||
import java.net.InetSocketAddress
|
||||
import akka.http.model
|
||||
import akka.stream.MaterializerSettings
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -22,5 +21,10 @@ private[http] object Accessors {
|
|||
/** INTERNAL API */
|
||||
private[http] def Uri(uri: model.Uri): Uri = JavaUri(uri)
|
||||
/** INTERNAL API */
|
||||
private[http] def Bind(host: String, port: Int): AnyRef = akka.http.Http.Bind(host, port)
|
||||
private[http] def Bind(host: String, port: Int): AnyRef =
|
||||
akka.http.Http.Bind(host, port, materializerSettings = None)
|
||||
|
||||
/** INTERNAL API */
|
||||
private[http] def Bind(host: String, port: Int, materializerSettings: MaterializerSettings): AnyRef =
|
||||
akka.http.Http.Bind(host, port, materializerSettings = Some(materializerSettings))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,8 +24,7 @@ public abstract class JavaTestServer {
|
|||
public static void main(String[] args) throws IOException, InterruptedException {
|
||||
ActorSystem system = ActorSystem.create();
|
||||
|
||||
MaterializerSettings settings = MaterializerSettings.create();
|
||||
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
||||
final FlowMaterializer materializer = FlowMaterializer.create(system);
|
||||
|
||||
ActorRef httpManager = Http.get(system).manager();
|
||||
Future<Object> binding = ask(httpManager, Http.bind("localhost", 8080), 1000);
|
||||
|
|
|
|||
|
|
@ -1,2 +1,2 @@
|
|||
# override strange reference.conf setting in akka-stream test scope
|
||||
akka.actor.default-mailbox.mailbox-type = akka.dispatch.UnboundedMailbox
|
||||
akka.actor.default-mailbox.mailbox-type = akka.dispatch.UnboundedMailbox
|
||||
|
|
|
|||
|
|
@ -4,29 +4,31 @@
|
|||
|
||||
package akka.http
|
||||
|
||||
import java.io.{ BufferedReader, BufferedWriter, InputStreamReader, OutputStreamWriter }
|
||||
import java.net.Socket
|
||||
import java.io.{ InputStreamReader, BufferedReader, OutputStreamWriter, BufferedWriter }
|
||||
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
|
||||
import com.typesafe.config.{ ConfigFactory, Config }
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.testkit.TestProbe
|
||||
import akka.io.IO
|
||||
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
||||
import akka.stream.testkit.StreamTestKit
|
||||
import akka.stream.impl.SynchronousPublisherFromIterable
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.http.server.ServerSettings
|
||||
import akka.http.client.ClientConnectionSettings
|
||||
import akka.http.model._
|
||||
import akka.http.util._
|
||||
import headers._
|
||||
import HttpMethods._
|
||||
import HttpEntity._
|
||||
import HttpMethods._
|
||||
import TestUtils._
|
||||
import akka.http.server.ServerSettings
|
||||
import akka.http.util._
|
||||
import akka.io.IO
|
||||
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
||||
import akka.stream.impl.SynchronousPublisherFromIterable
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.stream.testkit.StreamTestKit
|
||||
import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe }
|
||||
import akka.testkit.TestProbe
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
import headers._
|
||||
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
||||
val testConf: Config = ConfigFactory.parseString("""
|
||||
|
|
@ -35,7 +37,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||
import system.dispatcher
|
||||
|
||||
val materializerSettings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
||||
val materializerSettings = MaterializerSettings(system)
|
||||
val materializer = FlowMaterializer(materializerSettings)
|
||||
|
||||
"The server-side HTTP infrastructure" should {
|
||||
|
|
@ -43,7 +45,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
"properly bind and unbind a server" in {
|
||||
val (hostname, port) = temporaryServerHostnameAndPort()
|
||||
val commander = TestProbe()
|
||||
commander.send(IO(Http), Http.Bind(hostname, port, materializerSettings = materializerSettings))
|
||||
commander.send(IO(Http), Http.Bind(hostname, port, materializerSettings = Some(materializerSettings)))
|
||||
|
||||
val Http.ServerBinding(localAddress, connectionStream) = commander.expectMsgType[Http.ServerBinding]
|
||||
localAddress.getHostName shouldEqual hostname
|
||||
|
|
@ -118,7 +120,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
val connectionStream: SubscriberProbe[Http.IncomingConnection] = {
|
||||
val commander = TestProbe()
|
||||
val settings = configOverrides.toOption.map(ServerSettings.apply)
|
||||
commander.send(IO(Http), Http.Bind(hostname, port, serverSettings = settings, materializerSettings = materializerSettings))
|
||||
commander.send(IO(Http), Http.Bind(hostname, port, serverSettings = settings, materializerSettings = Some(materializerSettings)))
|
||||
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]
|
||||
commander.expectMsgType[Http.ServerBinding].connectionStream.subscribe(probe)
|
||||
probe
|
||||
|
|
@ -127,7 +129,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
|
||||
def openNewClientConnection[T](settings: Option[ClientConnectionSettings] = None): (PublisherProbe[(HttpRequest, T)], SubscriberProbe[(HttpResponse, T)]) = {
|
||||
val commander = TestProbe()
|
||||
commander.send(IO(Http), Http.Connect(hostname, port, settings = settings, materializerSettings = materializerSettings))
|
||||
commander.send(IO(Http), Http.Connect(hostname, port, settings = settings, materializerSettings = Some(materializerSettings)))
|
||||
val connection = commander.expectMsgType[Http.OutgoingConnection]
|
||||
connection.remoteAddress.getPort shouldEqual port
|
||||
connection.remoteAddress.getHostName shouldEqual hostname
|
||||
|
|
|
|||
|
|
@ -4,18 +4,19 @@
|
|||
|
||||
package akka.http
|
||||
|
||||
import com.typesafe.config.{ ConfigFactory, Config }
|
||||
import scala.concurrent.Future
|
||||
import scala.util.{ Failure, Success }
|
||||
import scala.concurrent.duration._
|
||||
import akka.util.Timeout
|
||||
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.io.IO
|
||||
import akka.actor.ActorSystem
|
||||
import akka.pattern.ask
|
||||
import akka.http.model.HttpMethods._
|
||||
import akka.http.model._
|
||||
import HttpMethods._
|
||||
import akka.io.IO
|
||||
import akka.pattern.ask
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.util.Timeout
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.{ Failure, Success }
|
||||
|
||||
object TestClient extends App {
|
||||
val testConf: Config = ConfigFactory.parseString("""
|
||||
|
|
@ -23,9 +24,9 @@ object TestClient extends App {
|
|||
akka.log-dead-letters = off
|
||||
""")
|
||||
implicit val system = ActorSystem("ServerTest", testConf)
|
||||
import system.dispatcher
|
||||
import akka.http.TestClient.system.dispatcher
|
||||
|
||||
implicit val materializer = FlowMaterializer(MaterializerSettings())
|
||||
implicit val materializer = FlowMaterializer()
|
||||
implicit val askTimeout: Timeout = 500.millis
|
||||
val host = "spray.io"
|
||||
|
||||
|
|
|
|||
|
|
@ -30,7 +30,7 @@ object TestServer extends App {
|
|||
case _: HttpRequest ⇒ HttpResponse(404, entity = "Unknown resource!")
|
||||
}
|
||||
|
||||
implicit val materializer = FlowMaterializer(MaterializerSettings())
|
||||
implicit val materializer = FlowMaterializer()
|
||||
|
||||
implicit val askTimeout: Timeout = 500.millis
|
||||
val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080)
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
|
|||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||
import system.dispatcher
|
||||
|
||||
val materializer = FlowMaterializer(MaterializerSettings())
|
||||
val materializer = FlowMaterializer()
|
||||
override def afterAll() = system.shutdown()
|
||||
|
||||
"HttpEntity" - {
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
|||
import system.dispatcher
|
||||
|
||||
val BOLT = HttpMethods.register(HttpMethod.custom("BOLT", safe = false, idempotent = true, entityAccepted = true))
|
||||
val materializer = FlowMaterializer(MaterializerSettings())
|
||||
val materializer = FlowMaterializer()
|
||||
|
||||
"The request parsing logic should" - {
|
||||
"properly parse a request" - {
|
||||
|
|
|
|||
|
|
@ -34,7 +34,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
|||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||
import system.dispatcher
|
||||
|
||||
val materializer = FlowMaterializer(MaterializerSettings())
|
||||
val materializer = FlowMaterializer()
|
||||
val ServerOnTheMove = StatusCodes.registerCustom(331, "Server on the move")
|
||||
|
||||
"The response parsing logic should" - {
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
|
|||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||
import system.dispatcher
|
||||
|
||||
val materializer = FlowMaterializer(MaterializerSettings())
|
||||
val materializer = FlowMaterializer()
|
||||
|
||||
"The request preparation logic should" - {
|
||||
"properly render an unchunked" - {
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
|
|||
import system.dispatcher
|
||||
|
||||
val ServerOnTheMove = StatusCodes.registerCustom(330, "Server on the move")
|
||||
val materializer = FlowMaterializer(MaterializerSettings())
|
||||
val materializer = FlowMaterializer()
|
||||
|
||||
"The response preparation logic should properly render" - {
|
||||
"a response with no body" - {
|
||||
|
|
|
|||
|
|
@ -5,25 +5,21 @@
|
|||
package akka.http
|
||||
package server
|
||||
|
||||
import akka.http.model.HttpEntity.{ LastChunk, Chunk, ChunkStreamPart }
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.event.NoLogging
|
||||
import akka.http.model.headers.Host
|
||||
import akka.http.model.HttpEntity.{ Chunk, ChunkStreamPart, LastChunk }
|
||||
import akka.http.model._
|
||||
import akka.http.model.headers.Host
|
||||
import akka.http.util._
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.io.StreamTcp
|
||||
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
|
||||
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
||||
import akka.util.ByteString
|
||||
import org.scalatest._
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterAll with Inside {
|
||||
val materializerSettings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
||||
val materializer = FlowMaterializer(materializerSettings)
|
||||
val materializer = FlowMaterializer()
|
||||
|
||||
"The server implementation should" should {
|
||||
"deliver an empty request as soon as all headers are received" in new TestSetup {
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue